You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2017/11/01 15:39:55 UTC

[01/10] hbase git commit: Revert "HBASE-19053 Split out o.a.h.h.http from hbase-server into a separate module"

Repository: hbase
Updated Branches:
  refs/heads/HBASE-19124 [created] d8a7b7b39


http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..234bd7a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/ssl/KeyStoreTestUtil.java
@@ -0,0 +1,342 @@
+/**
+ * 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.InvalidKeyException;
+import java.security.Key;
+import java.security.KeyPair;
+import java.security.KeyPairGenerator;
+import java.security.KeyStore;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+import java.security.SecureRandom;
+import java.security.SignatureException;
+import java.security.cert.Certificate;
+import java.security.cert.CertificateEncodingException;
+import java.security.cert.X509Certificate;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.security.auth.x500.X500Principal;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.bouncycastle.x509.X509V1CertificateGenerator;
+
+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.
+   *
+   * @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
+   */
+  public static X509Certificate generateCertificate(String dn, KeyPair pair, int days, String algorithm) 
+      throws CertificateEncodingException, InvalidKeyException, IllegalStateException, 
+      NoSuchProviderException, NoSuchAlgorithmException, SignatureException {
+    Date from = new Date();
+    Date to = new Date(from.getTime() + days * 86400000l);
+    BigInteger sn = new BigInteger(64, new SecureRandom());
+    KeyPair keyPair = pair;
+    X509V1CertificateGenerator certGen = new X509V1CertificateGenerator();
+    X500Principal  dnName = new X500Principal(dn);
+
+    certGen.setSerialNumber(sn);
+    certGen.setIssuerDN(dnName);
+    certGen.setNotBefore(from);
+    certGen.setNotAfter(to);
+    certGen.setSubjectDN(dnName);
+    certGen.setPublicKey(keyPair.getPublic());
+    certGen.setSignatureAlgorithm(algorithm);
+    X509Certificate cert = certGen.generate(pair.getPrivate());
+    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<>();
+
+    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/d8a7b7b3/hbase-shaded/hbase-shaded-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/hbase-shaded-mapreduce/pom.xml b/hbase-shaded/hbase-shaded-mapreduce/pom.xml
index cfcc357..1f2a2bf 100644
--- a/hbase-shaded/hbase-shaded-mapreduce/pom.xml
+++ b/hbase-shaded/hbase-shaded-mapreduce/pom.xml
@@ -142,10 +142,6 @@
                 <artifactId>javax.servlet.jsp</artifactId>
               </exclusion>
               <exclusion>
-                <groupId>org.glassfish.jersey.core</groupId>
-                <artifactId>jersey-server</artifactId>
-              </exclusion>
-              <exclusion>
                 <groupId>org.glassfish.jersey.containers</groupId>
                 <artifactId>jersey-container-servlet-core</artifactId>
               </exclusion>

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2a9b8c9..ddf6486 100755
--- a/pom.xml
+++ b/pom.xml
@@ -66,7 +66,6 @@
     <module>hbase-replication</module>
     <module>hbase-mapreduce</module>
     <module>hbase-resource-bundle</module>
-    <module>hbase-http</module>
     <module>hbase-server</module>
     <module>hbase-thrift</module>
     <module>hbase-shell</module>
@@ -1646,18 +1645,6 @@
         <version>${project.version}</version>
       </dependency>
       <dependency>
-        <artifactId>hbase-http</artifactId>
-        <groupId>org.apache.hbase</groupId>
-        <version>${project.version}</version>
-      </dependency>
-      <dependency>
-        <artifactId>hbase-http</artifactId>
-        <groupId>org.apache.hbase</groupId>
-        <version>${project.version}</version>
-        <type>test-jar</type>
-        <scope>test</scope>
-      </dependency>
-      <dependency>
         <artifactId>hbase-server</artifactId>
         <groupId>org.apache.hbase</groupId>
         <version>${project.version}</version>
@@ -2001,11 +1988,6 @@
         <version>${jersey.version}</version>
       </dependency>
       <dependency>
-        <groupId>org.glassfish.jersey.core</groupId>
-        <artifactId>jersey-server</artifactId>
-        <version>${jersey.version}</version>
-      </dependency>
-      <dependency>
         <!--This lib has JspC in it. Needed precompiling jsps in hbase-rest, etc.-->
         <groupId>org.glassfish.web</groupId>
         <artifactId>javax.servlet.jsp</artifactId>
@@ -2068,11 +2050,6 @@
       </dependency>
       <dependency>
         <groupId>org.apache.kerby</groupId>
-        <artifactId>kerb-core</artifactId>
-        <version>${kerby.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.apache.kerby</groupId>
         <artifactId>kerb-client</artifactId>
         <version>${kerby.version}</version>
       </dependency>


[05/10] hbase git commit: Revert "HBASE-19053 Split out o.a.h.h.http from hbase-server into a separate module"

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java
deleted file mode 100644
index 4fad031..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java
+++ /dev/null
@@ -1,258 +0,0 @@
-/*
- * 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.File;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.net.HttpURLConnection;
-import java.net.URL;
-import java.security.Principal;
-import java.security.PrivilegedExceptionAction;
-import java.util.Set;
-
-import javax.security.auth.Subject;
-import javax.security.auth.kerberos.KerberosTicket;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.http.TestHttpServer.EchoServlet;
-import org.apache.hadoop.hbase.http.resource.JerseyResource;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.security.authentication.util.KerberosName;
-import org.apache.http.HttpHost;
-import org.apache.http.HttpResponse;
-import org.apache.http.auth.AuthSchemeProvider;
-import org.apache.http.auth.AuthScope;
-import org.apache.http.auth.KerberosCredentials;
-import org.apache.http.client.HttpClient;
-import org.apache.http.client.config.AuthSchemes;
-import org.apache.http.client.methods.HttpGet;
-import org.apache.http.client.protocol.HttpClientContext;
-import org.apache.http.config.Lookup;
-import org.apache.http.config.RegistryBuilder;
-import org.apache.http.entity.ByteArrayEntity;
-import org.apache.http.entity.ContentType;
-import org.apache.http.impl.auth.SPNegoSchemeFactory;
-import org.apache.http.impl.client.BasicCredentialsProvider;
-import org.apache.http.impl.client.HttpClients;
-import org.apache.http.util.EntityUtils;
-import org.apache.kerby.kerberos.kerb.KrbException;
-import org.apache.kerby.kerberos.kerb.client.JaasKrbUtil;
-import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer;
-import org.ietf.jgss.GSSCredential;
-import org.ietf.jgss.GSSManager;
-import org.ietf.jgss.GSSName;
-import org.ietf.jgss.Oid;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-/**
- * Test class for SPNEGO authentication on the HttpServer. Uses Kerby's MiniKDC and Apache
- * HttpComponents to verify that a simple Servlet is reachable via SPNEGO and unreachable w/o.
- */
-@Category({MiscTests.class, SmallTests.class})
-public class TestSpnegoHttpServer extends HttpServerFunctionalTest {
-  private static final Log LOG = LogFactory.getLog(TestSpnegoHttpServer.class);
-  private static final String KDC_SERVER_HOST = "localhost";
-  private static final String CLIENT_PRINCIPAL = "client";
-
-  private static HttpServer server;
-  private static URL baseUrl;
-  private static SimpleKdcServer kdc;
-  private static File infoServerKeytab;
-  private static File clientKeytab;
-
-  @BeforeClass
-  public static void setupServer() throws Exception {
-    final String serverPrincipal = "HTTP/" + KDC_SERVER_HOST;
-    final File target = new File(System.getProperty("user.dir"), "target");
-    assertTrue(target.exists());
-
-    kdc = buildMiniKdc();
-    kdc.start();
-
-    File keytabDir = new File(target, TestSpnegoHttpServer.class.getSimpleName()
-        + "_keytabs");
-    if (keytabDir.exists()) {
-      deleteRecursively(keytabDir);
-    }
-    keytabDir.mkdirs();
-
-    infoServerKeytab = new File(keytabDir, serverPrincipal.replace('/', '_') + ".keytab");
-    clientKeytab = new File(keytabDir, CLIENT_PRINCIPAL + ".keytab");
-
-    setupUser(kdc, clientKeytab, CLIENT_PRINCIPAL);
-    setupUser(kdc, infoServerKeytab, serverPrincipal);
-
-    Configuration conf = buildSpnegoConfiguration(serverPrincipal, infoServerKeytab);
-
-    server = createTestServerWithSecurity(conf);
-    server.addServlet("echo", "/echo", EchoServlet.class);
-    server.addJerseyResourcePackage(JerseyResource.class.getPackage().getName(), "/jersey/*");
-    server.start();
-    baseUrl = getServerURL(server);
-
-    LOG.info("HTTP server started: "+ baseUrl);
-  }
-
-  @AfterClass
-  public static void stopServer() throws Exception {
-    try {
-      if (null != server) {
-        server.stop();
-      }
-    } catch (Exception e) {
-      LOG.info("Failed to stop info server", e);
-    }
-    try {
-      if (null != kdc) {
-        kdc.stop();
-      }
-    } catch (Exception e) {
-      LOG.info("Failed to stop mini KDC", e);
-    }
-  }
-
-  private static void setupUser(SimpleKdcServer kdc, File keytab, String principal)
-      throws KrbException {
-    kdc.createPrincipal(principal);
-    kdc.exportPrincipal(principal, keytab);
-  }
-
-  private static SimpleKdcServer buildMiniKdc() throws Exception {
-    SimpleKdcServer kdc = new SimpleKdcServer();
-
-    final File target = new File(System.getProperty("user.dir"), "target");
-    File kdcDir = new File(target, TestSpnegoHttpServer.class.getSimpleName());
-    if (kdcDir.exists()) {
-      deleteRecursively(kdcDir);
-    }
-    kdcDir.mkdirs();
-    kdc.setWorkDir(kdcDir);
-
-    kdc.setKdcHost(KDC_SERVER_HOST);
-    int kdcPort = getFreePort();
-    kdc.setAllowTcp(true);
-    kdc.setAllowUdp(false);
-    kdc.setKdcTcpPort(kdcPort);
-
-    LOG.info("Starting KDC server at " + KDC_SERVER_HOST + ":" + kdcPort);
-
-    kdc.init();
-
-    return kdc;
-  }
-
-  private static Configuration buildSpnegoConfiguration(String serverPrincipal, File
-      serverKeytab) {
-    Configuration conf = new Configuration();
-    KerberosName.setRules("DEFAULT");
-
-    conf.setInt(HttpServer.HTTP_MAX_THREADS, TestHttpServer.MAX_THREADS);
-
-    // Enable Kerberos (pre-req)
-    conf.set("hbase.security.authentication", "kerberos");
-    conf.set(HttpServer.HTTP_UI_AUTHENTICATION, "kerberos");
-    conf.set(HttpServer.HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_KEY, serverPrincipal);
-    conf.set(HttpServer.HTTP_SPNEGO_AUTHENTICATION_KEYTAB_KEY, serverKeytab.getAbsolutePath());
-
-    return conf;
-  }
-
-  @Test
-  public void testUnauthorizedClientsDisallowed() throws IOException {
-    URL url = new URL(getServerURL(server), "/echo?a=b");
-    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
-    assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode());
-  }
-
-  @Test
-  public void testAllowedClient() throws Exception {
-    // Create the subject for the client
-    final Subject clientSubject = JaasKrbUtil.loginUsingKeytab(CLIENT_PRINCIPAL, clientKeytab);
-    final Set<Principal> clientPrincipals = clientSubject.getPrincipals();
-    // Make sure the subject has a principal
-    assertFalse(clientPrincipals.isEmpty());
-
-    // Get a TGT for the subject (might have many, different encryption types). The first should
-    // be the default encryption type.
-    Set<KerberosTicket> privateCredentials =
-            clientSubject.getPrivateCredentials(KerberosTicket.class);
-    assertFalse(privateCredentials.isEmpty());
-    KerberosTicket tgt = privateCredentials.iterator().next();
-    assertNotNull(tgt);
-
-    // The name of the principal
-    final String principalName = clientPrincipals.iterator().next().getName();
-
-    // Run this code, logged in as the subject (the client)
-    HttpResponse resp = Subject.doAs(clientSubject,
-        new PrivilegedExceptionAction<HttpResponse>() {
-      @Override
-      public HttpResponse run() throws Exception {
-        // Logs in with Kerberos via GSS
-        GSSManager gssManager = GSSManager.getInstance();
-        // jGSS Kerberos login constant
-        Oid oid = new Oid("1.2.840.113554.1.2.2");
-        GSSName gssClient = gssManager.createName(principalName, GSSName.NT_USER_NAME);
-        GSSCredential credential = gssManager.createCredential(gssClient,
-            GSSCredential.DEFAULT_LIFETIME, oid, GSSCredential.INITIATE_ONLY);
-
-        HttpClientContext context = HttpClientContext.create();
-        Lookup<AuthSchemeProvider> authRegistry = RegistryBuilder.<AuthSchemeProvider>create()
-            .register(AuthSchemes.SPNEGO, new SPNegoSchemeFactory(true, true))
-            .build();
-
-        HttpClient client = HttpClients.custom().setDefaultAuthSchemeRegistry(authRegistry).build();
-        BasicCredentialsProvider credentialsProvider = new BasicCredentialsProvider();
-        credentialsProvider.setCredentials(AuthScope.ANY, new KerberosCredentials(credential));
-
-        URL url = new URL(getServerURL(server), "/echo?a=b");
-        context.setTargetHost(new HttpHost(url.getHost(), url.getPort()));
-        context.setCredentialsProvider(credentialsProvider);
-        context.setAuthSchemeRegistry(authRegistry);
-
-        HttpGet get = new HttpGet(url.toURI());
-        return client.execute(get, context);
-      }
-    });
-
-    assertNotNull(resp);
-    assertEquals(HttpURLConnection.HTTP_OK, resp.getStatusLine().getStatusCode());
-    assertEquals("a:b", EntityUtils.toString(resp.getEntity()).trim());
-  }
-
-  @Test(expected = IllegalArgumentException.class)
-  public void testMissingConfigurationThrowsException() throws Exception {
-    Configuration conf = new Configuration();
-    conf.setInt(HttpServer.HTTP_MAX_THREADS, TestHttpServer.MAX_THREADS);
-    // Enable Kerberos (pre-req)
-    conf.set("hbase.security.authentication", "kerberos");
-    // Intentionally skip keytab and principal
-
-    HttpServer customServer = createTestServerWithSecurity(conf);
-    customServer.addServlet("echo", "/echo", EchoServlet.class);
-    customServer.addJerseyResourcePackage(JerseyResource.class.getPackage().getName(), "/jersey/*");
-    customServer.start();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/conf/TestConfServlet.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/conf/TestConfServlet.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/conf/TestConfServlet.java
deleted file mode 100644
index 8bd1e6d..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/conf/TestConfServlet.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/**
- * 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.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.eclipse.jetty.util.ajax.JSON;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-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({MiscTests.class, 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());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
deleted file mode 100644
index 484162a..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/*
- * 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.HttpURLConnection;
-import java.net.URL;
-import java.net.URLEncoder;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import javax.servlet.http.HttpServletResponse;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.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({MiscTests.class, 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 {
-    // Eclipse doesn't pick this up correctly from the plugin
-    // configuration in the pom.
-    System.setProperty(HttpServerFunctionalTest.TEST_BUILD_WEBAPPS, "target/test-classes/webapps");
-    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);
-
-  }
-
-  @Test
-  public void testDisallowedJSONPCallback() throws Exception {
-    String callback = "function(){alert('bigproblems!')};foo";
-    URL url = new URL(
-        baseUrl, "/jmx?qry=java.lang:type=Memory&callback="+URLEncoder.encode(callback, "UTF-8"));
-    HttpURLConnection cnxn = (HttpURLConnection) url.openConnection();
-    assertEquals(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, cnxn.getResponseCode());
-  }
-
-  @Test
-  public void testUnderscoresInJSONPCallback() throws Exception {
-    String callback = "my_function";
-    URL url = new URL(
-        baseUrl, "/jmx?qry=java.lang:type=Memory&callback="+URLEncoder.encode(callback, "UTF-8"));
-    HttpURLConnection cnxn = (HttpURLConnection) url.openConnection();
-    assertEquals(HttpServletResponse.SC_OK, cnxn.getResponseCode());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/lib/TestStaticUserWebFilter.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/lib/TestStaticUserWebFilter.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/lib/TestStaticUserWebFilter.java
deleted file mode 100644
index 3adca50..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/lib/TestStaticUserWebFilter.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/**
- * 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.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.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({MiscTests.class, 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/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java
deleted file mode 100644
index e14e3b4..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/**
-* 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 static org.junit.Assert.assertTrue;
-
-import java.io.*;
-import java.net.*;
-
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.http.HttpServer;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.commons.logging.*;
-import org.apache.commons.logging.impl.*;
-import org.apache.log4j.*;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({MiscTests.class, SmallTests.class})
-public class TestLogLevel {
-  static final PrintStream out = System.out;
-
-  @Test (timeout=60000)
-  @SuppressWarnings("deprecation")
-  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 = null;
-      try {
-        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);
-        try (BufferedReader in = new BufferedReader(new InputStreamReader(url.openStream()))) {
-          for(String line; (line = in.readLine()) != null; out.println(line));
-        }
-        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()));
-      } finally {
-        if (server != null) {
-          server.stop();
-        }
-      }
-    }
-    else {
-      out.println(testlog.getClass() + " not tested.");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/resource/JerseyResource.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/resource/JerseyResource.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/resource/JerseyResource.java
deleted file mode 100644
index bf0e609..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/resource/JerseyResource.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
- * 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.eclipse.jetty.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 {
-  private 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<>();
-    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/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/ssl/KeyStoreTestUtil.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/ssl/KeyStoreTestUtil.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/ssl/KeyStoreTestUtil.java
deleted file mode 100644
index 35d6236..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/ssl/KeyStoreTestUtil.java
+++ /dev/null
@@ -1,342 +0,0 @@
-/**
- * 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.InvalidKeyException;
-import java.security.Key;
-import java.security.KeyPair;
-import java.security.KeyPairGenerator;
-import java.security.KeyStore;
-import java.security.NoSuchAlgorithmException;
-import java.security.NoSuchProviderException;
-import java.security.SecureRandom;
-import java.security.SignatureException;
-import java.security.cert.Certificate;
-import java.security.cert.CertificateEncodingException;
-import java.security.cert.X509Certificate;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Map;
-
-import javax.security.auth.x500.X500Principal;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory;
-import org.apache.hadoop.security.ssl.SSLFactory;
-import org.bouncycastle.x509.X509V1CertificateGenerator;
-
-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.
-   *
-   * @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
-   */
-  public static X509Certificate generateCertificate(String dn, KeyPair pair, int days, String algorithm)
-      throws CertificateEncodingException, InvalidKeyException, IllegalStateException,
-      NoSuchProviderException, NoSuchAlgorithmException, SignatureException {
-    Date from = new Date();
-    Date to = new Date(from.getTime() + days * 86400000l);
-    BigInteger sn = new BigInteger(64, new SecureRandom());
-    KeyPair keyPair = pair;
-    X509V1CertificateGenerator certGen = new X509V1CertificateGenerator();
-    X500Principal  dnName = new X500Principal(dn);
-
-    certGen.setSerialNumber(sn);
-    certGen.setIssuerDN(dnName);
-    certGen.setNotBefore(from);
-    certGen.setNotAfter(to);
-    certGen.setSubjectDN(dnName);
-    certGen.setPublicKey(keyPair.getPublic());
-    certGen.setSignatureAlgorithm(algorithm);
-    X509Certificate cert = certGen.generate(pair.getPrivate());
-    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<>();
-
-    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/d8a7b7b3/hbase-http/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/resources/log4j.properties b/hbase-http/src/test/resources/log4j.properties
deleted file mode 100644
index c322699..0000000
--- a/hbase-http/src/test/resources/log4j.properties
+++ /dev/null
@@ -1,68 +0,0 @@
-# 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.
-
-# Define some default values that can be overridden by system properties
-hbase.root.logger=INFO,console
-hbase.log.dir=.
-hbase.log.file=hbase.log
-
-# Define the root logger to the system property "hbase.root.logger".
-log4j.rootLogger=${hbase.root.logger}
-
-# Logging Threshold
-log4j.threshold=ALL
-
-#
-# Daily Rolling File Appender
-#
-log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.DRFA.File=${hbase.log.dir}/${hbase.log.file}
-
-# Rollver at midnight
-log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
-
-# 30-day backup
-#log4j.appender.DRFA.MaxBackupIndex=30
-log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
-# Debugging Pattern format
-log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %C{2}(%L): %m%n
-
-
-#
-# console
-# Add "console" to rootlogger above if you want to use this
-#
-log4j.appender.console=org.apache.log4j.ConsoleAppender
-log4j.appender.console.target=System.err
-log4j.appender.console.layout=org.apache.log4j.PatternLayout
-log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %C{2}(%L): %m%n
-
-# Custom Logging levels
-
-#log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG
-
-log4j.logger.org.apache.hadoop=WARN
-log4j.logger.org.apache.zookeeper=ERROR
-log4j.logger.org.apache.hadoop.hbase=DEBUG
-
-#These settings are workarounds against spurious logs from the minicluster.
-#See HBASE-4709
-log4j.logger.org.apache.hadoop.metrics2.impl.MetricsConfig=WARN
-log4j.logger.org.apache.hadoop.metrics2.impl.MetricsSinkAdapter=WARN
-log4j.logger.org.apache.hadoop.metrics2.impl.MetricsSystemImpl=WARN
-log4j.logger.org.apache.hadoop.metrics2.util.MBeans=WARN
-# Enable this to get detailed connection error/retry logging.
-# log4j.logger.org.apache.hadoop.hbase.client.ConnectionImplementation=TRACE

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/test/resources/webapps/static/test.css
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/resources/webapps/static/test.css b/hbase-http/src/test/resources/webapps/static/test.css
deleted file mode 100644
index ae43828..0000000
--- a/hbase-http/src/test/resources/webapps/static/test.css
+++ /dev/null
@@ -1,21 +0,0 @@
-/**
- * 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!
- */

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/test/resources/webapps/test/testjsp.jsp
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/resources/webapps/test/testjsp.jsp b/hbase-http/src/test/resources/webapps/test/testjsp.jsp
deleted file mode 100644
index 540adf9..0000000
--- a/hbase-http/src/test/resources/webapps/test/testjsp.jsp
+++ /dev/null
@@ -1,21 +0,0 @@
-<?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/d8a7b7b3/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
index 360ab9d..c2f6cf6 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServer.java
@@ -20,8 +20,10 @@ package org.apache.hadoop.hbase.rest;
 
 import java.lang.management.ManagementFactory;
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.EnumSet;
 import java.util.concurrent.ArrayBlockingQueue;
 
@@ -44,7 +46,7 @@ import org.apache.hadoop.hbase.rest.filter.GzipFilter;
 import org.apache.hadoop.hbase.rest.filter.RestCsrfPreventionFilter;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.DNS;
-import org.apache.hadoop.hbase.http.HttpServerUtil;
+import org.apache.hadoop.hbase.util.HttpServerUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Strings;
 import org.apache.hadoop.hbase.util.VersionInfo;

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/HBaseRESTTestingUtility.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/HBaseRESTTestingUtility.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/HBaseRESTTestingUtility.java
index 4c977fd..0714c7b 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/HBaseRESTTestingUtility.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/HBaseRESTTestingUtility.java
@@ -24,7 +24,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.http.HttpServerUtil;
+import org.apache.hadoop.hbase.util.HttpServerUtil;
 import org.apache.hadoop.util.StringUtils;
 
 import org.eclipse.jetty.server.HttpConfiguration;
@@ -99,7 +99,7 @@ public class HBaseRESTTestingUtility {
     // get the port
     testServletPort = ((ServerConnector)server.getConnectors()[0]).getLocalPort();
 
-    LOG.info("started " + server.getClass().getName() + " on port " +
+    LOG.info("started " + server.getClass().getName() + " on port " + 
       testServletPort);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index ea02f26..3a025d1 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -360,16 +360,6 @@
       <artifactId>hbase-common</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-http</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-http</artifactId>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <!--Needed by the visiblity tags and acl CPEP things
            in here in hbase-server (that should be out in hbase-endpoints
            or integrated). -->
@@ -470,13 +460,48 @@
     </dependency>
     <dependency>
       <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-util</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-util-ajax</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-jsp</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-webapp</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-http</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-security</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.containers</groupId>
+      <artifactId>jersey-container-servlet-core</artifactId>
+    </dependency>
+    <dependency>
       <!--For JspC used in ant task-->
       <groupId>org.glassfish.web</groupId>
       <artifactId>javax.servlet.jsp</artifactId>
     </dependency>
+    <!-- Specifically needed for jetty-jsp, included
+         to bypass version scanning that hits a bad repo
+         see HBASE-18831 -->
+    <dependency>
+      <groupId>org.glassfish</groupId>
+      <artifactId>javax.el</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.codehaus.jettison</groupId>
       <artifactId>jettison</artifactId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..bd8570e
--- /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.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import org.eclipse.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/d8a7b7b3/hbase-server/src/main/java/org/apache/hadoop/hbase/http/ClickjackingPreventionFilter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/http/ClickjackingPreventionFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/ClickjackingPreventionFilter.java
new file mode 100644
index 0000000..9944d29
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/ClickjackingPreventionFilter.java
@@ -0,0 +1,55 @@
+/**
+ * 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;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class ClickjackingPreventionFilter implements Filter {
+
+    private FilterConfig filterConfig;
+
+    @Override
+    public void init(FilterConfig filterConfig) throws ServletException {
+        this.filterConfig = filterConfig;
+    }
+
+    @Override
+    public void doFilter(ServletRequest req, ServletResponse res,
+                         FilterChain chain)
+            throws IOException, ServletException {
+        HttpServletResponse httpRes = (HttpServletResponse) res;
+        httpRes.addHeader("X-Frame-Options", filterConfig.getInitParameter("xframeoptions"));
+        chain.doFilter(req, res);
+    }
+
+    @Override
+    public void destroy() {
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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/d8a7b7b3/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..e033c10
--- /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);
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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/d8a7b7b3/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..b9dde23
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpConfig.java
@@ -0,0 +1,80 @@
+/**
+ * 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.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Statics to get access to Http related configuration.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class HttpConfig {
+  private Policy policy;
+  public enum Policy {
+    HTTP_ONLY,
+    HTTPS_ONLY,
+    HTTP_AND_HTTPS;
+
+    public 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;
+    }
+  }
+
+   public HttpConfig(final Configuration conf) {
+    boolean sslEnabled = conf.getBoolean(
+      ServerConfigurationKeys.HBASE_SSL_ENABLED_KEY,
+      ServerConfigurationKeys.HBASE_SSL_ENABLED_DEFAULT);
+    policy = sslEnabled ? Policy.HTTPS_ONLY : Policy.HTTP_ONLY;
+    if (sslEnabled) {
+      conf.addResource("ssl-server.xml");
+      conf.addResource("ssl-client.xml");
+    }
+  }
+
+  public void setPolicy(Policy policy) {
+    this.policy = policy;
+  }
+
+  public boolean isSecure() {
+    return policy == Policy.HTTPS_ONLY;
+  }
+
+  public String getSchemePrefix() {
+    return (isSecure()) ? "https://" : "http://";
+  }
+
+  public String getScheme(Policy policy) {
+    return policy == Policy.HTTPS_ONLY ? "https://" : "http://";
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..cfc0640
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLog.java
@@ -0,0 +1,93 @@
+/**
+ * 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.eclipse.jetty.server.RequestLog;
+import org.eclipse.jetty.server.NCSARequestLog;
+
+/**
+ * RequestLog object for use with Http
+ */
+public class HttpRequestLog {
+
+  private static final Log LOG = LogFactory.getLog(HttpRequestLog.class);
+  private static final HashMap<String, String> serverToComponent;
+
+  static {
+    serverToComponent = new HashMap<>();
+    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/d8a7b7b3/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;
+  }
+}


[02/10] hbase git commit: Revert "HBASE-19053 Split out o.a.h.h.http from hbase-server into a separate module"

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..e9a56ff
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpServer.java
@@ -0,0 +1,621 @@
+/**
+ * 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.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 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.http.HttpServer.QuotingInputFilter.RequestQuoter;
+import org.apache.hadoop.hbase.http.resource.JerseyResource;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+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.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.util.ajax.JSON;
+import org.junit.AfterClass;
+import org.junit.Assert;
+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;
+
+@Category({MiscTests.class, SmallTests.class})
+public class TestHttpServer extends HttpServerFunctionalTest {
+  private static final Log LOG = LogFactory.getLog(TestHttpServer.class);
+  private static HttpServer server;
+  private static URL baseUrl;
+  // jetty 9.4.x needs this many threads to start, even in the small.
+  static final int MAX_THREADS = 16;
+  
+  @SuppressWarnings("serial")
+  public static class EchoMapServlet extends HttpServlet {
+    @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<>(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 {
+    @Override
+    public void doGet(HttpServletRequest request, 
+                      HttpServletResponse response
+                      ) throws ServletException, IOException {
+      PrintWriter out = response.getWriter();
+      SortedSet<String> sortedKeys = new TreeSet<>();
+      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, MAX_THREADS);
+    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
+  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
+    // JSPs do not work from unit tests
+    // 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<>();
+
+    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");
+      ServerConnector listener = (ServerConnector) 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 testXFrameHeaderSameOrigin() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set("hbase.http.filter.xframeoptions.mode", "SAMEORIGIN");
+
+    HttpServer myServer = new HttpServer.Builder().setName("test")
+            .addEndpoint(new URI("http://localhost:0"))
+            .setFindPort(true).setConf(conf).build();
+    myServer.setAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE, conf);
+    myServer.addServlet("echo", "/echo", EchoServlet.class);
+    myServer.start();
+
+    String serverURL = "http://"
+            + NetUtils.getHostPortString(myServer.getConnectorAddress(0));
+    URL url = new URL(new URL(serverURL), "/echo?a=b&c=d");
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+    assertEquals("SAMEORIGIN", conn.getHeaderField("X-Frame-Options"));
+    myServer.stop();
+  }
+
+
+
+  @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"));
+    assertEquals("DENY", conn.getHeaderField("X-Frame-Options"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..d0f2825
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerLifecycle.java
@@ -0,0 +1,135 @@
+/**
+ * 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.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MiscTests.class, 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
+   */
+  @Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
+  public void testCreatedServerIsNotAlive() throws Throwable {
+    HttpServer server = createTestServer();
+    assertNotLive(server);
+  }
+
+  @Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
+  public void testStopUnstartedServer() throws Throwable {
+    HttpServer server = createTestServer();
+    stop(server);
+  }
+
+  /**
+   * Test that the server is alive once started
+   *
+   * @throws Throwable on failure
+   */
+  @Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
+  public void testStartedServerIsAlive() throws Throwable {
+    HttpServer server = null;
+    server = createTestServer();
+    assertNotLive(server);
+    server.start();
+    assertAlive(server);
+    stop(server);
+  }
+
+  /**
+   * 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
+   */
+  @Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
+  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
+   */
+  @Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
+  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
+   */
+  @Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
+  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/d8a7b7b3/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..db394a8
--- /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.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+
+import java.io.FileNotFoundException;
+
+/**
+ * Test webapp loading
+ */
+@Category({MiscTests.class, 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/d8a7b7b3/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..3c2de53
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestPathFilter.java
@@ -0,0 +1,155 @@
+/**
+ * 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.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.net.NetUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MiscTests.class, SmallTests.class})
+public class TestPathFilter extends HttpServerFunctionalTest {
+  private static final Log LOG = LogFactory.getLog(HttpServer.class);
+  static final Set<String> RECORDS = new TreeSet<>();
+
+  /** 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/d8a7b7b3/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..b599350
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestSSLHttpServer.java
@@ -0,0 +1,124 @@
+/**
+ * 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.HBaseConfiguration;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.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({MiscTests.class, 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, TestHttpServer.MAX_THREADS);
+
+    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(HBaseConfiguration.getPassword(sslConf, "ssl.server.keystore.keypassword",
+            null))
+        .keyStore(sslConf.get("ssl.server.keystore.location"),
+            HBaseConfiguration.getPassword(sslConf, "ssl.server.keystore.password", null),
+            sslConf.get("ssl.server.keystore.type", "jks"))
+        .trustStore(sslConf.get("ssl.server.truststore.location"),
+            HBaseConfiguration.getPassword(sslConf, "ssl.server.truststore.password", null),
+            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/d8a7b7b3/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..1d24ec2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java
@@ -0,0 +1,210 @@
+/**
+ * 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.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.net.NetUtils;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MiscTests.class, SmallTests.class})
+public class TestServletFilter extends HttpServerFunctionalTest {
+  private 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
+  @Ignore
+  //From stack
+  // Its a 'foreign' test, one that came in from hadoop when we copy/pasted http
+  // It's second class. Could comment it out if only failing test (as per @nkeywal – sort of)
+  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) {
+      GenericTestUtils.assertExceptionContains("Problem starting http server", e);
+    }
+  }
+  
+  /**
+   * 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/d8a7b7b3/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java
new file mode 100644
index 0000000..4fad031
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java
@@ -0,0 +1,258 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.security.Principal;
+import java.security.PrivilegedExceptionAction;
+import java.util.Set;
+
+import javax.security.auth.Subject;
+import javax.security.auth.kerberos.KerberosTicket;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.http.TestHttpServer.EchoServlet;
+import org.apache.hadoop.hbase.http.resource.JerseyResource;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.security.authentication.util.KerberosName;
+import org.apache.http.HttpHost;
+import org.apache.http.HttpResponse;
+import org.apache.http.auth.AuthSchemeProvider;
+import org.apache.http.auth.AuthScope;
+import org.apache.http.auth.KerberosCredentials;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.config.AuthSchemes;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.protocol.HttpClientContext;
+import org.apache.http.config.Lookup;
+import org.apache.http.config.RegistryBuilder;
+import org.apache.http.entity.ByteArrayEntity;
+import org.apache.http.entity.ContentType;
+import org.apache.http.impl.auth.SPNegoSchemeFactory;
+import org.apache.http.impl.client.BasicCredentialsProvider;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.util.EntityUtils;
+import org.apache.kerby.kerberos.kerb.KrbException;
+import org.apache.kerby.kerberos.kerb.client.JaasKrbUtil;
+import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer;
+import org.ietf.jgss.GSSCredential;
+import org.ietf.jgss.GSSManager;
+import org.ietf.jgss.GSSName;
+import org.ietf.jgss.Oid;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test class for SPNEGO authentication on the HttpServer. Uses Kerby's MiniKDC and Apache
+ * HttpComponents to verify that a simple Servlet is reachable via SPNEGO and unreachable w/o.
+ */
+@Category({MiscTests.class, SmallTests.class})
+public class TestSpnegoHttpServer extends HttpServerFunctionalTest {
+  private static final Log LOG = LogFactory.getLog(TestSpnegoHttpServer.class);
+  private static final String KDC_SERVER_HOST = "localhost";
+  private static final String CLIENT_PRINCIPAL = "client";
+
+  private static HttpServer server;
+  private static URL baseUrl;
+  private static SimpleKdcServer kdc;
+  private static File infoServerKeytab;
+  private static File clientKeytab;
+
+  @BeforeClass
+  public static void setupServer() throws Exception {
+    final String serverPrincipal = "HTTP/" + KDC_SERVER_HOST;
+    final File target = new File(System.getProperty("user.dir"), "target");
+    assertTrue(target.exists());
+
+    kdc = buildMiniKdc();
+    kdc.start();
+
+    File keytabDir = new File(target, TestSpnegoHttpServer.class.getSimpleName()
+        + "_keytabs");
+    if (keytabDir.exists()) {
+      deleteRecursively(keytabDir);
+    }
+    keytabDir.mkdirs();
+
+    infoServerKeytab = new File(keytabDir, serverPrincipal.replace('/', '_') + ".keytab");
+    clientKeytab = new File(keytabDir, CLIENT_PRINCIPAL + ".keytab");
+
+    setupUser(kdc, clientKeytab, CLIENT_PRINCIPAL);
+    setupUser(kdc, infoServerKeytab, serverPrincipal);
+
+    Configuration conf = buildSpnegoConfiguration(serverPrincipal, infoServerKeytab);
+
+    server = createTestServerWithSecurity(conf);
+    server.addServlet("echo", "/echo", EchoServlet.class);
+    server.addJerseyResourcePackage(JerseyResource.class.getPackage().getName(), "/jersey/*");
+    server.start();
+    baseUrl = getServerURL(server);
+
+    LOG.info("HTTP server started: "+ baseUrl);
+  }
+
+  @AfterClass
+  public static void stopServer() throws Exception {
+    try {
+      if (null != server) {
+        server.stop();
+      }
+    } catch (Exception e) {
+      LOG.info("Failed to stop info server", e);
+    }
+    try {
+      if (null != kdc) {
+        kdc.stop();
+      }
+    } catch (Exception e) {
+      LOG.info("Failed to stop mini KDC", e);
+    }
+  }
+
+  private static void setupUser(SimpleKdcServer kdc, File keytab, String principal)
+      throws KrbException {
+    kdc.createPrincipal(principal);
+    kdc.exportPrincipal(principal, keytab);
+  }
+
+  private static SimpleKdcServer buildMiniKdc() throws Exception {
+    SimpleKdcServer kdc = new SimpleKdcServer();
+
+    final File target = new File(System.getProperty("user.dir"), "target");
+    File kdcDir = new File(target, TestSpnegoHttpServer.class.getSimpleName());
+    if (kdcDir.exists()) {
+      deleteRecursively(kdcDir);
+    }
+    kdcDir.mkdirs();
+    kdc.setWorkDir(kdcDir);
+
+    kdc.setKdcHost(KDC_SERVER_HOST);
+    int kdcPort = getFreePort();
+    kdc.setAllowTcp(true);
+    kdc.setAllowUdp(false);
+    kdc.setKdcTcpPort(kdcPort);
+
+    LOG.info("Starting KDC server at " + KDC_SERVER_HOST + ":" + kdcPort);
+
+    kdc.init();
+
+    return kdc;
+  }
+
+  private static Configuration buildSpnegoConfiguration(String serverPrincipal, File
+      serverKeytab) {
+    Configuration conf = new Configuration();
+    KerberosName.setRules("DEFAULT");
+
+    conf.setInt(HttpServer.HTTP_MAX_THREADS, TestHttpServer.MAX_THREADS);
+
+    // Enable Kerberos (pre-req)
+    conf.set("hbase.security.authentication", "kerberos");
+    conf.set(HttpServer.HTTP_UI_AUTHENTICATION, "kerberos");
+    conf.set(HttpServer.HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_KEY, serverPrincipal);
+    conf.set(HttpServer.HTTP_SPNEGO_AUTHENTICATION_KEYTAB_KEY, serverKeytab.getAbsolutePath());
+
+    return conf;
+  }
+
+  @Test
+  public void testUnauthorizedClientsDisallowed() throws IOException {
+    URL url = new URL(getServerURL(server), "/echo?a=b");
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    assertEquals(HttpURLConnection.HTTP_UNAUTHORIZED, conn.getResponseCode());
+  }
+
+  @Test
+  public void testAllowedClient() throws Exception {
+    // Create the subject for the client
+    final Subject clientSubject = JaasKrbUtil.loginUsingKeytab(CLIENT_PRINCIPAL, clientKeytab);
+    final Set<Principal> clientPrincipals = clientSubject.getPrincipals();
+    // Make sure the subject has a principal
+    assertFalse(clientPrincipals.isEmpty());
+
+    // Get a TGT for the subject (might have many, different encryption types). The first should
+    // be the default encryption type.
+    Set<KerberosTicket> privateCredentials =
+            clientSubject.getPrivateCredentials(KerberosTicket.class);
+    assertFalse(privateCredentials.isEmpty());
+    KerberosTicket tgt = privateCredentials.iterator().next();
+    assertNotNull(tgt);
+
+    // The name of the principal
+    final String principalName = clientPrincipals.iterator().next().getName();
+
+    // Run this code, logged in as the subject (the client)
+    HttpResponse resp = Subject.doAs(clientSubject,
+        new PrivilegedExceptionAction<HttpResponse>() {
+      @Override
+      public HttpResponse run() throws Exception {
+        // Logs in with Kerberos via GSS
+        GSSManager gssManager = GSSManager.getInstance();
+        // jGSS Kerberos login constant
+        Oid oid = new Oid("1.2.840.113554.1.2.2");
+        GSSName gssClient = gssManager.createName(principalName, GSSName.NT_USER_NAME);
+        GSSCredential credential = gssManager.createCredential(gssClient,
+            GSSCredential.DEFAULT_LIFETIME, oid, GSSCredential.INITIATE_ONLY);
+
+        HttpClientContext context = HttpClientContext.create();
+        Lookup<AuthSchemeProvider> authRegistry = RegistryBuilder.<AuthSchemeProvider>create()
+            .register(AuthSchemes.SPNEGO, new SPNegoSchemeFactory(true, true))
+            .build();
+
+        HttpClient client = HttpClients.custom().setDefaultAuthSchemeRegistry(authRegistry).build();
+        BasicCredentialsProvider credentialsProvider = new BasicCredentialsProvider();
+        credentialsProvider.setCredentials(AuthScope.ANY, new KerberosCredentials(credential));
+
+        URL url = new URL(getServerURL(server), "/echo?a=b");
+        context.setTargetHost(new HttpHost(url.getHost(), url.getPort()));
+        context.setCredentialsProvider(credentialsProvider);
+        context.setAuthSchemeRegistry(authRegistry);
+
+        HttpGet get = new HttpGet(url.toURI());
+        return client.execute(get, context);
+      }
+    });
+
+    assertNotNull(resp);
+    assertEquals(HttpURLConnection.HTTP_OK, resp.getStatusLine().getStatusCode());
+    assertEquals("a:b", EntityUtils.toString(resp.getEntity()).trim());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testMissingConfigurationThrowsException() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(HttpServer.HTTP_MAX_THREADS, TestHttpServer.MAX_THREADS);
+    // Enable Kerberos (pre-req)
+    conf.set("hbase.security.authentication", "kerberos");
+    // Intentionally skip keytab and principal
+
+    HttpServer customServer = createTestServerWithSecurity(conf);
+    customServer.addServlet("echo", "/echo", EchoServlet.class);
+    customServer.addJerseyResourcePackage(JerseyResource.class.getPackage().getName(), "/jersey/*");
+    customServer.start();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..e1d9aca
--- /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.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.eclipse.jetty.util.ajax.JSON;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+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({MiscTests.class, 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());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..d7e68d5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/jmx/TestJMXJsonServlet.java
@@ -0,0 +1,134 @@
+/*
+ * 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.HttpURLConnection;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.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({MiscTests.class, 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 {
+    // Eclipse doesn't pick this up correctly from the plugin
+    // configuration in the pom.
+    System.setProperty(HttpServerFunctionalTest.TEST_BUILD_WEBAPPS, "target/test-classes/webapps");
+    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);
+
+  }
+
+  @Test
+  public void testDisallowedJSONPCallback() throws Exception {
+    String callback = "function(){alert('bigproblems!')};foo";
+    URL url = new URL(
+        baseUrl, "/jmx?qry=java.lang:type=Memory&callback="+URLEncoder.encode(callback, "UTF-8"));
+    HttpURLConnection cnxn = (HttpURLConnection) url.openConnection();
+    assertEquals(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, cnxn.getResponseCode());
+  }
+
+  @Test
+  public void testUnderscoresInJSONPCallback() throws Exception {
+    String callback = "my_function";
+    URL url = new URL(
+        baseUrl, "/jmx?qry=java.lang:type=Memory&callback="+URLEncoder.encode(callback, "UTF-8"));
+    HttpURLConnection cnxn = (HttpURLConnection) url.openConnection();
+    assertEquals(HttpServletResponse.SC_OK, cnxn.getResponseCode());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..81bcbd5
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/lib/TestStaticUserWebFilter.java
@@ -0,0 +1,86 @@
+/**
+ * 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.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.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({MiscTests.class, 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/d8a7b7b3/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..e14e3b4
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/log/TestLogLevel.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.log;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.*;
+import java.net.*;
+
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.http.HttpServer;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.commons.logging.*;
+import org.apache.commons.logging.impl.*;
+import org.apache.log4j.*;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MiscTests.class, SmallTests.class})
+public class TestLogLevel {
+  static final PrintStream out = System.out;
+
+  @Test (timeout=60000)
+  @SuppressWarnings("deprecation")
+  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 = null;
+      try {
+        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);
+        try (BufferedReader in = new BufferedReader(new InputStreamReader(url.openStream()))) {
+          for(String line; (line = in.readLine()) != null; out.println(line));
+        }
+        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()));
+      } finally {
+        if (server != null) {
+          server.stop();
+        }
+      }
+    }
+    else {
+      out.println(testlog.getClass() + " not tested.");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..bf0e609
--- /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.eclipse.jetty.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 {
+  private 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<>();
+    m.put(PATH, path);
+    m.put(OP, op);
+    final String js = JSON.toString(m);
+    return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+  }
+}


[04/10] hbase git commit: Revert "HBASE-19053 Split out o.a.h.h.http from hbase-server into a separate module"

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..726595b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
@@ -0,0 +1,1380 @@
+/**
+ * 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.PrintStream;
+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.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+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.hbase.util.Threads;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
+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.Shell;
+
+import org.eclipse.jetty.http.HttpVersion;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.Handler;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.server.SecureRequestCustomizer;
+import org.eclipse.jetty.server.SslConnectionFactory;
+import org.eclipse.jetty.server.handler.ContextHandlerCollection;
+import org.eclipse.jetty.server.handler.HandlerCollection;
+import org.eclipse.jetty.server.RequestLog;
+import org.eclipse.jetty.server.handler.RequestLogHandler;
+import org.eclipse.jetty.servlet.FilterMapping;
+import org.eclipse.jetty.servlet.ServletHandler;
+import org.eclipse.jetty.servlet.FilterHolder;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.DefaultServlet;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.util.MultiException;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.eclipse.jetty.util.thread.QueuedThreadPool;
+import org.eclipse.jetty.webapp.WebAppContext;
+
+import org.glassfish.jersey.server.ResourceConfig;
+import org.glassfish.jersey.servlet.ServletContainer;
+
+import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
+import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
+
+/**
+ * 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/" -&gt; points to the log directory
+ *   "/static/" -&gt; points to common static files (src/webapps/static)
+ *   "/" -&gt; the jsp server code from (src/webapps/&lt;name&gt;)
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class HttpServer implements FilterContainer {
+  private static final Log LOG = LogFactory.getLog(HttpServer.class);
+  private static final String EMPTY_STRING = "";
+
+  private static final int DEFAULT_MAX_HEADER_SIZE = 64 * 1024; // 64K
+
+  static final String FILTER_INITIALIZERS_PROPERTY
+      = "hbase.http.filter.initializers";
+  static final String HTTP_MAX_THREADS = "hbase.http.max.threads";
+
+  public static final String HTTP_UI_AUTHENTICATION = "hbase.security.authentication.ui";
+  static final String HTTP_AUTHENTICATION_PREFIX = "hbase.security.authentication.";
+  static final String HTTP_SPNEGO_AUTHENTICATION_PREFIX = HTTP_AUTHENTICATION_PREFIX
+      + "spnego.";
+  static final String HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX = "kerberos.principal";
+  public static final String HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_KEY =
+      HTTP_SPNEGO_AUTHENTICATION_PREFIX + HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX;
+  static final String HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX = "kerberos.keytab";
+  public static final String HTTP_SPNEGO_AUTHENTICATION_KEYTAB_KEY =
+      HTTP_SPNEGO_AUTHENTICATION_PREFIX + HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX;
+  static final String HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_SUFFIX = "kerberos.name.rules";
+  public static final String HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_KEY =
+      HTTP_SPNEGO_AUTHENTICATION_PREFIX + HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_SUFFIX;
+  static final String HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_SUFFIX =
+      "signature.secret.file";
+  public static final String HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_KEY =
+      HTTP_AUTHENTICATION_PREFIX + HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_SUFFIX;
+
+  // 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 ServerConnector listener;
+    private ListenerInfo(boolean isManaged, ServerConnector 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<ServletContextHandler, Boolean> defaultContexts = new HashMap<>();
+  protected final List<String> filterNames = new ArrayList<>();
+  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 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;
+
+    private String kerberosNameRulesKey;
+    private String signatureSecretFileKey;
+
+    @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 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 setKerberosNameRulesKey(String kerberosNameRulesKey) {
+      this.kerberosNameRulesKey = kerberosNameRulesKey;
+      return this;
+    }
+
+    public Builder setSignatureSecretFileKey(String signatureSecretFileKey) {
+      this.signatureSecretFileKey = signatureSecretFileKey;
+      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.isEmpty()) {
+        throw new HadoopIllegalArgumentException("No endpoints specified");
+      }
+
+      if (hostName == null) {
+        hostName = 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, kerberosNameRulesKey,
+            signatureSecretFileKey);
+      }
+
+      for (URI ep : endpoints) {
+        ServerConnector listener = null;
+        String scheme = ep.getScheme();
+        HttpConfiguration httpConfig = new HttpConfiguration();
+        httpConfig.setSecureScheme("https");
+        httpConfig.setHeaderCacheSize(DEFAULT_MAX_HEADER_SIZE);
+        httpConfig.setResponseHeaderSize(DEFAULT_MAX_HEADER_SIZE);
+        httpConfig.setRequestHeaderSize(DEFAULT_MAX_HEADER_SIZE);
+
+        if ("http".equals(scheme)) {
+          listener = new ServerConnector(server.webServer, new HttpConnectionFactory(httpConfig));
+        } else if ("https".equals(scheme)) {
+          HttpConfiguration httpsConfig = new HttpConfiguration(httpConfig);
+          httpsConfig.addCustomizer(new SecureRequestCustomizer());
+          SslContextFactory sslCtxFactory = new SslContextFactory();
+          sslCtxFactory.setNeedClientAuth(needsClientAuth);
+          sslCtxFactory.setKeyManagerPassword(keyPassword);
+
+          if (keyStore != null) {
+            sslCtxFactory.setKeyStorePath(keyStore);
+            sslCtxFactory.setKeyStoreType(keyStoreType);
+            sslCtxFactory.setKeyStorePassword(keyStorePassword);
+          }
+
+          if (trustStore != null) {
+            sslCtxFactory.setTrustStorePath(trustStore);
+            sslCtxFactory.setTrustStoreType(trustStoreType);
+            sslCtxFactory.setTrustStorePassword(trustStorePassword);
+
+          }
+          listener = new ServerConnector(server.webServer, new SslConnectionFactory(sslCtxFactory,
+              HttpVersion.HTTP_1_1.toString()), new HttpConnectionFactory(httpsConfig));
+        } else {
+          throw new HadoopIllegalArgumentException(
+              "unknown scheme for endpoint:" + ep);
+        }
+
+        // default settings for connector
+        listener.setAcceptQueueSize(128);
+        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
+          listener.setReuseAddress(false);
+        }
+
+        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());
+  }
+
+  /**
+   * 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, pathSpecs);
+  }
+
+  /**
+   * Create a status server on the given port.
+   * The jsp scripts are taken from src/webapps/&lt;name&gt;.
+   * @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);
+  }
+
+  @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);
+  }
+
+  /**
+   * Create a status server on the given port.
+   * The jsp scripts are taken from src/webapps/&lt;name&gt;.
+   * @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 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,
+      String[] pathSpecs) throws IOException {
+    this(new Builder().setName(name)
+        .addEndpoint(URI.create("http://" + bindAddress + ":" + port))
+        .setFindPort(findPort).setConf(conf).setACL(adminsAcl)
+        .setPathSpec(pathSpecs));
+  }
+
+  private HttpServer(final Builder b) throws IOException {
+    this.appDir = b.appDir;
+    this.logDir = b.logDir;
+    final String appDir = getWebAppsPath(b.name);
+
+
+    int maxThreads = b.conf.getInt(HTTP_MAX_THREADS, 16);
+    // If HTTP_MAX_THREADS is less than or equal to 0, QueueThreadPool() will use the
+    // default value (currently 200).
+    QueuedThreadPool threadPool = maxThreads <= 0 ? new QueuedThreadPool()
+        : new QueuedThreadPool(maxThreads);
+    threadPool.setDaemon(true);
+    this.webServer = new Server(threadPool);
+
+    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);
+
+    HandlerCollection handlerCollection = new HandlerCollection();
+
+    ContextHandlerCollection contexts = new ContextHandlerCollection();
+    RequestLog requestLog = HttpRequestLog.getRequestLog(name);
+
+    if (requestLog != null) {
+      RequestLogHandler requestLogHandler = new RequestLogHandler();
+      requestLogHandler.setRequestLog(requestLog);
+      handlerCollection.addHandler(requestLogHandler);
+    }
+
+    final String appDir = getWebAppsPath(name);
+
+    handlerCollection.addHandler(contexts);
+    handlerCollection.addHandler(webAppContext);
+
+    webServer.setHandler(handlerCollection);
+
+    addDefaultApps(contexts, appDir, conf);
+
+    addGlobalFilter("safety", QuotingInputFilter.class.getName(), null);
+    Map<String, String> params = new HashMap<>();
+    params.put("xframeoptions", conf.get("hbase.http.filter.xframeoptions.mode", "DENY"));
+    addGlobalFilter("clickjackingprevention",
+            ClickjackingPreventionFilter.class.getName(), params);
+    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 addManagedListener(ServerConnector 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[] { "/*" });
+  }
+
+  /** 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]);
+    }
+    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) {
+      ServletContextHandler logContext = new ServletContextHandler(parent, "/logs");
+      logContext.addServlet(AdminAuthorizedServlet.class, "/*");
+      logContext.setResourceBase(logDir);
+
+      if (conf.getBoolean(
+          ServerConfigurationKeys.HBASE_JETTY_LOGS_SERVE_ALIASES,
+          ServerConfigurationKeys.DEFAULT_HBASE_JETTY_LOGS_SERVE_ALIASES)) {
+        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/*"
+    ServletContextHandler staticContext = new ServletContextHandler(parent, "/static");
+    staticContext.setResourceBase(appDir + "/static");
+    staticContext.addServlet(DefaultServlet.class, "/*");
+    staticContext.setDisplayName("static");
+    setContextAttributes(staticContext, conf);
+    defaultContexts.put(staticContext, true);
+  }
+
+  private void setContextAttributes(ServletContextHandler 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);
+
+    // Hadoop3 has moved completely to metrics2, and  dropped support for Metrics v1's
+    // MetricsServlet (see HADOOP-12504).  We'll using reflection to load if against hadoop2.
+    // Remove when we drop support for hbase on hadoop2.x.
+    try {
+      Class clz = Class.forName("org.apache.hadoop.metrics.MetricsServlet");
+      addServlet("metrics", "/metrics", clz);
+    } catch (Exception e) {
+      // do nothing
+    }
+
+    addServlet("jmx", "/jmx", JMXJsonServlet.class);
+    addServlet("conf", "/conf", ConfServlet.class);
+  }
+
+  /**
+   * 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);
+
+    ResourceConfig application = new ResourceConfig().packages(packageName);
+    final ServletHolder sh = new ServletHolder(new ServletContainer(application));
+    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(FilterMapping.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<ServletContextHandler, Boolean> e : defaultContexts.entrySet()) {
+      if (e.getValue()) {
+        ServletContextHandler handler = e.getKey();
+        defineFilter(handler, name, classname, parameters, ALL_URLS);
+        LOG.info("Added filter " + name + " (class=" + classname
+            + ") to context " + handler.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 (ServletContextHandler 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(ServletContextHandler handler, String name,
+      String classname, Map<String,String> parameters, String[] urls) {
+
+    FilterHolder holder = new FilterHolder();
+    holder.setName(name);
+    holder.setClassName(classname);
+    if (parameters != null) {
+      holder.setInitParameters(parameters);
+    }
+    FilterMapping fmap = new FilterMapping();
+    fmap.setPathSpecs(urls);
+    fmap.setDispatches(FilterMapping.ALL);
+    fmap.setFilterName(name);
+    handler.getServletHandler().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,
+      WebAppContext webAppCtx) {
+    for(String name : filterNames) {
+      FilterMapping fmap = new FilterMapping();
+      fmap.setPathSpec(pathSpec);
+      fmap.setFilterName(name);
+      fmap.setDispatches(FilterMapping.ALL);
+      webAppCtx.getServletHandler().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 ((ServerConnector)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;
+
+    ServerConnector c = (ServerConnector)webServer.getConnectors()[index];
+    if (c.getLocalPort() == -1 || c.getLocalPort() == -2) {
+      // -1 if the connector has not been opened
+      // -2 if it has been closed
+      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, String kerberosNameRuleKey,
+      String signatureSecretKeyFileKey) throws IOException {
+    Map<String, String> params = new HashMap<>();
+    String principalInConf = getOrEmptyString(conf, usernameConfKey);
+    if (!principalInConf.isEmpty()) {
+      params.put(HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX, SecurityUtil.getServerPrincipal(
+          principalInConf, hostName));
+    }
+    String httpKeytab = getOrEmptyString(conf, keytabConfKey);
+    if (!httpKeytab.isEmpty()) {
+      params.put(HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX, httpKeytab);
+    }
+    String kerberosNameRule = getOrEmptyString(conf, kerberosNameRuleKey);
+    if (!kerberosNameRule.isEmpty()) {
+      params.put(HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_SUFFIX, kerberosNameRule);
+    }
+    String signatureSecretKeyFile = getOrEmptyString(conf, signatureSecretKeyFileKey);
+    if (!signatureSecretKeyFile.isEmpty()) {
+      params.put(HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_SUFFIX,
+          signatureSecretKeyFile);
+    }
+    params.put(AuthenticationFilter.AUTH_TYPE, "kerberos");
+
+    // Verify that the required options were provided
+    if (isMissing(params.get(HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX)) ||
+            isMissing(params.get(HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX))) {
+      throw new IllegalArgumentException(usernameConfKey + " and "
+          + keytabConfKey + " are both required in the configuration "
+          + "to enable SPNEGO/Kerberos authentication for the Web UI");
+    }
+
+    addGlobalFilter(SPNEGO_FILTER, AuthenticationFilter.class.getName(), params);
+  }
+
+  /**
+   * Returns true if the argument is non-null and not whitespace
+   */
+  private boolean isMissing(String value) {
+    if (null == value) {
+      return true;
+    }
+    return value.trim().isEmpty();
+  }
+
+  /**
+   * Extracts the value for the given key from the configuration of returns a string of
+   * zero length.
+   */
+  private String getOrEmptyString(Configuration conf, String key) {
+    if (null == key) {
+      return EMPTY_STRING;
+    }
+    final String value = conf.get(key.trim());
+    return null == value ? EMPTY_STRING : value;
+  }
+
+  /**
+   * 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) {
+      ServerConnector listener = li.listener;
+      if (!li.isManaged || (li.listener.getLocalPort() != -1 && li.listener.getLocalPort() != -2)) {
+        // This listener is either started externally, or has not been opened, or has been closed
+        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.isEmpty()) {
+      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) {
+        ServerConnector 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><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.
+   * </p>
+   *
+   * @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");
+      try (PrintStream out = new PrintStream(
+        response.getOutputStream(), false, "UTF-8")) {
+        Threads.printThreadInfo(out, "");
+        out.flush();
+      }
+      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.
+   */
+  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+  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.
+       */
+      @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;
+      }
+
+      @Override
+      public Map<String, String[]> getParameterMap() {
+        Map<String, String[]> result = new HashMap<>();
+        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();
+      ServletContext context = config.getServletContext();
+      return context.getMimeType(path);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..b5f4183
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java
@@ -0,0 +1,112 @@
+/**
+ *
+ * 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.hbase.HBaseConfiguration;
+import org.apache.yetus.audience.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/" -&gt; points to stack trace
+ *   "/static/" -&gt; points to common static files (src/hbase-webapps/static)
+ *   "/" -&gt; the jsp server code from (src/hbase-webapps/&lt;name&gt;)
+ */
+@InterfaceAudience.Private
+public class InfoServer {
+  
+  private static final String HBASE_APP_DIR = "hbase-webapps";
+  private final org.apache.hadoop.hbase.http.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 {
+    HttpConfig httpConfig = new HttpConfig(c);
+    HttpServer.Builder builder =
+      new org.apache.hadoop.hbase.http.HttpServer.Builder();
+
+      builder.setName(name).addEndpoint(URI.create(httpConfig.getSchemePrefix() +
+        bindAddress + ":" +
+        port)).setAppDir(HBASE_APP_DIR).setFindPort(findPort).setConf(c);
+      String logDir = System.getProperty("hbase.log.dir");
+      if (logDir != null) {
+        builder.setLogDir(logDir);
+      }
+    if (httpConfig.isSecure()) {
+    builder.keyPassword(HBaseConfiguration.getPassword(c, "ssl.server.keystore.keypassword", null))
+      .keyStore(c.get("ssl.server.keystore.location"),
+        HBaseConfiguration.getPassword(c,"ssl.server.keystore.password", null),
+        c.get("ssl.server.keystore.type", "jks"))
+      .trustStore(c.get("ssl.server.truststore.location"),
+        HBaseConfiguration.getPassword(c, "ssl.server.truststore.password", null),
+        c.get("ssl.server.truststore.type", "jks"));
+    }
+    // Enable SPNEGO authentication
+    if ("kerberos".equalsIgnoreCase(c.get(HttpServer.HTTP_UI_AUTHENTICATION, null))) {
+      builder.setUsernameConfKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_KEY)
+        .setKeytabConfKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_KEYTAB_KEY)
+        .setKerberosNameRulesKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_KEY)
+        .setSignatureSecretFileKey(
+            HttpServer.HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_KEY)
+        .setSecurityEnabled(true);
+    }
+    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/d8a7b7b3/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..a1daf15
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/NoCacheFilter.java
@@ -0,0 +1,56 @@
+/**
+ * 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;
+
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+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/d8a7b7b3/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..4ae4a2f
--- /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.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.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";
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..d9aa7b6
--- /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.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.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/d8a7b7b3/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..3abad3a
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
@@ -0,0 +1,240 @@
+/*
+ * 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 javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.openmbean.CompositeData;
+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.apache.hadoop.hbase.util.JSONBean;
+
+/*
+ * 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>
+ * <p>
+ * For example <code>http://.../jmx?qry=Hadoop:*</code> will return
+ * all hadoop metrics exposed through JMX.
+ * </p>
+ * <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>
+ * <p>
+ * For example 
+ * <code>
+ * http://../jmx?get=Hadoop:service=NameNode,name=NameNodeInfo::ClusterId
+ * </code> will return the cluster id of the namenode mxbean.
+ * </p>
+ * <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>
+ * <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>
+ * <p>
+ * The return format is JSON and in the form
+ * </p>
+ *  <pre><code>
+ *  {
+ *    "beans" : [
+ *      {
+ *        "name":"bean-name"
+ *        ...
+ *      }
+ *    ]
+ *  }
+ *  </code></pre>
+ *  <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.
+ * </p>
+ *  
+ */
+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";
+  /**
+   * If query string includes 'description', then we will emit bean and attribute descriptions to
+   * output IFF they are not null and IFF the description is not the same as the attribute name:
+   * i.e. specify an URL like so: /jmx?description=true
+   */
+  private static final String INCLUDE_DESCRIPTION = "description";
+
+  /**
+   * MBean server.
+   */
+  protected transient MBeanServer mBeanServer;
+
+  protected transient JSONBean jsonBeanWriter;
+
+  /**
+   * Initialize this servlet.
+   */
+  @Override
+  public void init() throws ServletException {
+    // Retrieve the MBean server
+    mBeanServer = ManagementFactory.getPlatformMBeanServer();
+    this.jsonBeanWriter = new JSONBean();
+  }
+
+  /**
+   * 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) throws IOException {
+    try {
+      if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(), request, response)) {
+        return;
+      }
+      String jsonpcb = null;
+      PrintWriter writer = null;
+      JSONBean.Writer beanWriter = null;
+      try {
+        jsonpcb = checkCallbackName(request.getParameter(CALLBACK_PARAM));
+        writer = response.getWriter();
+        beanWriter = this.jsonBeanWriter.open(writer);
+ 
+        // "callback" parameter implies JSONP outpout
+        if (jsonpcb != null) {
+          response.setContentType("application/javascript; charset=utf8");
+          writer.write(jsonpcb + "(");
+        } else {
+          response.setContentType("application/json; charset=utf8");
+        }
+        // Should we output description on each attribute and bean?
+        String tmpStr = request.getParameter(INCLUDE_DESCRIPTION);
+        boolean description = tmpStr != null && tmpStr.length() > 0;
+
+        // query per mbean attribute
+        String getmethod = request.getParameter("get");
+        if (getmethod != null) {
+          String[] splitStrings = getmethod.split("\\:\\:");
+          if (splitStrings.length != 2) {
+            beanWriter.write("result", "ERROR");
+            beanWriter.write("message", "query format is not as expected.");
+            beanWriter.flush();
+            response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+            return;
+          }
+          if (beanWriter.write(this.mBeanServer, new ObjectName(splitStrings[0]),
+              splitStrings[1], description) != 0) {
+            beanWriter.flush();
+            response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+          }
+          return;
+        }
+
+        // query per mbean
+        String qry = request.getParameter("qry");
+        if (qry == null) {
+          qry = "*:*";
+        }
+        if (beanWriter.write(this.mBeanServer, new ObjectName(qry), null, description) != 0) {
+          beanWriter.flush();
+          response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
+        }
+      } finally {
+        if (beanWriter != null) beanWriter.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.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
+    } catch (MalformedObjectNameException e) {
+      LOG.error("Caught an exception while processing JMX request", e);
+      response.sendError(HttpServletResponse.SC_BAD_REQUEST);
+    }
+  }
+
+  /**
+   * Verifies that the callback property, if provided, is purely alphanumeric.
+   * This prevents a malicious callback name (that is javascript code) from being
+   * returned by the UI to an unsuspecting user.
+   *
+   * @param callbackName The callback name, can be null.
+   * @return The callback name
+   * @throws IOException If the name is disallowed.
+   */
+  private String checkCallbackName(String callbackName) throws IOException {
+    if (null == callbackName) {
+      return null;
+    }
+    if (callbackName.matches("[A-Za-z0-9_]+")) {
+      return callbackName;
+    }
+    throw new IOException("'callback' must be alphanumeric");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..324cc2d
--- /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;

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..bce3a07
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/lib/StaticUserWebFilter.java
@@ -0,0 +1,155 @@
+/**
+ * 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.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+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.
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+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;
+    }    
+  }
+
+  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+  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 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/d8a7b7b3/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..7bb9a0f
--- /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.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;


[07/10] hbase git commit: Revert "HBASE-19053 Split out o.a.h.h.http from hbase-server into a separate module"

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
deleted file mode 100644
index 726595b..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServer.java
+++ /dev/null
@@ -1,1380 +0,0 @@
-/**
- * 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.PrintStream;
-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.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-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.hbase.util.Threads;
-import org.apache.hadoop.hbase.util.ReflectionUtils;
-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.Shell;
-
-import org.eclipse.jetty.http.HttpVersion;
-import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.server.Handler;
-import org.eclipse.jetty.server.HttpConfiguration;
-import org.eclipse.jetty.server.HttpConnectionFactory;
-import org.eclipse.jetty.server.ServerConnector;
-import org.eclipse.jetty.server.SecureRequestCustomizer;
-import org.eclipse.jetty.server.SslConnectionFactory;
-import org.eclipse.jetty.server.handler.ContextHandlerCollection;
-import org.eclipse.jetty.server.handler.HandlerCollection;
-import org.eclipse.jetty.server.RequestLog;
-import org.eclipse.jetty.server.handler.RequestLogHandler;
-import org.eclipse.jetty.servlet.FilterMapping;
-import org.eclipse.jetty.servlet.ServletHandler;
-import org.eclipse.jetty.servlet.FilterHolder;
-import org.eclipse.jetty.servlet.ServletContextHandler;
-import org.eclipse.jetty.servlet.DefaultServlet;
-import org.eclipse.jetty.servlet.ServletHolder;
-import org.eclipse.jetty.util.MultiException;
-import org.eclipse.jetty.util.ssl.SslContextFactory;
-import org.eclipse.jetty.util.thread.QueuedThreadPool;
-import org.eclipse.jetty.webapp.WebAppContext;
-
-import org.glassfish.jersey.server.ResourceConfig;
-import org.glassfish.jersey.servlet.ServletContainer;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
-import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
-
-/**
- * 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/" -&gt; points to the log directory
- *   "/static/" -&gt; points to common static files (src/webapps/static)
- *   "/" -&gt; the jsp server code from (src/webapps/&lt;name&gt;)
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class HttpServer implements FilterContainer {
-  private static final Log LOG = LogFactory.getLog(HttpServer.class);
-  private static final String EMPTY_STRING = "";
-
-  private static final int DEFAULT_MAX_HEADER_SIZE = 64 * 1024; // 64K
-
-  static final String FILTER_INITIALIZERS_PROPERTY
-      = "hbase.http.filter.initializers";
-  static final String HTTP_MAX_THREADS = "hbase.http.max.threads";
-
-  public static final String HTTP_UI_AUTHENTICATION = "hbase.security.authentication.ui";
-  static final String HTTP_AUTHENTICATION_PREFIX = "hbase.security.authentication.";
-  static final String HTTP_SPNEGO_AUTHENTICATION_PREFIX = HTTP_AUTHENTICATION_PREFIX
-      + "spnego.";
-  static final String HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX = "kerberos.principal";
-  public static final String HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_KEY =
-      HTTP_SPNEGO_AUTHENTICATION_PREFIX + HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX;
-  static final String HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX = "kerberos.keytab";
-  public static final String HTTP_SPNEGO_AUTHENTICATION_KEYTAB_KEY =
-      HTTP_SPNEGO_AUTHENTICATION_PREFIX + HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX;
-  static final String HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_SUFFIX = "kerberos.name.rules";
-  public static final String HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_KEY =
-      HTTP_SPNEGO_AUTHENTICATION_PREFIX + HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_SUFFIX;
-  static final String HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_SUFFIX =
-      "signature.secret.file";
-  public static final String HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_KEY =
-      HTTP_AUTHENTICATION_PREFIX + HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_SUFFIX;
-
-  // 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 ServerConnector listener;
-    private ListenerInfo(boolean isManaged, ServerConnector 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<ServletContextHandler, Boolean> defaultContexts = new HashMap<>();
-  protected final List<String> filterNames = new ArrayList<>();
-  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 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;
-
-    private String kerberosNameRulesKey;
-    private String signatureSecretFileKey;
-
-    @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 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 setKerberosNameRulesKey(String kerberosNameRulesKey) {
-      this.kerberosNameRulesKey = kerberosNameRulesKey;
-      return this;
-    }
-
-    public Builder setSignatureSecretFileKey(String signatureSecretFileKey) {
-      this.signatureSecretFileKey = signatureSecretFileKey;
-      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.isEmpty()) {
-        throw new HadoopIllegalArgumentException("No endpoints specified");
-      }
-
-      if (hostName == null) {
-        hostName = 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, kerberosNameRulesKey,
-            signatureSecretFileKey);
-      }
-
-      for (URI ep : endpoints) {
-        ServerConnector listener = null;
-        String scheme = ep.getScheme();
-        HttpConfiguration httpConfig = new HttpConfiguration();
-        httpConfig.setSecureScheme("https");
-        httpConfig.setHeaderCacheSize(DEFAULT_MAX_HEADER_SIZE);
-        httpConfig.setResponseHeaderSize(DEFAULT_MAX_HEADER_SIZE);
-        httpConfig.setRequestHeaderSize(DEFAULT_MAX_HEADER_SIZE);
-
-        if ("http".equals(scheme)) {
-          listener = new ServerConnector(server.webServer, new HttpConnectionFactory(httpConfig));
-        } else if ("https".equals(scheme)) {
-          HttpConfiguration httpsConfig = new HttpConfiguration(httpConfig);
-          httpsConfig.addCustomizer(new SecureRequestCustomizer());
-          SslContextFactory sslCtxFactory = new SslContextFactory();
-          sslCtxFactory.setNeedClientAuth(needsClientAuth);
-          sslCtxFactory.setKeyManagerPassword(keyPassword);
-
-          if (keyStore != null) {
-            sslCtxFactory.setKeyStorePath(keyStore);
-            sslCtxFactory.setKeyStoreType(keyStoreType);
-            sslCtxFactory.setKeyStorePassword(keyStorePassword);
-          }
-
-          if (trustStore != null) {
-            sslCtxFactory.setTrustStorePath(trustStore);
-            sslCtxFactory.setTrustStoreType(trustStoreType);
-            sslCtxFactory.setTrustStorePassword(trustStorePassword);
-
-          }
-          listener = new ServerConnector(server.webServer, new SslConnectionFactory(sslCtxFactory,
-              HttpVersion.HTTP_1_1.toString()), new HttpConnectionFactory(httpsConfig));
-        } else {
-          throw new HadoopIllegalArgumentException(
-              "unknown scheme for endpoint:" + ep);
-        }
-
-        // default settings for connector
-        listener.setAcceptQueueSize(128);
-        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
-          listener.setReuseAddress(false);
-        }
-
-        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());
-  }
-
-  /**
-   * 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, pathSpecs);
-  }
-
-  /**
-   * Create a status server on the given port.
-   * The jsp scripts are taken from src/webapps/&lt;name&gt;.
-   * @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);
-  }
-
-  @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);
-  }
-
-  /**
-   * Create a status server on the given port.
-   * The jsp scripts are taken from src/webapps/&lt;name&gt;.
-   * @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 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,
-      String[] pathSpecs) throws IOException {
-    this(new Builder().setName(name)
-        .addEndpoint(URI.create("http://" + bindAddress + ":" + port))
-        .setFindPort(findPort).setConf(conf).setACL(adminsAcl)
-        .setPathSpec(pathSpecs));
-  }
-
-  private HttpServer(final Builder b) throws IOException {
-    this.appDir = b.appDir;
-    this.logDir = b.logDir;
-    final String appDir = getWebAppsPath(b.name);
-
-
-    int maxThreads = b.conf.getInt(HTTP_MAX_THREADS, 16);
-    // If HTTP_MAX_THREADS is less than or equal to 0, QueueThreadPool() will use the
-    // default value (currently 200).
-    QueuedThreadPool threadPool = maxThreads <= 0 ? new QueuedThreadPool()
-        : new QueuedThreadPool(maxThreads);
-    threadPool.setDaemon(true);
-    this.webServer = new Server(threadPool);
-
-    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);
-
-    HandlerCollection handlerCollection = new HandlerCollection();
-
-    ContextHandlerCollection contexts = new ContextHandlerCollection();
-    RequestLog requestLog = HttpRequestLog.getRequestLog(name);
-
-    if (requestLog != null) {
-      RequestLogHandler requestLogHandler = new RequestLogHandler();
-      requestLogHandler.setRequestLog(requestLog);
-      handlerCollection.addHandler(requestLogHandler);
-    }
-
-    final String appDir = getWebAppsPath(name);
-
-    handlerCollection.addHandler(contexts);
-    handlerCollection.addHandler(webAppContext);
-
-    webServer.setHandler(handlerCollection);
-
-    addDefaultApps(contexts, appDir, conf);
-
-    addGlobalFilter("safety", QuotingInputFilter.class.getName(), null);
-    Map<String, String> params = new HashMap<>();
-    params.put("xframeoptions", conf.get("hbase.http.filter.xframeoptions.mode", "DENY"));
-    addGlobalFilter("clickjackingprevention",
-            ClickjackingPreventionFilter.class.getName(), params);
-    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 addManagedListener(ServerConnector 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[] { "/*" });
-  }
-
-  /** 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]);
-    }
-    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) {
-      ServletContextHandler logContext = new ServletContextHandler(parent, "/logs");
-      logContext.addServlet(AdminAuthorizedServlet.class, "/*");
-      logContext.setResourceBase(logDir);
-
-      if (conf.getBoolean(
-          ServerConfigurationKeys.HBASE_JETTY_LOGS_SERVE_ALIASES,
-          ServerConfigurationKeys.DEFAULT_HBASE_JETTY_LOGS_SERVE_ALIASES)) {
-        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/*"
-    ServletContextHandler staticContext = new ServletContextHandler(parent, "/static");
-    staticContext.setResourceBase(appDir + "/static");
-    staticContext.addServlet(DefaultServlet.class, "/*");
-    staticContext.setDisplayName("static");
-    setContextAttributes(staticContext, conf);
-    defaultContexts.put(staticContext, true);
-  }
-
-  private void setContextAttributes(ServletContextHandler 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);
-
-    // Hadoop3 has moved completely to metrics2, and  dropped support for Metrics v1's
-    // MetricsServlet (see HADOOP-12504).  We'll using reflection to load if against hadoop2.
-    // Remove when we drop support for hbase on hadoop2.x.
-    try {
-      Class clz = Class.forName("org.apache.hadoop.metrics.MetricsServlet");
-      addServlet("metrics", "/metrics", clz);
-    } catch (Exception e) {
-      // do nothing
-    }
-
-    addServlet("jmx", "/jmx", JMXJsonServlet.class);
-    addServlet("conf", "/conf", ConfServlet.class);
-  }
-
-  /**
-   * 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);
-
-    ResourceConfig application = new ResourceConfig().packages(packageName);
-    final ServletHolder sh = new ServletHolder(new ServletContainer(application));
-    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(FilterMapping.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<ServletContextHandler, Boolean> e : defaultContexts.entrySet()) {
-      if (e.getValue()) {
-        ServletContextHandler handler = e.getKey();
-        defineFilter(handler, name, classname, parameters, ALL_URLS);
-        LOG.info("Added filter " + name + " (class=" + classname
-            + ") to context " + handler.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 (ServletContextHandler 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(ServletContextHandler handler, String name,
-      String classname, Map<String,String> parameters, String[] urls) {
-
-    FilterHolder holder = new FilterHolder();
-    holder.setName(name);
-    holder.setClassName(classname);
-    if (parameters != null) {
-      holder.setInitParameters(parameters);
-    }
-    FilterMapping fmap = new FilterMapping();
-    fmap.setPathSpecs(urls);
-    fmap.setDispatches(FilterMapping.ALL);
-    fmap.setFilterName(name);
-    handler.getServletHandler().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,
-      WebAppContext webAppCtx) {
-    for(String name : filterNames) {
-      FilterMapping fmap = new FilterMapping();
-      fmap.setPathSpec(pathSpec);
-      fmap.setFilterName(name);
-      fmap.setDispatches(FilterMapping.ALL);
-      webAppCtx.getServletHandler().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 ((ServerConnector)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;
-
-    ServerConnector c = (ServerConnector)webServer.getConnectors()[index];
-    if (c.getLocalPort() == -1 || c.getLocalPort() == -2) {
-      // -1 if the connector has not been opened
-      // -2 if it has been closed
-      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, String kerberosNameRuleKey,
-      String signatureSecretKeyFileKey) throws IOException {
-    Map<String, String> params = new HashMap<>();
-    String principalInConf = getOrEmptyString(conf, usernameConfKey);
-    if (!principalInConf.isEmpty()) {
-      params.put(HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX, SecurityUtil.getServerPrincipal(
-          principalInConf, hostName));
-    }
-    String httpKeytab = getOrEmptyString(conf, keytabConfKey);
-    if (!httpKeytab.isEmpty()) {
-      params.put(HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX, httpKeytab);
-    }
-    String kerberosNameRule = getOrEmptyString(conf, kerberosNameRuleKey);
-    if (!kerberosNameRule.isEmpty()) {
-      params.put(HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_SUFFIX, kerberosNameRule);
-    }
-    String signatureSecretKeyFile = getOrEmptyString(conf, signatureSecretKeyFileKey);
-    if (!signatureSecretKeyFile.isEmpty()) {
-      params.put(HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_SUFFIX,
-          signatureSecretKeyFile);
-    }
-    params.put(AuthenticationFilter.AUTH_TYPE, "kerberos");
-
-    // Verify that the required options were provided
-    if (isMissing(params.get(HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_SUFFIX)) ||
-            isMissing(params.get(HTTP_SPNEGO_AUTHENTICATION_KEYTAB_SUFFIX))) {
-      throw new IllegalArgumentException(usernameConfKey + " and "
-          + keytabConfKey + " are both required in the configuration "
-          + "to enable SPNEGO/Kerberos authentication for the Web UI");
-    }
-
-    addGlobalFilter(SPNEGO_FILTER, AuthenticationFilter.class.getName(), params);
-  }
-
-  /**
-   * Returns true if the argument is non-null and not whitespace
-   */
-  private boolean isMissing(String value) {
-    if (null == value) {
-      return true;
-    }
-    return value.trim().isEmpty();
-  }
-
-  /**
-   * Extracts the value for the given key from the configuration of returns a string of
-   * zero length.
-   */
-  private String getOrEmptyString(Configuration conf, String key) {
-    if (null == key) {
-      return EMPTY_STRING;
-    }
-    final String value = conf.get(key.trim());
-    return null == value ? EMPTY_STRING : value;
-  }
-
-  /**
-   * 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) {
-      ServerConnector listener = li.listener;
-      if (!li.isManaged || (li.listener.getLocalPort() != -1 && li.listener.getLocalPort() != -2)) {
-        // This listener is either started externally, or has not been opened, or has been closed
-        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.isEmpty()) {
-      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) {
-        ServerConnector 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><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.
-   * </p>
-   *
-   * @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");
-      try (PrintStream out = new PrintStream(
-        response.getOutputStream(), false, "UTF-8")) {
-        Threads.printThreadInfo(out, "");
-        out.flush();
-      }
-      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.
-   */
-  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-  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.
-       */
-      @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;
-      }
-
-      @Override
-      public Map<String, String[]> getParameterMap() {
-        Map<String, String[]> result = new HashMap<>();
-        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();
-      ServletContext context = config.getServletContext();
-      return context.getMimeType(path);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServerUtil.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServerUtil.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServerUtil.java
deleted file mode 100644
index 0b33fd1..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpServerUtil.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * 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.eclipse.jetty.security.ConstraintSecurityHandler;
-import org.eclipse.jetty.util.security.Constraint;
-import org.eclipse.jetty.security.ConstraintMapping;
-import org.eclipse.jetty.servlet.ServletContextHandler;
-
-/**
- * HttpServer utility.
- */
-public class HttpServerUtil {
-  /**
-   * Add constraints to a Jetty Context to disallow undesirable Http methods.
-   * @param ctxHandler The context to modify
-   */
-  public static void constrainHttpMethods(ServletContextHandler ctxHandler) {
-    Constraint c = new Constraint();
-    c.setAuthenticate(true);
-
-    ConstraintMapping cmt = new ConstraintMapping();
-    cmt.setConstraint(c);
-    cmt.setMethod("TRACE");
-    cmt.setPathSpec("/*");
-
-    ConstraintMapping cmo = new ConstraintMapping();
-    cmo.setConstraint(c);
-    cmo.setMethod("OPTIONS");
-    cmo.setPathSpec("/*");
-
-    ConstraintSecurityHandler securityHandler = new ConstraintSecurityHandler();
-    securityHandler.setConstraintMappings(new ConstraintMapping[]{ cmt, cmo });
-
-    ctxHandler.setSecurityHandler(securityHandler);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java
deleted file mode 100644
index 5fd6514..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/InfoServer.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/**
- *
- * 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.hbase.HBaseConfiguration;
-import org.apache.yetus.audience.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/" -&gt; points to stack trace
- *   "/static/" -&gt; points to common static files (src/hbase-webapps/static)
- *   "/" -&gt; the jsp server code from (src/hbase-webapps/&lt;name&gt;)
- */
-@InterfaceAudience.Private
-public class InfoServer {
-
-  private static final String HBASE_APP_DIR = "hbase-webapps";
-  private final org.apache.hadoop.hbase.http.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 {
-    HttpConfig httpConfig = new HttpConfig(c);
-    HttpServer.Builder builder =
-      new org.apache.hadoop.hbase.http.HttpServer.Builder();
-
-      builder.setName(name).addEndpoint(URI.create(httpConfig.getSchemePrefix() +
-        bindAddress + ":" +
-        port)).setAppDir(HBASE_APP_DIR).setFindPort(findPort).setConf(c);
-      String logDir = System.getProperty("hbase.log.dir");
-      if (logDir != null) {
-        builder.setLogDir(logDir);
-      }
-    if (httpConfig.isSecure()) {
-    builder.keyPassword(HBaseConfiguration.getPassword(c, "ssl.server.keystore.keypassword", null))
-      .keyStore(c.get("ssl.server.keystore.location"),
-        HBaseConfiguration.getPassword(c,"ssl.server.keystore.password", null),
-        c.get("ssl.server.keystore.type", "jks"))
-      .trustStore(c.get("ssl.server.truststore.location"),
-        HBaseConfiguration.getPassword(c, "ssl.server.truststore.password", null),
-        c.get("ssl.server.truststore.type", "jks"));
-    }
-    // Enable SPNEGO authentication
-    if ("kerberos".equalsIgnoreCase(c.get(HttpServer.HTTP_UI_AUTHENTICATION, null))) {
-      builder.setUsernameConfKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_KEY)
-        .setKeytabConfKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_KEYTAB_KEY)
-        .setKerberosNameRulesKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_KRB_NAME_KEY)
-        .setSignatureSecretFileKey(
-            HttpServer.HTTP_AUTHENTICATION_SIGNATURE_SECRET_FILE_KEY)
-        .setSecurityEnabled(true);
-    }
-    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/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/NoCacheFilter.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/NoCacheFilter.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/NoCacheFilter.java
deleted file mode 100644
index a1daf15..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/NoCacheFilter.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * 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;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-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/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ServerConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ServerConfigurationKeys.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ServerConfigurationKeys.java
deleted file mode 100644
index 8f338a7..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ServerConfigurationKeys.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * 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.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.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";
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/conf/ConfServlet.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/conf/ConfServlet.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/conf/ConfServlet.java
deleted file mode 100644
index d9aa7b6..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/conf/ConfServlet.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/**
- * 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.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.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/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
deleted file mode 100644
index 2e43be2..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/jmx/JMXJsonServlet.java
+++ /dev/null
@@ -1,240 +0,0 @@
-/*
- * 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 javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import javax.management.openmbean.CompositeData;
-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.apache.hadoop.hbase.util.JSONBean;
-
-/*
- * 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>
- * <p>
- * For example <code>http://.../jmx?qry=Hadoop:*</code> will return
- * all hadoop metrics exposed through JMX.
- * </p>
- * <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>
- * <p>
- * For example
- * <code>
- * http://../jmx?get=Hadoop:service=NameNode,name=NameNodeInfo::ClusterId
- * </code> will return the cluster id of the namenode mxbean.
- * </p>
- * <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>
- * <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>
- * <p>
- * The return format is JSON and in the form
- * </p>
- *  <pre><code>
- *  {
- *    "beans" : [
- *      {
- *        "name":"bean-name"
- *        ...
- *      }
- *    ]
- *  }
- *  </code></pre>
- *  <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.
- * </p>
- *
- */
-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";
-  /**
-   * If query string includes 'description', then we will emit bean and attribute descriptions to
-   * output IFF they are not null and IFF the description is not the same as the attribute name:
-   * i.e. specify an URL like so: /jmx?description=true
-   */
-  private static final String INCLUDE_DESCRIPTION = "description";
-
-  /**
-   * MBean server.
-   */
-  protected transient MBeanServer mBeanServer;
-
-  protected transient JSONBean jsonBeanWriter;
-
-  /**
-   * Initialize this servlet.
-   */
-  @Override
-  public void init() throws ServletException {
-    // Retrieve the MBean server
-    mBeanServer = ManagementFactory.getPlatformMBeanServer();
-    this.jsonBeanWriter = new JSONBean();
-  }
-
-  /**
-   * 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) throws IOException {
-    try {
-      if (!HttpServer.isInstrumentationAccessAllowed(getServletContext(), request, response)) {
-        return;
-      }
-      String jsonpcb = null;
-      PrintWriter writer = null;
-      JSONBean.Writer beanWriter = null;
-      try {
-        jsonpcb = checkCallbackName(request.getParameter(CALLBACK_PARAM));
-        writer = response.getWriter();
-        beanWriter = this.jsonBeanWriter.open(writer);
-
-        // "callback" parameter implies JSONP outpout
-        if (jsonpcb != null) {
-          response.setContentType("application/javascript; charset=utf8");
-          writer.write(jsonpcb + "(");
-        } else {
-          response.setContentType("application/json; charset=utf8");
-        }
-        // Should we output description on each attribute and bean?
-        String tmpStr = request.getParameter(INCLUDE_DESCRIPTION);
-        boolean description = tmpStr != null && tmpStr.length() > 0;
-
-        // query per mbean attribute
-        String getmethod = request.getParameter("get");
-        if (getmethod != null) {
-          String[] splitStrings = getmethod.split("\\:\\:");
-          if (splitStrings.length != 2) {
-            beanWriter.write("result", "ERROR");
-            beanWriter.write("message", "query format is not as expected.");
-            beanWriter.flush();
-            response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
-            return;
-          }
-          if (beanWriter.write(this.mBeanServer, new ObjectName(splitStrings[0]),
-              splitStrings[1], description) != 0) {
-            beanWriter.flush();
-            response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
-          }
-          return;
-        }
-
-        // query per mbean
-        String qry = request.getParameter("qry");
-        if (qry == null) {
-          qry = "*:*";
-        }
-        if (beanWriter.write(this.mBeanServer, new ObjectName(qry), null, description) != 0) {
-          beanWriter.flush();
-          response.setStatus(HttpServletResponse.SC_BAD_REQUEST);
-        }
-      } finally {
-        if (beanWriter != null) beanWriter.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.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR);
-    } catch (MalformedObjectNameException e) {
-      LOG.error("Caught an exception while processing JMX request", e);
-      response.sendError(HttpServletResponse.SC_BAD_REQUEST);
-    }
-  }
-
-  /**
-   * Verifies that the callback property, if provided, is purely alphanumeric.
-   * This prevents a malicious callback name (that is javascript code) from being
-   * returned by the UI to an unsuspecting user.
-   *
-   * @param callbackName The callback name, can be null.
-   * @return The callback name
-   * @throws IOException If the name is disallowed.
-   */
-  private String checkCallbackName(String callbackName) throws IOException {
-    if (null == callbackName) {
-      return null;
-    }
-    if (callbackName.matches("[A-Za-z0-9_]+")) {
-      return callbackName;
-    }
-    throw new IOException("'callback' must be alphanumeric");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/jmx/package-info.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/jmx/package-info.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/jmx/package-info.java
deleted file mode 100644
index 21667d7..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/jmx/package-info.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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;

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/lib/StaticUserWebFilter.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/lib/StaticUserWebFilter.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/lib/StaticUserWebFilter.java
deleted file mode 100644
index a1fa9f0..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/lib/StaticUserWebFilter.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/**
- * 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.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-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.
- */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-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;
-    }
-  }
-
-  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-  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 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/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/lib/package-info.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/lib/package-info.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/lib/package-info.java
deleted file mode 100644
index 7bb9a0f..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/lib/package-info.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;


[10/10] hbase git commit: HBASE-19124 Move nightly test of source artifact creation from Jenkinsfile to a script.

Posted by bu...@apache.org.
HBASE-19124 Move nightly test of source artifact creation from Jenkinsfile to a script.


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

Branch: refs/heads/HBASE-19124
Commit: ac6aff3e3fa958cdf8a9bd545ad660d35a3be5a6
Parents: 91273e7
Author: Sean Busbey <bu...@apache.org>
Authored: Wed Nov 1 08:12:03 2017 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Wed Nov 1 10:32:27 2017 -0500

----------------------------------------------------------------------
 dev-support/Jenkinsfile                      |  61 +++-----
 dev-support/hbase_nightly_source-artifact.sh | 168 ++++++++++++++++++++++
 2 files changed, 186 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ac6aff3e/dev-support/Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index d7faeb6..5bd41aa 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -258,51 +258,26 @@ curl -L  -o personality.sh "${env.PROJET_PERSONALITY}"
         jdk "JDK 1.8 (latest)"
       }
       steps {
-        sh 'mvn --version --offline'
-        dir ("component") {
-          sh '''#!/bin/bash -e
-            echo "Clean..."
-            mvn --batch-mode -DskipTests clean
-            echo "Step 3 Build the source tarball"
-            mvn -Prelease --batch-mode -Dmaven.repo.local="${WORKSPACE}/.m2-for-repo" install -DskipTests assembly:single -Dassembly.file=hbase-assembly/src/main/assembly/src.xml
-'''
-        }
-        dir ("unpacked_src_tarball") {
-          sh '''#!/bin/bash -e
-            echo "Unpack the source tarball"
-            tar --strip-components=1 -xzf ../component/hbase-assembly/target/hbase-*-src.tar.gz
-            echo "Building from source artifact."
-            mvn -DskipTests -Prelease --batch-mode -Dmaven.repo.local="${WORKSPACE}/.m2-for-src" clean install assembly:single
-            echo "Clean up after checking ability to build."
-            mvn -DskipTests --batch-mode clean
-'''
-        }
-        dir ("component") {
-          sh '''#!/bin/bash -e
-            echo "Clean up the source checkout"
-            mvn -DskipTests --batch-mode clean
-            echo "Diff against source tree"
-            diff --binary --recursive . ../unpacked_src_tarball >../diff_output || true
-'''
-        }
-        // expectation check largely based on HBASE-14952
         sh '''#!/bin/bash -e
-          echo "Checking against things we don't expect to include in the source tarball (git related, hbase-native-client, etc.)"
-          cat >known_excluded <<END
-Only in .: .git
-Only in .: .gitattributes
-Only in .: .gitignore
-Only in .: hbase-native-client
-END
-          if ! diff known_excluded diff_output >/dev/null ; then
-            echo "Any output here are unexpected differences between the source artifact we'd make for an RC and the current branch."
-            echo "The expected differences are on the < side and the current differences are on the > side."
-            echo "In a given set of differences, '.' refers to the branch in the repo and 'unpacked_src_tarball' refers to what we pulled out of the tarball."
-            diff known_excluded diff_output
-          else
-            echo "Everything looks as expected."
-          fi
+          echo "Setting up directories"
+          rm -rf "output-srctarball" && mkdir "output-srctarball"
+          rm -rf "unpacked_src_tarball" && mkdir "unpacked_src_tarball"
+          rm -rf ".m2-for-repo" && mkdir ".m2-for-repo"
+          rm -rf ".m2-for-src" && mkdir ".m2-for-src"
 '''
+        sh """#!/bin/bash -e
+          ${env.BASEDIR}/dev-support/hbase_nightly_source-artifact.sh \
+              --intermediate-file-dir output-srctarball \
+              --unpack-temp-dir unpacked_src_tarball \
+              --maven-m2-initial .m2-for-repo \
+              --maven-m2-src-build .m2-for-src \
+              ${env.BASEDIR}
+"""
+      }
+      post {
+        always {
+          archive 'output-srctarball/*'
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ac6aff3e/dev-support/hbase_nightly_source-artifact.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase_nightly_source-artifact.sh b/dev-support/hbase_nightly_source-artifact.sh
new file mode 100755
index 0000000..2b64d13
--- /dev/null
+++ b/dev-support/hbase_nightly_source-artifact.sh
@@ -0,0 +1,168 @@
+#!/usr/bin/env bash
+# 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.
+
+set -e
+function usage {
+  echo "Usage: ${0} [options] /path/to/component/checkout"
+  echo ""
+  echo "    --intermediate-file-dir /path/to/use  Path for writing listings and diffs. must exist."
+  echo "                                          defaults to making a directory in /tmp."
+  echo "    --unpack-temp-dir /path/to/use        Path for unpacking tarball. default to"
+  echo "                                          'unpacked_src_tarball' in intermediate directory."
+  echo "    --maven-m2-initial /path/to/use       Path for maven artifacts while building in"
+  echo "                                          component-dir."
+  echo "    --maven-m2-src-build /path/to/use     Path for maven artifacts while building from the"
+  echo "                                          unpacked source tarball."
+  exit 1
+}
+# if no args specified, show usage
+if [ $# -lt 1 ]; then
+  usage
+fi
+
+# Get arguments
+declare component_dir
+declare unpack_dir
+declare m2_initial
+declare m2_tarbuild
+declare working_dir
+while [ $# -gt 0 ]
+do
+  case "$1" in
+    --unpack-temp-dir) shift; unpack_dir=$1; shift;;
+    --maven-m2-initial) shift; m2_initial=$1; shift;;
+    --maven-m2-src-build) shift; m2_tarbuild=$1; shift;;
+    --intermediate-file-dir) shift; working_dir=$1; shift;;
+    --) shift; break;;
+    -*) usage ;;
+    *)  break;;	# terminate while loop
+  esac
+done
+
+# should still have where component checkout is.
+if [ $# -lt 1 ]; then
+  usage
+fi
+component_dir="$(cd "$(dirname "$1")"; pwd)/$(basename "$1")"
+
+if [ -z "${working_dir}" ]; then
+  working_dir=/tmp
+  while [[ -e ${working_dir} ]]; do
+    working_dir=/tmp/hbase-srctarball-test-${RANDOM}.${RANDOM}
+  done
+  mkdir "${working_dir}"
+else
+  # absolutes please
+  working_dir="$(cd "$(dirname "${working_dir}")"; pwd)/$(basename "${working_dir}")"
+  if [ ! -d "${working_dir}" ]; then
+    echo "passed working directory '${working_dir}' must already exist."
+    exit 1
+  fi
+fi
+
+echo "You'll find logs and temp files in ${working_dir}"
+
+if [ -z "${unpack_dir}" ]; then
+  unpack_dir="${working_dir}/unpacked_src_tarball"
+  mkdir "${unpack_dir}"
+else
+  # absolutes please
+  unpack_dir="$(cd "$(dirname "${unpack_dir}")"; pwd)/$(basename "${unpack_dir}")"
+  if [ ! -d "${unpack_dir}" ]; then
+    echo "passed directory for unpacking the source tarball '${unpack_dir}' must already exist."
+    exit 1
+  fi
+  rm -rf "${unpack_dir:?}/*"
+fi
+
+if [ -z "${m2_initial}" ]; then
+  m2_initial="${working_dir}/.m2-initial"
+  mkdir "${m2_initial}"
+else
+  # absolutes please
+  m2_initial="$(cd "$(dirname "${m2_initial}")"; pwd)/$(basename "${m2_initial}")"
+  if [ ! -d "${m2_initial}" ]; then
+    echo "passed directory for storing the initial build's maven repo  '${m2_initial}' " \
+        "must already exist."
+    exit 1
+  fi
+fi
+
+if [ -z "${m2_tarbuild}" ]; then
+  m2_tarbuild="${working_dir}/.m2-tarbuild"
+  mkdir "${m2_tarbuild}"
+else
+  # absolutes please
+  m2_tarbuild="$(cd "$(dirname "${m2_tarbuild}")"; pwd)/$(basename "${m2_tarbuild}")"
+  if [ ! -d "${m2_tarbuild}" ]; then
+    echo "passed directory for storing the build from src tarball's maven repo  '${m2_tarbuild}' " \
+        "must already exist."
+    exit 1
+  fi
+fi
+
+# This is meant to mimic what a release manager will do to create RCs.
+# See http://hbase.apache.org/book.html#maven.release
+
+echo "Maven details, in case our JDK doesn't match expectations:"
+mvn --version --offline | tee "${working_dir}/maven_version"
+
+echo "Do a clean building of the source artifact using code in ${component_dir}"
+cd "${component_dir}"
+echo "Clean..."
+mvn --batch-mode -DskipTests clean >"${working_dir}/component_clean_before.log" 2>&1
+echo "Step 3 Build the source tarball"
+mvn -Prelease --batch-mode -Dmaven.repo.local="${m2_initial}" install -DskipTests assembly:single \
+    -Dassembly.file=hbase-assembly/src/main/assembly/src.xml \
+    >"${working_dir}/component_build_src_tarball.log" 2>&1
+
+cd "${unpack_dir}"
+echo "Unpack the source tarball"
+tar --strip-components=1 -xzf "${component_dir}"/hbase-assembly/target/hbase-*-src.tar.gz \
+    >"${working_dir}/srctarball_unpack.log" 2>&1
+echo "Building from source artifact."
+mvn -DskipTests -Prelease --batch-mode -Dmaven.repo.local="${m2_tarbuild}" clean install \
+    assembly:single >"${working_dir}/srctarball_install.log" 2>&1
+
+echo "Clean up after checking ability to build."
+mvn -DskipTests --batch-mode clean >"${working_dir}/srctarball_clean.log" 2>&1
+
+
+cd "${component_dir}"
+echo "Clean up the source checkout"
+mvn --batch-mode -DskipTests clean >"${working_dir}/component_clean_after.log" 2>&1
+echo "Diff against source tree"
+diff --binary --recursive . "${unpack_dir}" >"${working_dir}/diff_output" || true
+
+cd "${working_dir}"
+# expectation check largely based on HBASE-14952
+echo "Checking against things we don't expect to include in the source tarball (git related, hbase-native-client, etc.)"
+cat >known_excluded <<END
+Only in .: .git
+Only in .: .gitattributes
+Only in .: .gitignore
+Only in .: hbase-native-client
+END
+if ! diff known_excluded diff_output >"${working_dir}/unexpected.diff" ; then
+  echo "Any output here are unexpected differences between the source artifact we'd make for an RC and the current branch."
+  echo "The expected differences are on the < side and the current differences are on the > side."
+  echo "In a given set of differences, '.' refers to the branch in the repo and 'unpacked_src_tarball' refers to what we pulled out of the tarball."
+  diff known_excluded diff_output
+else
+  echo "Everything looks as expected."
+fi


[03/10] hbase git commit: Revert "HBASE-19053 Split out o.a.h.h.http from hbase-server into a separate module"

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..e23eecd
--- /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.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.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();
+      try (InputStreamReader streamReader = new InputStreamReader(connection.getInputStream());
+           BufferedReader bufferedReader = new BufferedReader(streamReader)) {
+        for(String line; (line = bufferedReader.readLine()) != null; ) {
+          if (line.startsWith(MARKER)) {
+            System.out.println(TAG.matcher(line).replaceAll(""));
+          }
+        }
+      }
+    } 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/d8a7b7b3/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..f55e24b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/http/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * 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>
+ * 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.yetus.audience.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DumpRegionServerMetrics.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DumpRegionServerMetrics.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DumpRegionServerMetrics.java
deleted file mode 100644
index 2b07a64..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DumpRegionServerMetrics.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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.regionserver;
-
-import org.apache.hadoop.hbase.util.JSONBean;
-
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.lang.management.ManagementFactory;
-import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-
-/**
- * Utility for doing JSON and MBeans.
- */
-public class DumpRegionServerMetrics {
-  /**
-   * Dump out a subset of regionserver mbeans only, not all of them, as json on System.out.
-   */
-  public static String dumpMetrics() throws MalformedObjectNameException, IOException {
-    StringWriter sw = new StringWriter(1024 * 100); // Guess this size
-    try (PrintWriter writer = new PrintWriter(sw)) {
-      JSONBean dumper = new JSONBean();
-      try (JSONBean.Writer jsonBeanWriter = dumper.open(writer)) {
-        MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer();
-        jsonBeanWriter.write(mbeanServer,
-          new ObjectName("java.lang:type=Memory"), null, false);
-        jsonBeanWriter.write(mbeanServer,
-          new ObjectName("Hadoop:service=HBase,name=RegionServer,sub=IPC"), null, false);
-        jsonBeanWriter.write(mbeanServer,
-          new ObjectName("Hadoop:service=HBase,name=RegionServer,sub=Replication"), null, false);
-        jsonBeanWriter.write(mbeanServer,
-          new ObjectName("Hadoop:service=HBase,name=RegionServer,sub=Server"), null, false);
-      }
-    }
-    sw.close();
-    return sw.toString();
-  }
-
-  public static void main(String[] args) throws IOException, MalformedObjectNameException {
-    String str = dumpMetrics();
-    System.out.println(str);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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 f384c1f..ef3f10d 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
@@ -146,6 +146,7 @@ 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.JSONBean;
 import org.apache.hadoop.hbase.util.JvmPauseMonitor;
 import org.apache.hadoop.hbase.util.NettyEventLoopGroupConfig;
 import org.apache.hadoop.hbase.util.Pair;
@@ -2400,7 +2401,7 @@ public class HRegionServer extends HasThread implements
         CoprocessorHost.getLoadedCoprocessors());
     // Try and dump metrics if abort -- might give clue as to how fatal came about....
     try {
-      LOG.info("Dump of metrics as JSON on abort: " + DumpRegionServerMetrics.dumpMetrics());
+      LOG.info("Dump of metrics as JSON on abort: " + JSONBean.dumpRegionServerMetrics());
     } catch (MalformedObjectNameException | IOException e) {
       LOG.warn("Failed dumping metrics", e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HttpServerUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HttpServerUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HttpServerUtil.java
new file mode 100644
index 0000000..59c6ad6
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HttpServerUtil.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.util;
+
+import org.eclipse.jetty.security.ConstraintSecurityHandler;
+import org.eclipse.jetty.util.security.Constraint;
+import org.eclipse.jetty.security.ConstraintMapping;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+
+/**
+ * HttpServer utility.
+ */
+public class HttpServerUtil {
+  /**
+   * Add constraints to a Jetty Context to disallow undesirable Http methods.
+   * @param ctxHandler The context to modify
+   */
+  public static void constrainHttpMethods(ServletContextHandler ctxHandler) {
+    Constraint c = new Constraint();
+    c.setAuthenticate(true);
+
+    ConstraintMapping cmt = new ConstraintMapping();
+    cmt.setConstraint(c);
+    cmt.setMethod("TRACE");
+    cmt.setPathSpec("/*");
+
+    ConstraintMapping cmo = new ConstraintMapping();
+    cmo.setConstraint(c);
+    cmo.setMethod("OPTIONS");
+    cmo.setPathSpec("/*");
+
+    ConstraintSecurityHandler securityHandler = new ConstraintSecurityHandler();
+    securityHandler.setConstraintMappings(new ConstraintMapping[]{ cmt, cmo });
+
+    ctxHandler.setSecurityHandler(securityHandler);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
new file mode 100644
index 0000000..f4a146e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
@@ -0,0 +1,387 @@
+/*
+ * 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.util;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+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 com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerationException;
+import com.fasterxml.jackson.core.JsonGenerator;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Utility for doing JSON and MBeans.
+ */
+public class JSONBean {
+  private static final Log LOG = LogFactory.getLog(JSONBean.class);
+  private final JsonFactory jsonFactory;
+
+  public JSONBean() {
+    this.jsonFactory = new JsonFactory();
+  }
+
+  /**
+   * Use dumping out mbeans as JSON.
+   */
+  public interface Writer extends Closeable {
+    void write(final String key, final String value) throws JsonGenerationException, IOException;
+    int write(final MBeanServer mBeanServer, ObjectName qry, String attribute,
+        final boolean description) throws IOException;
+    void flush() throws IOException;
+  }
+
+  public Writer open(final PrintWriter writer) throws IOException {
+    final JsonGenerator jg = jsonFactory.createJsonGenerator(writer);
+    jg.disable(JsonGenerator.Feature.AUTO_CLOSE_TARGET);
+    jg.useDefaultPrettyPrinter();
+    jg.writeStartObject();
+    return new Writer() {
+      @Override
+      public void flush() throws IOException {
+        jg.flush();
+      }
+
+      @Override
+      public void close() throws IOException {
+        jg.close();
+      }
+
+      @Override
+      public void write(String key, String value) throws JsonGenerationException, IOException {
+        jg.writeStringField(key, value);
+      }
+
+      @Override
+      public int write(MBeanServer mBeanServer, ObjectName qry, String attribute,
+          boolean description)
+      throws IOException {
+        return JSONBean.write(jg, mBeanServer, qry, attribute, description);
+      }
+    };
+  }
+
+  /**
+   * @param mBeanServer
+   * @param qry
+   * @param attribute
+   * @param description
+   * @return Return non-zero if failed to find bean. 0
+   * @throws IOException
+   */
+  private static int write(final JsonGenerator jg,
+      final MBeanServer mBeanServer, ObjectName qry, String attribute,
+      final boolean description)
+  throws IOException {
+    LOG.trace("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 = "";
+      String descriptionStr = null;
+      Object attributeinfo = null;
+      try {
+        minfo = mBeanServer.getMBeanInfo(oname);
+        code = minfo.getClassName();
+        if (description) descriptionStr = minfo.getDescription();
+        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 (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) {
+           if (LOG.isTraceEnabled()) {
+             LOG.trace("Getting attribute " + prs + " of " + oname + " threw " + e);
+           }
+         } else {
+           LOG.error("Getting attribute " + prs + " of " + oname + " threw an exception", e);
+         }
+         return 0;
+        } 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());
+      if (description && descriptionStr != null && descriptionStr.length() > 0) {
+        jg.writeStringField("description", descriptionStr);
+      }
+      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();
+        return -1;
+      }
+
+      if (attribute != null) {
+        writeAttribute(jg, attribute, descriptionStr, attributeinfo);
+      } else {
+        MBeanAttributeInfo[] attrs = minfo.getAttributes();
+        for (int i = 0; i < attrs.length; i++) {
+          writeAttribute(jg, mBeanServer, oname, description, attrs[i]);
+        }
+      }
+      jg.writeEndObject();
+    }
+    jg.writeEndArray();
+    return 0;
+  }
+
+  private static void writeAttribute(final JsonGenerator jg,
+      final MBeanServer mBeanServer, ObjectName oname,
+      final boolean description, final 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;
+    }
+    String descriptionStr = description? attr.getDescription(): null;
+    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) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Getting attribute " + attName + " of " + oname + " threw " + 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, descriptionStr, value);
+  }
+
+  private static void writeAttribute(JsonGenerator jg, String attName, final String descriptionStr,
+      Object value)
+  throws IOException {
+    boolean description = false;
+    if (descriptionStr != null && descriptionStr.length() > 0 && !attName.equals(descriptionStr)) {
+      description = true;
+      jg.writeFieldName(attName);
+      jg.writeStartObject();
+      jg.writeFieldName("description");
+      jg.writeString(descriptionStr);
+      jg.writeFieldName("value");
+      writeObject(jg, description, value);
+      jg.writeEndObject();
+    } else {
+      jg.writeFieldName(attName);
+      writeObject(jg, description, value);
+    }
+  }
+
+  private static void writeObject(final JsonGenerator jg, final boolean description, 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, description, 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, null, cds.get(key));
+        }
+        jg.writeEndObject();
+      } else if(value instanceof TabularData) {
+        TabularData tds = (TabularData)value;
+        jg.writeStartArray();
+        for(Object entry : tds.values()) {
+          writeObject(jg, description, entry);
+        }
+        jg.writeEndArray();
+      } else {
+        jg.writeString(value.toString());
+      }
+    }
+  }
+
+  /**
+   * Dump out a subset of regionserver mbeans only, not all of them, as json on System.out.
+   * @throws MalformedObjectNameException
+   * @throws IOException
+   */
+  public static String dumpRegionServerMetrics() throws MalformedObjectNameException, IOException {
+    StringWriter sw = new StringWriter(1024 * 100); // Guess this size
+    try (PrintWriter writer = new PrintWriter(sw)) {
+      JSONBean dumper = new JSONBean();
+      try (JSONBean.Writer jsonBeanWriter = dumper.open(writer)) {
+        MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer();
+        jsonBeanWriter.write(mbeanServer,
+          new ObjectName("java.lang:type=Memory"), null, false);
+        jsonBeanWriter.write(mbeanServer,
+          new ObjectName("Hadoop:service=HBase,name=RegionServer,sub=IPC"), null, false);
+        jsonBeanWriter.write(mbeanServer,
+          new ObjectName("Hadoop:service=HBase,name=RegionServer,sub=Replication"), null, false);
+        jsonBeanWriter.write(mbeanServer,
+          new ObjectName("Hadoop:service=HBase,name=RegionServer,sub=Server"), null, false);
+      }
+    }
+    sw.close();
+    return sw.toString();
+  }
+
+  /**
+   * Dump out all registered mbeans as json on System.out.
+   * @throws IOException
+   * @throws MalformedObjectNameException
+   */
+  public static void dumpAllBeans() throws IOException, MalformedObjectNameException {
+    try (PrintWriter writer = new PrintWriter(System.out)) {
+      JSONBean dumper = new JSONBean();
+      try (JSONBean.Writer jsonBeanWriter = dumper.open(writer)) {
+        MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer();
+        jsonBeanWriter.write(mbeanServer, new ObjectName("*:*"), null, false);
+      }
+    }
+  }
+
+  public static void main(String[] args) throws IOException, MalformedObjectNameException {
+    String str = dumpRegionServerMetrics();
+    System.out.println(str);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JSONMetricUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JSONMetricUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JSONMetricUtil.java
new file mode 100644
index 0000000..879f32e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JSONMetricUtil.java
@@ -0,0 +1,214 @@
+/**
+ * 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.util;
+
+import java.beans.IntrospectionException;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.lang.management.GarbageCollectorMXBean;
+import java.lang.management.ManagementFactory;
+import java.lang.management.MemoryPoolMXBean;
+import java.lang.management.RuntimeMXBean;
+import java.util.Hashtable;
+import java.util.List;
+import java.util.Set;
+
+import javax.management.InstanceNotFoundException;
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanInfo;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.ReflectionException;
+import javax.management.openmbean.CompositeData;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.codehaus.jettison.json.JSONException;
+
+public final class JSONMetricUtil {
+
+  private static final Log LOG = LogFactory.getLog(JSONMetricUtil.class);
+
+  private static MBeanServer mbServer = ManagementFactory.getPlatformMBeanServer();
+  //MBeans ObjectName domain names
+  public static final String JAVA_LANG_DOMAIN = "java.lang";
+  public static final String JAVA_NIO_DOMAIN = "java.nio";
+  public static final String SUN_MGMT_DOMAIN = "com.sun.management";
+  public static final String HADOOP_DOMAIN = "Hadoop";
+
+  //MBeans ObjectName properties key names
+  public static final String TYPE_KEY = "type";
+  public static final String NAME_KEY = "name";
+  public static final String SERVICE_KEY = "service";
+  public static final String SUBSYSTEM_KEY = "sub";
+
+/**
+ * Utility for getting metric values. Collection of static methods intended for
+ * easier access to metric values.
+ */
+  private JSONMetricUtil() {
+    // Not to be called
+  }
+
+  public static MBeanAttributeInfo[] getMBeanAttributeInfo(ObjectName bean)
+      throws IntrospectionException, InstanceNotFoundException, ReflectionException,
+      IntrospectionException, javax.management.IntrospectionException {
+    MBeanInfo mbinfo = mbServer.getMBeanInfo(bean);
+    return mbinfo.getAttributes();
+  }
+
+  public static Object getValueFromMBean(ObjectName bean, String attribute) {
+    Object value = null;
+    try {
+      value = mbServer.getAttribute(bean, attribute);
+    }
+    catch(Exception e) {
+      LOG.error("Unable to get value from MBean= "+ bean.toString() +
+        "for attribute=" + attribute + " " + e.getMessage());
+    }
+    return value;
+  }
+
+  /**
+   * Returns a subset of mbeans defined by qry.
+   * Modeled after {@link JSONBean#dumpRegionServerMetrics()}
+   * Example: String qry= "java.lang:type=Memory"
+   * @throws MalformedObjectNameException if json have bad format
+   * @throws IOException /
+   * @return String representation of json array.
+   */
+  public static String dumpBeanToString(String qry) throws MalformedObjectNameException,
+  IOException {
+    StringWriter sw = new StringWriter(1024 * 100); // Guess this size
+    try (PrintWriter writer = new PrintWriter(sw)) {
+      JSONBean dumper = new JSONBean();
+      try (JSONBean.Writer jsonBeanWriter = dumper.open(writer)) {
+        MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer();
+        jsonBeanWriter.write(mbeanServer,
+          new ObjectName(qry), null, false);
+      }
+    }
+    sw.close();
+    return sw.toString();
+  }
+
+  public static JsonNode mappStringToJsonNode(String jsonString)
+      throws JsonProcessingException, IOException {
+    ObjectMapper mapper = new ObjectMapper();
+    JsonNode node = mapper.readTree(jsonString);
+    return node;
+  }
+
+
+  public static JsonNode searchJson(JsonNode tree, String searchKey)
+      throws JsonProcessingException, IOException {
+    if (tree == null) {
+      return null;
+    }
+    if(tree.has(searchKey)) {
+      return tree.get(searchKey);
+    }
+    if(tree.isContainerNode()) {
+      for(JsonNode branch: tree) {
+        JsonNode branchResult = searchJson(branch, searchKey);
+        if (branchResult != null && !branchResult.isMissingNode()) {
+          return branchResult;
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Method for building hashtable used for constructing ObjectName.
+   * Mapping is done with arrays indices
+   * @param keys Hashtable keys
+   * @param values Hashtable values
+   * @return Hashtable or null if arrays are empty * or have different number of elements
+   */
+  public static Hashtable<String, String> buldKeyValueTable(String[] keys, String[] values) {
+    if (keys.length != values.length) {
+      LOG.error("keys and values arrays must be same size");
+      return null;
+    }
+    if (keys.length == 0 || values.length == 0) {
+      LOG.error("keys and values arrays can not be empty;");
+      return null;
+    }
+    Hashtable<String, String> table = new Hashtable<String, String>();
+    for(int i = 0; i < keys.length; i++) {
+      table.put(keys[i], values[i]);
+    }
+    return table;
+  }
+
+  public static ObjectName buildObjectName(String pattern) throws MalformedObjectNameException {
+    return new ObjectName(pattern);
+  }
+
+  public static ObjectName buildObjectName(String domain, Hashtable<String, String> keyValueTable)
+      throws MalformedObjectNameException {
+    return new ObjectName(domain, keyValueTable);
+  }
+
+  public static Set<ObjectName> getRegistredMBeans(ObjectName name, MBeanServer mbs) {
+    return mbs.queryNames(name, null);
+  }
+
+  public static String getProcessPID() {
+    return ManagementFactory.getRuntimeMXBean().getName().split("@")[0];
+  }
+
+  public static String getCommmand() throws MalformedObjectNameException,
+  IOException, JSONException {
+    RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean();
+    return runtimeBean.getSystemProperties().get("sun.java.command");
+  }
+
+  public static List<GarbageCollectorMXBean> getGcCollectorBeans() {
+    List<GarbageCollectorMXBean> gcBeans = ManagementFactory.getGarbageCollectorMXBeans();
+    return gcBeans;
+  }
+
+  public static long getLastGcDuration(ObjectName gcCollector) {
+    long lastGcDuration = 0;
+    Object lastGcInfo = getValueFromMBean(gcCollector, "LastGcInfo");
+    if (lastGcInfo != null && lastGcInfo instanceof CompositeData) {
+      CompositeData cds = (CompositeData)lastGcInfo;
+      lastGcDuration = (long) cds.get("duration");
+    }
+    return lastGcDuration;
+  }
+
+  public static List<MemoryPoolMXBean> getMemoryPools() {
+    List<MemoryPoolMXBean> mPools = ManagementFactory.getMemoryPoolMXBeans();
+    return mPools;
+  }
+
+  public static float calcPercentage(long a, long b) {
+    if (a == 0 || b == 0) {
+      return 0;
+    }
+    return ((float)a / (float)b) *100;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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
index 08565e0..2014b5b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/GenericTestUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/GenericTestUtils.java
@@ -35,6 +35,7 @@ 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;
@@ -69,14 +70,14 @@ public abstract class GenericTestUtils {
   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'.
@@ -84,7 +85,7 @@ public abstract class GenericTestUtils {
    */
   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)) {
@@ -97,6 +98,13 @@ public abstract class GenericTestUtils {
         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)
@@ -108,26 +116,26 @@ public abstract class GenericTestUtils {
       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;
@@ -135,36 +143,36 @@ public abstract class GenericTestUtils {
       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;
     }
@@ -175,7 +183,7 @@ public abstract class GenericTestUtils {
     public void waitForCall() throws InterruptedException {
       fireLatch.await();
     }
-
+  
     /**
      * Tell the method to proceed.
      * This should only be called after waitForCall()
@@ -183,7 +191,7 @@ public abstract class GenericTestUtils {
     public void proceed() {
       waitLatch.countDown();
     }
-
+  
     @Override
     public Object answer(InvocationOnMock invocation) throws Throwable {
       LOG.info("DelayAnswer firing fireLatch");
@@ -212,7 +220,7 @@ public abstract class GenericTestUtils {
         resultLatch.countDown();
       }
     }
-
+    
     /**
      * After calling proceed(), this will wait until the call has
      * completed and a result has been returned to the caller.
@@ -220,7 +228,7 @@ public abstract class GenericTestUtils {
     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.
@@ -228,7 +236,7 @@ public abstract class GenericTestUtils {
     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.
@@ -236,20 +244,20 @@ public abstract class GenericTestUtils {
     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>
@@ -260,14 +268,14 @@ public abstract class GenericTestUtils {
    *    ...
    * </code>
    */
-  public static class DelegateAnswer implements Answer<Object> {
+  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;
@@ -297,11 +305,11 @@ public abstract class GenericTestUtils {
   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;
@@ -325,11 +333,11 @@ public abstract class GenericTestUtils {
         " 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 + ","
@@ -344,7 +352,7 @@ public abstract class GenericTestUtils {
   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;

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..7d610e4
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java
@@ -0,0 +1,272 @@
+/**
+ * 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.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.http.HttpServer.Builder;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.ServerSocket;
+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 = "src/main/resources/hbase-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);
+  }
+
+  public static HttpServer createTestServerWithSecurity(Configuration conf) throws IOException {
+    prepareTestWebapp();
+    return localServerBuilder(TEST).setFindPort(true).setConf(conf).setSecurityEnabled(true)
+        // InfoServer normally sets these for us
+        .setUsernameConfKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_KEY)
+        .setKeytabConfKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_KEYTAB_KEY)
+        .build();
+  }
+
+  /**
+   * 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();
+  }
+
+  /**
+   * Recursively deletes a {@link File}.
+   */
+  protected static void deleteRecursively(File d) {
+    if (d.isDirectory()) {
+      for (String name : d.list()) {
+        File child = new File(d, name);
+        if (child.isFile()) {
+          child.delete();
+        } else {
+          deleteRecursively(child);
+        }
+      }
+    }
+    d.delete();
+  }
+
+  /**
+   * Picks a free port on the host by binding a Socket to '0'.
+   */
+  protected static int getFreePort() throws IOException {
+    ServerSocket s = new ServerSocket(0);
+    try {
+      s.setReuseAddress(true);
+      int port = s.getLocalPort();
+      return port;
+    } finally {
+      if (null != s) {
+        s.close();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/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..acfe929
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestGlobalFilter.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;
+
+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.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.net.NetUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({MiscTests.class, SmallTests.class})
+public class TestGlobalFilter extends HttpServerFunctionalTest {
+  private static final Log LOG = LogFactory.getLog(HttpServer.class);
+  static final Set<String> RECORDS = new TreeSet<>();
+
+  /** 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/d8a7b7b3/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..82fbe04
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestHtmlQuoting.java
@@ -0,0 +1,94 @@
+/**
+ * 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.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+@Category({MiscTests.class, 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/d8a7b7b3/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..b8d21d1
--- /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.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.log4j.Logger;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.eclipse.jetty.server.RequestLog;
+import org.eclipse.jetty.server.NCSARequestLog;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+@Category({MiscTests.class, 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/d8a7b7b3/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..a17b9e9
--- /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.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+
+@Category({MiscTests.class, 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());
+  }
+}


[08/10] hbase git commit: Revert "HBASE-19053 Split out o.a.h.h.http from hbase-server into a separate module"

Posted by bu...@apache.org.
Revert "HBASE-19053 Split out o.a.h.h.http from hbase-server into a separate module"

This reverts commit 3969b853b272c9d898024b3e86308c964e6fe6d0.


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

Branch: refs/heads/HBASE-19124
Commit: d8a7b7b39ade7f075a027d8444d7a8a280c0053c
Parents: d18e77a
Author: Sean Busbey <bu...@apache.org>
Authored: Wed Nov 1 09:19:06 2017 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Wed Nov 1 10:32:27 2017 -0500

----------------------------------------------------------------------
 hbase-common/pom.xml                            |    4 -
 .../org/apache/hadoop/hbase/util/JSONBean.java  |  351 -----
 .../hadoop/hbase/util/JSONMetricUtil.java       |  214 ---
 hbase-endpoint/pom.xml                          |    6 -
 hbase-http/pom.xml                              |  515 -------
 .../hbase/http/AdminAuthorizedServlet.java      |   49 -
 .../http/ClickjackingPreventionFilter.java      |   55 -
 .../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    |   80 -
 .../hadoop/hbase/http/HttpRequestLog.java       |   93 --
 .../hbase/http/HttpRequestLogAppender.java      |   63 -
 .../apache/hadoop/hbase/http/HttpServer.java    | 1380 ------------------
 .../hadoop/hbase/http/HttpServerUtil.java       |   52 -
 .../apache/hadoop/hbase/http/InfoServer.java    |  112 --
 .../apache/hadoop/hbase/http/NoCacheFilter.java |   56 -
 .../hbase/http/ServerConfigurationKeys.java     |   47 -
 .../hadoop/hbase/http/conf/ConfServlet.java     |  107 --
 .../hadoop/hbase/http/jmx/JMXJsonServlet.java   |  240 ---
 .../hadoop/hbase/http/jmx/package-info.java     |   26 -
 .../hbase/http/lib/StaticUserWebFilter.java     |  155 --
 .../hadoop/hbase/http/lib/package-info.java     |   38 -
 .../apache/hadoop/hbase/http/log/LogLevel.java  |  175 ---
 .../apache/hadoop/hbase/http/package-info.java  |   27 -
 .../hbase/http/HttpServerFunctionalTest.java    |  272 ----
 .../hadoop/hbase/http/TestGlobalFilter.java     |  151 --
 .../hadoop/hbase/http/TestHtmlQuoting.java      |   94 --
 .../hadoop/hbase/http/TestHttpRequestLog.java   |   52 -
 .../hbase/http/TestHttpRequestLogAppender.java  |   41 -
 .../hadoop/hbase/http/TestHttpServer.java       |  621 --------
 .../hbase/http/TestHttpServerLifecycle.java     |  135 --
 .../hbase/http/TestHttpServerWebapps.java       |   68 -
 .../hadoop/hbase/http/TestPathFilter.java       |  155 --
 .../hadoop/hbase/http/TestSSLHttpServer.java    |  124 --
 .../hadoop/hbase/http/TestServletFilter.java    |  217 ---
 .../hadoop/hbase/http/TestSpnegoHttpServer.java |  258 ----
 .../hadoop/hbase/http/conf/TestConfServlet.java |  116 --
 .../hbase/http/jmx/TestJMXJsonServlet.java      |  134 --
 .../hbase/http/lib/TestStaticUserWebFilter.java |   86 --
 .../hadoop/hbase/http/log/TestLogLevel.java     |   92 --
 .../hbase/http/resource/JerseyResource.java     |   64 -
 .../hadoop/hbase/http/ssl/KeyStoreTestUtil.java |  342 -----
 hbase-http/src/test/resources/log4j.properties  |   68 -
 .../src/test/resources/webapps/static/test.css  |   21 -
 .../src/test/resources/webapps/test/testjsp.jsp |   21 -
 .../apache/hadoop/hbase/rest/RESTServer.java    |    4 +-
 .../hbase/rest/HBaseRESTTestingUtility.java     |    4 +-
 hbase-server/pom.xml                            |   45 +-
 .../hbase/http/AdminAuthorizedServlet.java      |   49 +
 .../http/ClickjackingPreventionFilter.java      |   55 +
 .../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    |   80 +
 .../hadoop/hbase/http/HttpRequestLog.java       |   93 ++
 .../hbase/http/HttpRequestLogAppender.java      |   63 +
 .../apache/hadoop/hbase/http/HttpServer.java    | 1380 ++++++++++++++++++
 .../apache/hadoop/hbase/http/InfoServer.java    |  112 ++
 .../apache/hadoop/hbase/http/NoCacheFilter.java |   56 +
 .../hbase/http/ServerConfigurationKeys.java     |   47 +
 .../hadoop/hbase/http/conf/ConfServlet.java     |  107 ++
 .../hadoop/hbase/http/jmx/JMXJsonServlet.java   |  240 +++
 .../hadoop/hbase/http/jmx/package-info.java     |   26 +
 .../hbase/http/lib/StaticUserWebFilter.java     |  155 ++
 .../hadoop/hbase/http/lib/package-info.java     |   38 +
 .../apache/hadoop/hbase/http/log/LogLevel.java  |  175 +++
 .../apache/hadoop/hbase/http/package-info.java  |   27 +
 .../regionserver/DumpRegionServerMetrics.java   |   60 -
 .../hbase/regionserver/HRegionServer.java       |    3 +-
 .../hadoop/hbase/util/HttpServerUtil.java       |   52 +
 .../org/apache/hadoop/hbase/util/JSONBean.java  |  387 +++++
 .../hadoop/hbase/util/JSONMetricUtil.java       |  214 +++
 .../apache/hadoop/hbase/GenericTestUtils.java   |   74 +-
 .../hbase/http/HttpServerFunctionalTest.java    |  272 ++++
 .../hadoop/hbase/http/TestGlobalFilter.java     |  151 ++
 .../hadoop/hbase/http/TestHtmlQuoting.java      |   94 ++
 .../hadoop/hbase/http/TestHttpRequestLog.java   |   52 +
 .../hbase/http/TestHttpRequestLogAppender.java  |   41 +
 .../hadoop/hbase/http/TestHttpServer.java       |  621 ++++++++
 .../hbase/http/TestHttpServerLifecycle.java     |  135 ++
 .../hbase/http/TestHttpServerWebapps.java       |   68 +
 .../hadoop/hbase/http/TestPathFilter.java       |  155 ++
 .../hadoop/hbase/http/TestSSLHttpServer.java    |  124 ++
 .../hadoop/hbase/http/TestServletFilter.java    |  210 +++
 .../hadoop/hbase/http/TestSpnegoHttpServer.java |  258 ++++
 .../hadoop/hbase/http/conf/TestConfServlet.java |  116 ++
 .../hbase/http/jmx/TestJMXJsonServlet.java      |  134 ++
 .../hbase/http/lib/TestStaticUserWebFilter.java |   86 ++
 .../hadoop/hbase/http/log/TestLogLevel.java     |   92 ++
 .../hbase/http/resource/JerseyResource.java     |   64 +
 .../hadoop/hbase/http/ssl/KeyStoreTestUtil.java |  342 +++++
 hbase-shaded/hbase-shaded-mapreduce/pom.xml     |    4 -
 pom.xml                                         |   23 -
 94 files changed, 6742 insertions(+), 7399 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-common/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml
index 135e720..9bf4032 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -264,10 +264,6 @@
       <artifactId>findbugs-annotations</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
deleted file mode 100644
index 0571a08..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JSONBean.java
+++ /dev/null
@@ -1,351 +0,0 @@
-/*
- * 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.util;
-
-import java.io.Closeable;
-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 com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.core.JsonGenerator;
-import com.fasterxml.jackson.core.JsonGenerationException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * Utility for doing JSON and MBeans.
- */
-public class JSONBean {
-  private static final Log LOG = LogFactory.getLog(JSONBean.class);
-  private final JsonFactory jsonFactory;
-
-  public JSONBean() {
-    this.jsonFactory = new JsonFactory();
-  }
-
-  /**
-   * Use dumping out mbeans as JSON.
-   */
-  public interface Writer extends Closeable {
-    void write(final String key, final String value) throws JsonGenerationException, IOException;
-    int write(final MBeanServer mBeanServer, ObjectName qry, String attribute,
-        final boolean description) throws IOException;
-    void flush() throws IOException;
-  }
-
-  public Writer open(final PrintWriter writer) throws IOException {
-    final JsonGenerator jg = jsonFactory.createJsonGenerator(writer);
-    jg.disable(JsonGenerator.Feature.AUTO_CLOSE_TARGET);
-    jg.useDefaultPrettyPrinter();
-    jg.writeStartObject();
-    return new Writer() {
-      @Override
-      public void flush() throws IOException {
-        jg.flush();
-      }
-
-      @Override
-      public void close() throws IOException {
-        jg.close();
-      }
-
-      @Override
-      public void write(String key, String value) throws JsonGenerationException, IOException {
-        jg.writeStringField(key, value);
-      }
-
-      @Override
-      public int write(MBeanServer mBeanServer, ObjectName qry, String attribute,
-          boolean description)
-      throws IOException {
-        return JSONBean.write(jg, mBeanServer, qry, attribute, description);
-      }
-    };
-  }
-
-  /**
-   * @return Return non-zero if failed to find bean. 0
-   */
-  private static int write(final JsonGenerator jg,
-      final MBeanServer mBeanServer, ObjectName qry, String attribute,
-      final boolean description)
-  throws IOException {
-    LOG.trace("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 = "";
-      String descriptionStr = null;
-      Object attributeinfo = null;
-      try {
-        minfo = mBeanServer.getMBeanInfo(oname);
-        code = minfo.getClassName();
-        if (description) descriptionStr = minfo.getDescription();
-        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 (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) {
-           if (LOG.isTraceEnabled()) {
-             LOG.trace("Getting attribute " + prs + " of " + oname + " threw " + e);
-           }
-         } else {
-           LOG.error("Getting attribute " + prs + " of " + oname + " threw an exception", e);
-         }
-         return 0;
-        } 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());
-      if (description && descriptionStr != null && descriptionStr.length() > 0) {
-        jg.writeStringField("description", descriptionStr);
-      }
-      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();
-        return -1;
-      }
-
-      if (attribute != null) {
-        writeAttribute(jg, attribute, descriptionStr, attributeinfo);
-      } else {
-        MBeanAttributeInfo[] attrs = minfo.getAttributes();
-        for (int i = 0; i < attrs.length; i++) {
-          writeAttribute(jg, mBeanServer, oname, description, attrs[i]);
-        }
-      }
-      jg.writeEndObject();
-    }
-    jg.writeEndArray();
-    return 0;
-  }
-
-  private static void writeAttribute(final JsonGenerator jg,
-      final MBeanServer mBeanServer, ObjectName oname,
-      final boolean description, final 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;
-    }
-    String descriptionStr = description? attr.getDescription(): null;
-    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) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("Getting attribute " + attName + " of " + oname + " threw " + 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, descriptionStr, value);
-  }
-
-  private static void writeAttribute(JsonGenerator jg, String attName, final String descriptionStr,
-      Object value)
-  throws IOException {
-    boolean description = false;
-    if (descriptionStr != null && descriptionStr.length() > 0 && !attName.equals(descriptionStr)) {
-      description = true;
-      jg.writeFieldName(attName);
-      jg.writeStartObject();
-      jg.writeFieldName("description");
-      jg.writeString(descriptionStr);
-      jg.writeFieldName("value");
-      writeObject(jg, description, value);
-      jg.writeEndObject();
-    } else {
-      jg.writeFieldName(attName);
-      writeObject(jg, description, value);
-    }
-  }
-
-  private static void writeObject(final JsonGenerator jg, final boolean description, 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, description, 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, null, cds.get(key));
-        }
-        jg.writeEndObject();
-      } else if(value instanceof TabularData) {
-        TabularData tds = (TabularData)value;
-        jg.writeStartArray();
-        for(Object entry : tds.values()) {
-          writeObject(jg, description, entry);
-        }
-        jg.writeEndArray();
-      } else {
-        jg.writeString(value.toString());
-      }
-    }
-  }
-
-  /**
-   * Dump out all registered mbeans as json on System.out.
-   * @throws IOException
-   * @throws MalformedObjectNameException
-   */
-  public static void dumpAllBeans() throws IOException, MalformedObjectNameException {
-    try (PrintWriter writer = new PrintWriter(System.out)) {
-      JSONBean dumper = new JSONBean();
-      try (JSONBean.Writer jsonBeanWriter = dumper.open(writer)) {
-        MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer();
-        jsonBeanWriter.write(mbeanServer, new ObjectName("*:*"), null, false);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JSONMetricUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JSONMetricUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JSONMetricUtil.java
deleted file mode 100644
index d10610e..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/JSONMetricUtil.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/**
- * 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.util;
-
-import java.beans.IntrospectionException;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.lang.management.GarbageCollectorMXBean;
-import java.lang.management.ManagementFactory;
-import java.lang.management.MemoryPoolMXBean;
-import java.lang.management.RuntimeMXBean;
-import java.util.Hashtable;
-import java.util.List;
-import java.util.Set;
-
-import javax.management.InstanceNotFoundException;
-import javax.management.MBeanAttributeInfo;
-import javax.management.MBeanInfo;
-import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
-import javax.management.ReflectionException;
-import javax.management.openmbean.CompositeData;
-
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.codehaus.jettison.json.JSONException;
-
-public final class JSONMetricUtil {
-
-  private static final Log LOG = LogFactory.getLog(JSONMetricUtil.class);
-
-  private static MBeanServer mbServer = ManagementFactory.getPlatformMBeanServer();
-  //MBeans ObjectName domain names
-  public static final String JAVA_LANG_DOMAIN = "java.lang";
-  public static final String JAVA_NIO_DOMAIN = "java.nio";
-  public static final String SUN_MGMT_DOMAIN = "com.sun.management";
-  public static final String HADOOP_DOMAIN = "Hadoop";
-
-  //MBeans ObjectName properties key names
-  public static final String TYPE_KEY = "type";
-  public static final String NAME_KEY = "name";
-  public static final String SERVICE_KEY = "service";
-  public static final String SUBSYSTEM_KEY = "sub";
-
-/**
- * Utility for getting metric values. Collection of static methods intended for
- * easier access to metric values.
- */
-  private JSONMetricUtil() {
-    // Not to be called
-  }
-
-  public static MBeanAttributeInfo[] getMBeanAttributeInfo(ObjectName bean)
-      throws IntrospectionException, InstanceNotFoundException, ReflectionException,
-      IntrospectionException, javax.management.IntrospectionException {
-    MBeanInfo mbinfo = mbServer.getMBeanInfo(bean);
-    return mbinfo.getAttributes();
-  }
-
-  public static Object getValueFromMBean(ObjectName bean, String attribute) {
-    Object value = null;
-    try {
-      value = mbServer.getAttribute(bean, attribute);
-    }
-    catch(Exception e) {
-      LOG.error("Unable to get value from MBean= "+ bean.toString() +
-        "for attribute=" + attribute + " " + e.getMessage());
-    }
-    return value;
-  }
-
-  /**
-   * Returns a subset of mbeans defined by qry.
-   * Modeled after DumpRegionServerMetrics#dumpMetrics.
-   * Example: String qry= "java.lang:type=Memory"
-   * @throws MalformedObjectNameException if json have bad format
-   * @throws IOException /
-   * @return String representation of json array.
-   */
-  public static String dumpBeanToString(String qry) throws MalformedObjectNameException,
-  IOException {
-    StringWriter sw = new StringWriter(1024 * 100); // Guess this size
-    try (PrintWriter writer = new PrintWriter(sw)) {
-      JSONBean dumper = new JSONBean();
-      try (JSONBean.Writer jsonBeanWriter = dumper.open(writer)) {
-        MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer();
-        jsonBeanWriter.write(mbeanServer,
-          new ObjectName(qry), null, false);
-      }
-    }
-    sw.close();
-    return sw.toString();
-  }
-
-  public static JsonNode mappStringToJsonNode(String jsonString)
-      throws JsonProcessingException, IOException {
-    ObjectMapper mapper = new ObjectMapper();
-    JsonNode node = mapper.readTree(jsonString);
-    return node;
-  }
-
-
-  public static JsonNode searchJson(JsonNode tree, String searchKey)
-      throws JsonProcessingException, IOException {
-    if (tree == null) {
-      return null;
-    }
-    if(tree.has(searchKey)) {
-      return tree.get(searchKey);
-    }
-    if(tree.isContainerNode()) {
-      for(JsonNode branch: tree) {
-        JsonNode branchResult = searchJson(branch, searchKey);
-        if (branchResult != null && !branchResult.isMissingNode()) {
-          return branchResult;
-        }
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Method for building hashtable used for constructing ObjectName.
-   * Mapping is done with arrays indices
-   * @param keys Hashtable keys
-   * @param values Hashtable values
-   * @return Hashtable or null if arrays are empty * or have different number of elements
-   */
-  public static Hashtable<String, String> buldKeyValueTable(String[] keys, String[] values) {
-    if (keys.length != values.length) {
-      LOG.error("keys and values arrays must be same size");
-      return null;
-    }
-    if (keys.length == 0 || values.length == 0) {
-      LOG.error("keys and values arrays can not be empty;");
-      return null;
-    }
-    Hashtable<String, String> table = new Hashtable<String, String>();
-    for(int i = 0; i < keys.length; i++) {
-      table.put(keys[i], values[i]);
-    }
-    return table;
-  }
-
-  public static ObjectName buildObjectName(String pattern) throws MalformedObjectNameException {
-    return new ObjectName(pattern);
-  }
-
-  public static ObjectName buildObjectName(String domain, Hashtable<String, String> keyValueTable)
-      throws MalformedObjectNameException {
-    return new ObjectName(domain, keyValueTable);
-  }
-
-  public static Set<ObjectName> getRegistredMBeans(ObjectName name, MBeanServer mbs) {
-    return mbs.queryNames(name, null);
-  }
-
-  public static String getProcessPID() {
-    return ManagementFactory.getRuntimeMXBean().getName().split("@")[0];
-  }
-
-  public static String getCommmand() throws MalformedObjectNameException,
-  IOException, JSONException {
-    RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean();
-    return runtimeBean.getSystemProperties().get("sun.java.command");
-  }
-
-  public static List<GarbageCollectorMXBean> getGcCollectorBeans() {
-    List<GarbageCollectorMXBean> gcBeans = ManagementFactory.getGarbageCollectorMXBeans();
-    return gcBeans;
-  }
-
-  public static long getLastGcDuration(ObjectName gcCollector) {
-    long lastGcDuration = 0;
-    Object lastGcInfo = getValueFromMBean(gcCollector, "LastGcInfo");
-    if (lastGcInfo != null && lastGcInfo instanceof CompositeData) {
-      CompositeData cds = (CompositeData)lastGcInfo;
-      lastGcDuration = (long) cds.get("duration");
-    }
-    return lastGcDuration;
-  }
-
-  public static List<MemoryPoolMXBean> getMemoryPools() {
-    List<MemoryPoolMXBean> mPools = ManagementFactory.getMemoryPoolMXBeans();
-    return mPools;
-  }
-
-  public static float calcPercentage(long a, long b) {
-    if (a == 0 || b == 0) {
-      return 0;
-    }
-    return ((float)a / (float)b) *100;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-endpoint/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-endpoint/pom.xml b/hbase-endpoint/pom.xml
index a127763..b46eb64 100644
--- a/hbase-endpoint/pom.xml
+++ b/hbase-endpoint/pom.xml
@@ -174,12 +174,6 @@
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-http</artifactId>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
     <!-- The coprocessor.Export needs mapreduce.Import and mapreduce.Export to run the unit tests -->
     <!-- see org.apache.hadoop.hbase.coprocessor.TestImportExport -->
     <dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-http/pom.xml b/hbase-http/pom.xml
deleted file mode 100644
index d876f19..0000000
--- a/hbase-http/pom.xml
+++ /dev/null
@@ -1,515 +0,0 @@
-<?xml version="1.0"?>
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <!--
-/**
- * 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.
- */
--->
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>hbase-build-configuration</artifactId>
-    <groupId>org.apache.hbase</groupId>
-    <version>3.0.0-SNAPSHOT</version>
-    <relativePath>../hbase-build-configuration</relativePath>
-  </parent>
-  <artifactId>hbase-http</artifactId>
-  <name>Apache HBase - HTTP</name>
-  <description>HTTP functionality for HBase Servers</description>
-
-  <build>
-    <!-- Makes sure the resources get added before they are processed
-      by placing this first -->
-    <testResources>
-      <!-- Our test artifact has different license info than our source/bin ones -->
-      <testResource>
-        <directory>src/test/resources/META-INF/</directory>
-        <targetPath>META-INF/</targetPath>
-        <includes>
-          <include>NOTICE</include>
-        </includes>
-        <filtering>true</filtering>
-      </testResource>
-      <testResource>
-        <directory>src/test/resources</directory>
-        <includes>
-          <include>**/**</include>
-        </includes>
-      </testResource>
-    </testResources>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-site-plugin</artifactId>
-        <configuration>
-          <skip>true</skip>
-        </configuration>
-      </plugin>
-      <!-- licensing info from our bundled works -->
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-remote-resources-plugin</artifactId>
-        <version>1.5</version>
-        <executions>
-          <execution>
-            <id>default</id>
-            <configuration>
-              <attachToTest>false</attachToTest>
-              <properties>
-                <copyright-end-year>${build.year}</copyright-end-year>
-                <debug-print-included-work-info>${license.debug.print.included}</debug-print-included-work-info>
-                <bundled-dependencies>${license.bundles.dependencies}</bundled-dependencies>
-                <bundled-jquery>${license.bundles.jquery}</bundled-jquery>
-                <bundled-logo>${license.bundles.logo}</bundled-logo>
-                <bundled-bootstrap>${license.bundles.bootstrap}</bundled-bootstrap>
-              </properties>
-              <resourceBundles>
-                <resourceBundle>${project.groupId}:hbase-resource-bundle:${project.version}</resourceBundle>
-              </resourceBundles>
-              <supplementalModelArtifacts>
-                <supplementalModelArtifact>${project.groupId}:hbase-resource-bundle:${project.version}</supplementalModelArtifact>
-              </supplementalModelArtifacts>
-              <supplementalModels>
-                <supplementalModel>supplemental-models.xml</supplementalModel>
-              </supplementalModels>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-      <!-- Run with -Dmaven.test.skip.exec=true to build -tests.jar without running
-        tests (this is needed for upstream projects whose tests need this jar simply for
-        compilation) -->
-      <plugin>
-        <!--Make it so assembly:single does nothing in here-->
-        <artifactId>maven-assembly-plugin</artifactId>
-        <configuration>
-          <skipAssembly>true</skipAssembly>
-        </configuration>
-      </plugin>
-      <!-- Make a jar and put the sources in the jar -->
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-source-plugin</artifactId>
-        <executions>
-          <execution>
-            <phase>package</phase>
-            <goals>
-              <goal>jar</goal>
-              <goal>test-jar</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <!-- Run findbugs -->
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>findbugs-maven-plugin</artifactId>
-      </plugin>
-      <!-- Testing plugins -->
-      <plugin>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <configuration>
-          <properties>
-            <property>
-              <name>listener</name>
-              <value>org.apache.hadoop.hbase.ResourceCheckerJUnitListener</value>
-            </property>
-          </properties>
-          <systemPropertyVariables>
-            <test.build.webapps>target/test-classes/webapps</test.build.webapps>
-          </systemPropertyVariables>
-        </configuration>
-      </plugin>
-    </plugins>
-    <!-- General Resources -->
-    <pluginManagement>
-       <plugins>
-         <!--This plugin's configuration is used to store Eclipse m2e settings
-             only. It has no influence on the Maven build itself and needs to
-             be kept in plugin management, not in the actual plugins. -->
-        <plugin>
-          <groupId>org.eclipse.m2e</groupId>
-          <artifactId>lifecycle-mapping</artifactId>
-          <configuration>
-            <lifecycleMappingMetadata>
-              <pluginExecutions>
-                <pluginExecution>
-                  <pluginExecutionFilter>
-                    <groupId>org.apache.maven.plugins</groupId>
-                    <artifactId>maven-antrun-plugin</artifactId>
-                    <versionRange>[1.6,)</versionRange>
-                    <goals>
-                      <goal>run</goal>
-                    </goals>
-                  </pluginExecutionFilter>
-                  <action>
-                    <execute>
-                      <runOnIncremental>false</runOnIncremental>
-                      <runOnConfiguration>true</runOnConfiguration>
-                    </execute>
-                  </action>
-                </pluginExecution>
-                <pluginExecution>
-                  <pluginExecutionFilter>
-                    <groupId>org.apache.maven.plugins</groupId>
-                    <artifactId>maven-dependency-plugin</artifactId>
-                    <versionRange>[2.8,)</versionRange>
-                    <goals>
-                      <goal>build-classpath</goal>
-                    </goals>
-                  </pluginExecutionFilter>
-                  <action>
-                    <ignore></ignore>
-                  </action>
-                </pluginExecution>
-                <pluginExecution>
-                  <pluginExecutionFilter>
-                    <groupId>org.apache.maven.plugins</groupId>
-                    <artifactId>maven-compiler-plugin</artifactId>
-                    <versionRange>[3.2,)</versionRange>
-                    <goals>
-                      <goal>compile</goal>
-                    </goals>
-                  </pluginExecutionFilter>
-                  <action>
-                    <ignore></ignore>
-                  </action>
-                </pluginExecution>
-              </pluginExecutions>
-            </lifecycleMappingMetadata>
-          </configuration>
-        </plugin>
-      </plugins>
-    </pluginManagement>
-  </build>
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.hbase.thirdparty</groupId>
-      <artifactId>hbase-shaded-miscellaneous</artifactId>
-    </dependency>
-    <!-- Intra-project dependencies -->
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-common</artifactId>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-annotations</artifactId>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <!-- resource bundle only needed at build time -->
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-resource-bundle</artifactId>
-      <version>${project.version}</version>
-      <optional>true</optional>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-server</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-servlet</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-util</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-util-ajax</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-webapp</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-http</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-security</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.glassfish.jersey.core</groupId>
-      <artifactId>jersey-server</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.glassfish.jersey.containers</groupId>
-      <artifactId>jersey-container-servlet-core</artifactId>
-    </dependency>
-    <!-- General dependencies -->
-    <dependency>
-      <groupId>commons-logging</groupId>
-      <artifactId>commons-logging</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>log4j</groupId>
-      <artifactId>log4j</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>javax.servlet</groupId>
-      <artifactId>javax.servlet-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>javax.ws.rs</groupId>
-      <artifactId>javax.ws.rs-api</artifactId>
-    </dependency>
-
-    <!-- Test dependencies -->
-    <dependency>
-      <groupId>org.apache.kerby</groupId>
-      <artifactId>kerb-simplekdc</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.bouncycastle</groupId>
-      <artifactId>bcprov-jdk16</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.kerby</groupId>
-      <artifactId>kerb-core</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.httpcomponents</groupId>
-      <artifactId>httpclient</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.httpcomponents</groupId>
-      <artifactId>httpcore</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-all</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-  <profiles>
-    <!-- Needs to make the profile in apache parent pom -->
-    <profile>
-      <id>apache-release</id>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-resources-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>license-javadocs</id>
-                <phase>prepare-package</phase>
-                <goals>
-                  <goal>copy-resources</goal>
-                </goals>
-                <configuration>
-                  <outputDirectory>${project.build.directory}/apidocs</outputDirectory>
-                  <resources>
-                    <resource>
-                      <directory>src/main/javadoc/META-INF/</directory>
-                      <targetPath>META-INF/</targetPath>
-                      <includes>
-                        <include>LICENSE</include>
-                        <include>NOTICE</include>
-                      </includes>
-                      <filtering>true</filtering>
-                    </resource>
-                  </resources>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-    <!-- Skip the tests in this module -->
-    <profile>
-      <id>skipServerTests</id>
-      <activation>
-        <property>
-          <name>skipServerTests</name>
-        </property>
-      </activation>
-      <properties>
-        <surefire.skipFirstPart>true</surefire.skipFirstPart>
-        <surefire.skipSecondPart>true</surefire.skipSecondPart>
-      </properties>
-    </profile>
-    <!-- Special builds -->
-    <profile>
-      <id>native</id>
-      <activation>
-        <activeByDefault>false</activeByDefault>
-      </activation>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-antrun-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>make</id>
-                <phase>compile</phase>
-                <goals><goal>run</goal></goals>
-                <configuration>
-                  <target>
-                    <mkdir dir="${project.build.directory}/native"/>
-                    <exec executable="cmake" dir="${project.build.directory}/native" failonerror="true">
-                      <arg line="${basedir}/src/main/native -DJVM_ARCH_DATA_MODEL=${sun.arch.data.model}"/>
-                    </exec>
-                    <exec executable="make" dir="${project.build.directory}/native" failonerror="true">
-                      <arg line="VERBOSE=1"/>
-                    </exec>
-                  </target>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-    <!-- Profiles for building against different hadoop versions -->
-    <!-- There are a lot of common dependencies used here, should investigate
-    if we can combine these profiles somehow -->
-
-    <!-- profile for building against Hadoop 2.x.  This is the default.  -->
-    <profile>
-      <id>hadoop-2.0</id>
-      <activation>
-        <property>
-            <!--Below formatting for dev-support/generate-hadoopX-poms.sh-->
-            <!--h2--><name>!hadoop.profile</name>
-        </property>
-      </activation>
-      <dependencies>
-        <dependency>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-common</artifactId>
-        </dependency>
-        <dependency>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-minicluster</artifactId>
-          <exclusions>
-            <exclusion>
-              <groupId>com.google.guava</groupId>
-              <artifactId>guava</artifactId>
-            </exclusion>
-          </exclusions>
-          <scope>test</scope>
-        </dependency>
-        <dependency>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-auth</artifactId>
-        </dependency>
-      </dependencies>
-      <build>
-        <plugins>
-          <plugin>
-            <artifactId>maven-dependency-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>create-mrapp-generated-classpath</id>
-                <phase>generate-test-resources</phase>
-                <goals>
-                  <goal>build-classpath</goal>
-                </goals>
-                <configuration>
-                  <!-- needed to run the unit test for DS to generate
-                  the required classpath that is required in the env
-                  of the launch container in the mini mr/yarn cluster
-                  -->
-                  <outputFile>${project.build.directory}/test-classes/mrapp-generated-classpath</outputFile>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-    <!--
-      profile for building against Hadoop 3.0.x. Activate using:
-       mvn -Dhadoop.profile=3.0
-    -->
-    <profile>
-      <id>hadoop-3.0</id>
-      <activation>
-        <property>
-          <name>hadoop.profile</name>
-          <value>3.0</value>
-        </property>
-      </activation>
-      <properties>
-        <hadoop.version>${hadoop-three.version}</hadoop.version>
-      </properties>
-      <dependencies>
-        <dependency>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-common</artifactId>
-        </dependency>
-        <dependency>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-minicluster</artifactId>
-          <exclusions>
-            <exclusion>
-              <groupId>com.google.guava</groupId>
-              <artifactId>guava</artifactId>
-            </exclusion>
-          </exclusions>
-          <scope>test</scope>
-        </dependency>
-      </dependencies>
-      <build>
-        <plugins>
-          <plugin>
-            <artifactId>maven-dependency-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>create-mrapp-generated-classpath</id>
-                <phase>generate-test-resources</phase>
-                <goals>
-                  <goal>build-classpath</goal>
-                </goals>
-                <configuration>
-                  <!-- needed to run the unit test for DS to generate
-                  the required classpath that is required in the env
-                  of the launch container in the mini mr/yarn cluster
-                  -->
-                  <outputFile>${project.build.directory}/test-classes/mrapp-generated-classpath</outputFile>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
-</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/AdminAuthorizedServlet.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/AdminAuthorizedServlet.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/AdminAuthorizedServlet.java
deleted file mode 100644
index bd8570e..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/AdminAuthorizedServlet.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/**
- * 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.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.eclipse.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/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ClickjackingPreventionFilter.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ClickjackingPreventionFilter.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ClickjackingPreventionFilter.java
deleted file mode 100644
index 9944d29..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/ClickjackingPreventionFilter.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * 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;
-
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
-public class ClickjackingPreventionFilter implements Filter {
-
-    private FilterConfig filterConfig;
-
-    @Override
-    public void init(FilterConfig filterConfig) throws ServletException {
-        this.filterConfig = filterConfig;
-    }
-
-    @Override
-    public void doFilter(ServletRequest req, ServletResponse res,
-                         FilterChain chain)
-            throws IOException, ServletException {
-        HttpServletResponse httpRes = (HttpServletResponse) res;
-        httpRes.addHeader("X-Frame-Options", filterConfig.getInitParameter("xframeoptions"));
-        chain.doFilter(req, res);
-    }
-
-    @Override
-    public void destroy() {
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/FilterContainer.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/FilterContainer.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/FilterContainer.java
deleted file mode 100644
index 7a79acc..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/FilterContainer.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * 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/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/FilterInitializer.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/FilterInitializer.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/FilterInitializer.java
deleted file mode 100644
index d317343..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/FilterInitializer.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * 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);
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HtmlQuoting.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HtmlQuoting.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HtmlQuoting.java
deleted file mode 100644
index 7f4bb83..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HtmlQuoting.java
+++ /dev/null
@@ -1,215 +0,0 @@
-/**
- * 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/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpConfig.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpConfig.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpConfig.java
deleted file mode 100644
index b9dde23..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpConfig.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/**
- * 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.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Statics to get access to Http related configuration.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class HttpConfig {
-  private Policy policy;
-  public enum Policy {
-    HTTP_ONLY,
-    HTTPS_ONLY,
-    HTTP_AND_HTTPS;
-
-    public 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;
-    }
-  }
-
-   public HttpConfig(final Configuration conf) {
-    boolean sslEnabled = conf.getBoolean(
-      ServerConfigurationKeys.HBASE_SSL_ENABLED_KEY,
-      ServerConfigurationKeys.HBASE_SSL_ENABLED_DEFAULT);
-    policy = sslEnabled ? Policy.HTTPS_ONLY : Policy.HTTP_ONLY;
-    if (sslEnabled) {
-      conf.addResource("ssl-server.xml");
-      conf.addResource("ssl-client.xml");
-    }
-  }
-
-  public void setPolicy(Policy policy) {
-    this.policy = policy;
-  }
-
-  public boolean isSecure() {
-    return policy == Policy.HTTPS_ONLY;
-  }
-
-  public String getSchemePrefix() {
-    return (isSecure()) ? "https://" : "http://";
-  }
-
-  public String getScheme(Policy policy) {
-    return policy == Policy.HTTPS_ONLY ? "https://" : "http://";
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLog.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLog.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLog.java
deleted file mode 100644
index cfc0640..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLog.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/**
- * 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.eclipse.jetty.server.RequestLog;
-import org.eclipse.jetty.server.NCSARequestLog;
-
-/**
- * RequestLog object for use with Http
- */
-public class HttpRequestLog {
-
-  private static final Log LOG = LogFactory.getLog(HttpRequestLog.class);
-  private static final HashMap<String, String> serverToComponent;
-
-  static {
-    serverToComponent = new HashMap<>();
-    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/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLogAppender.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLogAppender.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLogAppender.java
deleted file mode 100644
index 8039b34..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/HttpRequestLogAppender.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/**
- * 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;
-  }
-}


[06/10] hbase git commit: Revert "HBASE-19053 Split out o.a.h.h.http from hbase-server into a separate module"

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java
deleted file mode 100644
index e23eecd..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/log/LogLevel.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/**
- * 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.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.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();
-      try (InputStreamReader streamReader = new InputStreamReader(connection.getInputStream());
-           BufferedReader bufferedReader = new BufferedReader(streamReader)) {
-        for(String line; (line = bufferedReader.readLine()) != null; ) {
-          if (line.startsWith(MARKER)) {
-            System.out.println(TAG.matcher(line).replaceAll(""));
-          }
-        }
-      }
-    } 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/d8a7b7b3/hbase-http/src/main/java/org/apache/hadoop/hbase/http/package-info.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/package-info.java b/hbase-http/src/main/java/org/apache/hadoop/hbase/http/package-info.java
deleted file mode 100644
index f55e24b..0000000
--- a/hbase-http/src/main/java/org/apache/hadoop/hbase/http/package-info.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * 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>
- * 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.yetus.audience.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java
deleted file mode 100644
index 69972a2..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/HttpServerFunctionalTest.java
+++ /dev/null
@@ -1,272 +0,0 @@
-/**
- * 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.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.http.HttpServer.Builder;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.ServerSocket;
-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 = "src/main/resources/hbase-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);
-  }
-
-  public static HttpServer createTestServerWithSecurity(Configuration conf) throws IOException {
-    prepareTestWebapp();
-    return localServerBuilder(TEST).setFindPort(true).setConf(conf).setSecurityEnabled(true)
-        // InfoServer normally sets these for us
-        .setUsernameConfKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_PRINCIPAL_KEY)
-        .setKeytabConfKey(HttpServer.HTTP_SPNEGO_AUTHENTICATION_KEYTAB_KEY)
-        .build();
-  }
-
-  /**
-   * 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();
-  }
-
-  /**
-   * Recursively deletes a {@link File}.
-   */
-  protected static void deleteRecursively(File d) {
-    if (d.isDirectory()) {
-      for (String name : d.list()) {
-        File child = new File(d, name);
-        if (child.isFile()) {
-          child.delete();
-        } else {
-          deleteRecursively(child);
-        }
-      }
-    }
-    d.delete();
-  }
-
-  /**
-   * Picks a free port on the host by binding a Socket to '0'.
-   */
-  protected static int getFreePort() throws IOException {
-    ServerSocket s = new ServerSocket(0);
-    try {
-      s.setReuseAddress(true);
-      int port = s.getLocalPort();
-      return port;
-    } finally {
-      if (null != s) {
-        s.close();
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestGlobalFilter.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestGlobalFilter.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestGlobalFilter.java
deleted file mode 100644
index 729dd06..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestGlobalFilter.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/**
- * 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.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.net.NetUtils;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({MiscTests.class, SmallTests.class})
-public class TestGlobalFilter extends HttpServerFunctionalTest {
-  private static final Log LOG = LogFactory.getLog(HttpServer.class);
-  static final Set<String> RECORDS = new TreeSet<>();
-
-  /** 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/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHtmlQuoting.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHtmlQuoting.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHtmlQuoting.java
deleted file mode 100644
index 5bc026c..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHtmlQuoting.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/**
- * 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.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-
-@Category({MiscTests.class, 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/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpRequestLog.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpRequestLog.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpRequestLog.java
deleted file mode 100644
index b8d21d1..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpRequestLog.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * 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.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.log4j.Logger;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.eclipse.jetty.server.RequestLog;
-import org.eclipse.jetty.server.NCSARequestLog;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-
-@Category({MiscTests.class, 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/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpRequestLogAppender.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpRequestLogAppender.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpRequestLogAppender.java
deleted file mode 100644
index a17b9e9..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpRequestLogAppender.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * 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.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import static org.junit.Assert.assertEquals;
-
-@Category({MiscTests.class, 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());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpServer.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpServer.java
deleted file mode 100644
index 2eb6a21..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpServer.java
+++ /dev/null
@@ -1,621 +0,0 @@
-/**
- * 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.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 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.http.HttpServer.QuotingInputFilter.RequestQuoter;
-import org.apache.hadoop.hbase.http.resource.JerseyResource;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-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.eclipse.jetty.server.ServerConnector;
-import org.eclipse.jetty.util.ajax.JSON;
-import org.junit.AfterClass;
-import org.junit.Assert;
-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;
-
-@Category({MiscTests.class, SmallTests.class})
-public class TestHttpServer extends HttpServerFunctionalTest {
-  private static final Log LOG = LogFactory.getLog(TestHttpServer.class);
-  private static HttpServer server;
-  private static URL baseUrl;
-  // jetty 9.4.x needs this many threads to start, even in the small.
-  static final int MAX_THREADS = 16;
-
-  @SuppressWarnings("serial")
-  public static class EchoMapServlet extends HttpServlet {
-    @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<>(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 {
-    @Override
-    public void doGet(HttpServletRequest request,
-                      HttpServletResponse response
-                      ) throws ServletException, IOException {
-      PrintWriter out = response.getWriter();
-      SortedSet<String> sortedKeys = new TreeSet<>();
-      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, MAX_THREADS);
-    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
-  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
-    // JSPs do not work from unit tests
-    // 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<>();
-
-    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");
-      ServerConnector listener = (ServerConnector) 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 testXFrameHeaderSameOrigin() throws Exception {
-    Configuration conf = new Configuration();
-    conf.set("hbase.http.filter.xframeoptions.mode", "SAMEORIGIN");
-
-    HttpServer myServer = new HttpServer.Builder().setName("test")
-            .addEndpoint(new URI("http://localhost:0"))
-            .setFindPort(true).setConf(conf).build();
-    myServer.setAttribute(HttpServer.CONF_CONTEXT_ATTRIBUTE, conf);
-    myServer.addServlet("echo", "/echo", EchoServlet.class);
-    myServer.start();
-
-    String serverURL = "http://"
-            + NetUtils.getHostPortString(myServer.getConnectorAddress(0));
-    URL url = new URL(new URL(serverURL), "/echo?a=b&c=d");
-    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
-    assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
-    assertEquals("SAMEORIGIN", conn.getHeaderField("X-Frame-Options"));
-    myServer.stop();
-  }
-
-
-
-  @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"));
-    assertEquals("DENY", conn.getHeaderField("X-Frame-Options"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerLifecycle.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerLifecycle.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerLifecycle.java
deleted file mode 100644
index d0f2825..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerLifecycle.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/**
- * 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.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({MiscTests.class, 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
-   */
-  @Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
-  public void testCreatedServerIsNotAlive() throws Throwable {
-    HttpServer server = createTestServer();
-    assertNotLive(server);
-  }
-
-  @Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
-  public void testStopUnstartedServer() throws Throwable {
-    HttpServer server = createTestServer();
-    stop(server);
-  }
-
-  /**
-   * Test that the server is alive once started
-   *
-   * @throws Throwable on failure
-   */
-  @Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
-  public void testStartedServerIsAlive() throws Throwable {
-    HttpServer server = null;
-    server = createTestServer();
-    assertNotLive(server);
-    server.start();
-    assertAlive(server);
-    stop(server);
-  }
-
-  /**
-   * 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
-   */
-  @Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
-  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
-   */
-  @Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
-  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
-   */
-  @Ignore ("Hangs on occasion; see HBASE-14430") @Test(timeout=60000)
-  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/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerWebapps.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerWebapps.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerWebapps.java
deleted file mode 100644
index db394a8..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestHttpServerWebapps.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * 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.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.apache.commons.logging.LogFactory;
-import org.apache.commons.logging.Log;
-
-import java.io.FileNotFoundException;
-
-/**
- * Test webapp loading
- */
-@Category({MiscTests.class, 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/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestPathFilter.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestPathFilter.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestPathFilter.java
deleted file mode 100644
index 5eff2b4..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestPathFilter.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/**
- * 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.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.net.NetUtils;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({MiscTests.class, SmallTests.class})
-public class TestPathFilter extends HttpServerFunctionalTest {
-  private static final Log LOG = LogFactory.getLog(HttpServer.class);
-  static final Set<String> RECORDS = new TreeSet<>();
-
-  /** 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/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSSLHttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSSLHttpServer.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSSLHttpServer.java
deleted file mode 100644
index b599350..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSSLHttpServer.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/**
- * 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.HBaseConfiguration;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.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({MiscTests.class, 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, TestHttpServer.MAX_THREADS);
-
-    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(HBaseConfiguration.getPassword(sslConf, "ssl.server.keystore.keypassword",
-            null))
-        .keyStore(sslConf.get("ssl.server.keystore.location"),
-            HBaseConfiguration.getPassword(sslConf, "ssl.server.keystore.password", null),
-            sslConf.get("ssl.server.keystore.type", "jks"))
-        .trustStore(sslConf.get("ssl.server.truststore.location"),
-            HBaseConfiguration.getPassword(sslConf, "ssl.server.truststore.password", null),
-            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/d8a7b7b3/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java
----------------------------------------------------------------------
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java
deleted file mode 100644
index 32bc03e..0000000
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestServletFilter.java
+++ /dev/null
@@ -1,217 +0,0 @@
-/**
- * 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.testclassification.MiscTests;
-import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.util.StringUtils;
-import org.junit.Assert;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category({MiscTests.class, SmallTests.class})
-public class TestServletFilter extends HttpServerFunctionalTest {
-  private 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);
-      }
-    }
-  }
-
-  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));
-  }
-
-  /** 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
-  @Ignore
-  //From stack
-  // Its a 'foreign' test, one that came in from hadoop when we copy/pasted http
-  // It's second class. Could comment it out if only failing test (as per @nkeywal – sort of)
-  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) {
-      assertExceptionContains("Problem starting http server", e);
-    }
-  }
-
-  /**
-   * 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) {
-      assertExceptionContains("Unable to initialize WebAppContext", e);
-    }
-  }
-
-}


[09/10] hbase git commit: Revert "HBASE-19119 hbase-http shouldn't have native profile"

Posted by bu...@apache.org.
Revert "HBASE-19119 hbase-http shouldn't have native profile"

This reverts commit a79b66b32b4e3eb98fb1c03cb545b8edeab26647.


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

Branch: refs/heads/HBASE-19124
Commit: d18e77a99cfadbf4048db71465f37dc2408be673
Parents: ac6aff3
Author: Sean Busbey <bu...@apache.org>
Authored: Wed Nov 1 09:18:57 2017 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Wed Nov 1 10:32:27 2017 -0500

----------------------------------------------------------------------
 hbase-http/pom.xml | 32 ++++++++++++++++++++++++++++++++
 1 file changed, 32 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d18e77a9/hbase-http/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-http/pom.xml b/hbase-http/pom.xml
index c2ec302..d876f19 100644
--- a/hbase-http/pom.xml
+++ b/hbase-http/pom.xml
@@ -365,6 +365,38 @@
       </properties>
     </profile>
     <!-- Special builds -->
+    <profile>
+      <id>native</id>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>make</id>
+                <phase>compile</phase>
+                <goals><goal>run</goal></goals>
+                <configuration>
+                  <target>
+                    <mkdir dir="${project.build.directory}/native"/>
+                    <exec executable="cmake" dir="${project.build.directory}/native" failonerror="true">
+                      <arg line="${basedir}/src/main/native -DJVM_ARCH_DATA_MODEL=${sun.arch.data.model}"/>
+                    </exec>
+                    <exec executable="make" dir="${project.build.directory}/native" failonerror="true">
+                      <arg line="VERBOSE=1"/>
+                    </exec>
+                  </target>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
     <!-- Profiles for building against different hadoop versions -->
     <!-- There are a lot of common dependencies used here, should investigate
     if we can combine these profiles somehow -->