You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2019/02/27 03:54:05 UTC

[hbase] branch branch-2.2 updated: HBASE-21960 Ensure RESTServletContainer used by RESTServer

This is an automated email from the ASF dual-hosted git repository.

elserj pushed a commit to branch branch-2.2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.2 by this push:
     new 2b26a33  HBASE-21960 Ensure RESTServletContainer used by RESTServer
2b26a33 is described below

commit 2b26a3383132d5f1291621f2e55745ff007ff04f
Author: Josh Elser <el...@apache.org>
AuthorDate: Fri Feb 22 15:53:24 2019 -0500

    HBASE-21960 Ensure RESTServletContainer used by RESTServer
    
    Signed-off-by: stack <st...@apache.org>
---
 hbase-rest/pom.xml                                 |  26 ++
 .../org/apache/hadoop/hbase/rest/RESTServer.java   | 157 +++++---
 .../hadoop/hbase/rest/RESTServletContainer.java    |  13 +-
 .../apache/hadoop/hbase/rest/SchemaResource.java   |   6 +
 .../hadoop/hbase/rest/filter/AuthFilter.java       |  10 +-
 .../hadoop/hbase/rest/HBaseRESTTestingUtility.java |  71 +---
 .../hadoop/hbase/rest/TestMultiRowResource.java    |   3 +
 .../hadoop/hbase/rest/TestSchemaResource.java      |   3 +
 .../hadoop/hbase/rest/TestSecureRESTServer.java    | 428 +++++++++++++++++++++
 hbase-rest/src/test/resources/log4j.properties     |   1 +
 10 files changed, 600 insertions(+), 118 deletions(-)

diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index e846cc1..bf2d603 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -234,6 +234,12 @@
       <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>
     <!-- REST dependencies -->
     <dependency>
       <groupId>org.apache.hbase.thirdparty</groupId>
@@ -337,6 +343,26 @@
       <artifactId>javax.el</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.kerby</groupId>
+      <artifactId>kerb-simplekdc</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minikdc</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kerby</groupId>
+      <artifactId>kerb-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
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 63c9e42..8a09647 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
@@ -40,9 +40,9 @@ 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.Pair;
+import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.hadoop.hbase.util.Strings;
 import org.apache.hadoop.hbase.util.VersionInfo;
-import org.apache.hadoop.util.StringUtils;
 
 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
@@ -85,19 +85,33 @@ import javax.servlet.DispatcherType;
 public class RESTServer implements Constants {
   static Logger LOG = LoggerFactory.getLogger("RESTServer");
 
-  static String REST_CSRF_ENABLED_KEY = "hbase.rest.csrf.enabled";
-  static boolean REST_CSRF_ENABLED_DEFAULT = false;
-  static boolean restCSRFEnabled = false;
-  static String REST_CSRF_CUSTOM_HEADER_KEY ="hbase.rest.csrf.custom.header";
-  static String REST_CSRF_CUSTOM_HEADER_DEFAULT = "X-XSRF-HEADER";
-  static String REST_CSRF_METHODS_TO_IGNORE_KEY = "hbase.rest.csrf.methods.to.ignore";
-  static String REST_CSRF_METHODS_TO_IGNORE_DEFAULT = "GET,OPTIONS,HEAD,TRACE";
+  static final String REST_CSRF_ENABLED_KEY = "hbase.rest.csrf.enabled";
+  static final boolean REST_CSRF_ENABLED_DEFAULT = false;
+  boolean restCSRFEnabled = false;
+  static final String REST_CSRF_CUSTOM_HEADER_KEY ="hbase.rest.csrf.custom.header";
+  static final String REST_CSRF_CUSTOM_HEADER_DEFAULT = "X-XSRF-HEADER";
+  static final String REST_CSRF_METHODS_TO_IGNORE_KEY = "hbase.rest.csrf.methods.to.ignore";
+  static final String REST_CSRF_METHODS_TO_IGNORE_DEFAULT = "GET,OPTIONS,HEAD,TRACE";
+  public static final String SKIP_LOGIN_KEY = "hbase.rest.skip.login";
 
   private static final String PATH_SPEC_ANY = "/*";
 
-  static String REST_HTTP_ALLOW_OPTIONS_METHOD = "hbase.rest.http.allow.options.method";
+  static final String REST_HTTP_ALLOW_OPTIONS_METHOD = "hbase.rest.http.allow.options.method";
   // HTTP OPTIONS method is commonly used in REST APIs for negotiation. So it is enabled by default.
   private static boolean REST_HTTP_ALLOW_OPTIONS_METHOD_DEFAULT = true;
+  static final String REST_CSRF_BROWSER_USERAGENTS_REGEX_KEY =
+    "hbase.rest-csrf.browser-useragents-regex";
+
+  // HACK, making this static for AuthFilter to get at our configuration. Necessary for unit tests.
+  public static Configuration conf = null;
+  private final UserProvider userProvider;
+  private Server server;
+  private InfoServer infoServer;
+
+  public RESTServer(Configuration conf) {
+    RESTServer.conf = conf;
+    this.userProvider = UserProvider.instantiate(conf);
+  }
 
   private static void printUsageAndExit(Options options, int exitCode) {
     HelpFormatter formatter = new HelpFormatter();
@@ -107,26 +121,7 @@ public class RESTServer implements Constants {
     System.exit(exitCode);
   }
 
-  /**
-   * Returns a list of strings from a comma-delimited configuration value.
-   *
-   * @param conf configuration to check
-   * @param name configuration property name
-   * @param defaultValue default value if no value found for name
-   * @return list of strings from comma-delimited configuration value, or an
-   *     empty list if not found
-   */
-  private static List<String> getTrimmedStringList(Configuration conf,
-    String name, String defaultValue) {
-    String valueString = conf.get(name, defaultValue);
-    if (valueString == null) {
-      return new ArrayList<>();
-    }
-    return new ArrayList<>(StringUtils.getTrimmedStringCollection(valueString));
-  }
-
-  static String REST_CSRF_BROWSER_USERAGENTS_REGEX_KEY = "hbase.rest-csrf.browser-useragents-regex";
-  static void addCSRFFilter(ServletContextHandler ctxHandler, Configuration conf) {
+  void addCSRFFilter(ServletContextHandler ctxHandler, Configuration conf) {
     restCSRFEnabled = conf.getBoolean(REST_CSRF_ENABLED_KEY, REST_CSRF_ENABLED_DEFAULT);
     if (restCSRFEnabled) {
       Map<String, String> restCsrfParams = RestCsrfPreventionFilter
@@ -153,7 +148,10 @@ public class RESTServer implements Constants {
       String principalConfig = conf.get(REST_KERBEROS_PRINCIPAL);
       Preconditions.checkArgument(principalConfig != null && !principalConfig.isEmpty(),
         REST_KERBEROS_PRINCIPAL + " should be set if security is enabled");
-      userProvider.login(REST_KEYTAB_FILE, REST_KERBEROS_PRINCIPAL, machineName);
+      // Hook for unit tests, this will log out any other user and mess up tests.
+      if (!conf.getBoolean(SKIP_LOGIN_KEY, false)) {
+        userProvider.login(REST_KEYTAB_FILE, REST_KERBEROS_PRINCIPAL, machineName);
+      }
       if (conf.get(REST_AUTHENTICATION_TYPE) != null) {
         containerClass = RESTServletContainer.class;
         FilterHolder authFilter = new FilterHolder();
@@ -165,7 +163,7 @@ public class RESTServer implements Constants {
     return new Pair<>(null, containerClass);
   }
 
-  private static void parseCommandLine(String[] args, RESTServlet servlet) {
+  private static void parseCommandLine(String[] args, Configuration conf) {
     Options options = new Options();
     options.addOption("p", "port", true, "Port to bind to [default: " + DEFAULT_LISTEN_PORT + "]");
     options.addOption("ro", "readonly", false, "Respond only to GET HTTP " +
@@ -183,7 +181,7 @@ public class RESTServer implements Constants {
     // check for user-defined port setting, if so override the conf
     if (commandLine != null && commandLine.hasOption("port")) {
       String val = commandLine.getOptionValue("port");
-      servlet.getConfiguration().setInt("hbase.rest.port", Integer.parseInt(val));
+      conf.setInt("hbase.rest.port", Integer.parseInt(val));
       if (LOG.isDebugEnabled()) {
         LOG.debug("port set to " + val);
       }
@@ -191,7 +189,7 @@ public class RESTServer implements Constants {
 
     // check if server should only process GET requests, if so override the conf
     if (commandLine != null && commandLine.hasOption("readonly")) {
-      servlet.getConfiguration().setBoolean("hbase.rest.readonly", true);
+      conf.setBoolean("hbase.rest.readonly", true);
       if (LOG.isDebugEnabled()) {
         LOG.debug("readonly set to true");
       }
@@ -200,13 +198,19 @@ public class RESTServer implements Constants {
     // check for user-defined info server port setting, if so override the conf
     if (commandLine != null && commandLine.hasOption("infoport")) {
       String val = commandLine.getOptionValue("infoport");
-      servlet.getConfiguration().setInt("hbase.rest.info.port", Integer.parseInt(val));
+      conf.setInt("hbase.rest.info.port", Integer.parseInt(val));
       if (LOG.isDebugEnabled()) {
         LOG.debug("Web UI port set to " + val);
       }
     }
 
-    @SuppressWarnings("unchecked")
+    if (commandLine != null && commandLine.hasOption("skipLogin")) {
+      conf.setBoolean(SKIP_LOGIN_KEY, true);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Skipping Kerberos login for REST server");
+      }
+    }
+
     List<String> remainingArgs = commandLine != null ? commandLine.getArgList() : new ArrayList<>();
     if (remainingArgs.size() != 1) {
       printUsageAndExit(options, 1);
@@ -222,29 +226,27 @@ public class RESTServer implements Constants {
     }
   }
 
+
   /**
-   * The main method for the HBase rest server.
-   * @param args command-line arguments
-   * @throws Exception exception
+   * Runs the REST server.
    */
-  public static void main(String[] args) throws Exception {
-    LOG.info("***** STARTING service '" + RESTServer.class.getSimpleName() + "' *****");
-    VersionInfo.logVersion();
-    Configuration conf = HBaseConfiguration.create();
-    UserProvider userProvider = UserProvider.instantiate(conf);
+  public synchronized void run() throws Exception {
     Pair<FilterHolder, Class<? extends ServletContainer>> pair = loginServerPrincipal(
       userProvider, conf);
     FilterHolder authFilter = pair.getFirst();
+    Class<? extends ServletContainer> containerClass = pair.getSecond();
     RESTServlet servlet = RESTServlet.getInstance(conf, userProvider);
 
-    parseCommandLine(args, servlet);
 
     // Set up the Jersey servlet container for Jetty
     // The Jackson1Feature is a signal to Jersey that it should use jackson doing json.
     // See here: https://stackoverflow.com/questions/39458230/how-register-jacksonfeature-on-clientconfig
     ResourceConfig application = new ResourceConfig().
         packages("org.apache.hadoop.hbase.rest").register(JacksonJaxbJsonProvider.class);
-    ServletHolder sh = new ServletHolder(new ServletContainer(application));
+    // Using our custom ServletContainer is tremendously important. This is what makes sure the
+    // UGI.doAs() is done for the remoteUser, and calls are not made as the REST server itself.
+    ServletContainer servletContainer = ReflectionUtils.newInstance(containerClass, application);
+    ServletHolder sh = new ServletHolder(servletContainer);
 
     // Set the default max thread number to 100 to limit
     // the number of concurrent requests so that REST server doesn't OOM easily.
@@ -261,7 +263,7 @@ public class RESTServer implements Constants {
         new QueuedThreadPool(maxThreads, minThreads, idleTimeout, new ArrayBlockingQueue<>(queueSize)) :
         new QueuedThreadPool(maxThreads, minThreads, idleTimeout);
 
-    Server server = new Server(threadPool);
+    this.server = new Server(threadPool);
 
     // Setup JMX
     MBeanContainer mbContainer=new MBeanContainer(ManagementFactory.getPlatformMBeanServer());
@@ -355,18 +357,73 @@ public class RESTServer implements Constants {
     if (port >= 0) {
       conf.setLong("startcode", System.currentTimeMillis());
       String a = conf.get("hbase.rest.info.bindAddress", "0.0.0.0");
-      InfoServer infoServer = new InfoServer("rest", a, port, false, conf);
-      infoServer.setAttribute("hbase.conf", conf);
-      infoServer.start();
+      this.infoServer = new InfoServer("rest", a, port, false, conf);
+      this.infoServer.setAttribute("hbase.conf", conf);
+      this.infoServer.start();
     }
     try {
       // start server
       server.start();
-      server.join();
     } catch (Exception e) {
       LOG.error(HBaseMarkers.FATAL, "Failed to start server", e);
+      throw e;
+    }
+  }
+
+  public synchronized void join() throws Exception {
+    if (server == null) {
+      throw new IllegalStateException("Server is not running");
+    }
+    server.join();
+  }
+
+  public synchronized void stop() throws Exception {
+    if (server == null) {
+      throw new IllegalStateException("Server is not running");
+    }
+    server.stop();
+    server = null;
+    RESTServlet.stop();
+  }
+
+  public synchronized int getPort() {
+    if (server == null) {
+      throw new IllegalStateException("Server is not running");
+    }
+    return ((ServerConnector) server.getConnectors()[0]).getLocalPort();
+  }
+
+  @SuppressWarnings("deprecation")
+  public synchronized int getInfoPort() {
+    if (infoServer == null) {
+      throw new IllegalStateException("InfoServer is not running");
+    }
+    return infoServer.getPort();
+  }
+
+  public Configuration getConf() {
+    return conf;
+  }
+
+  /**
+   * The main method for the HBase rest server.
+   * @param args command-line arguments
+   * @throws Exception exception
+   */
+  public static void main(String[] args) throws Exception {
+    LOG.info("***** STARTING service '" + RESTServer.class.getSimpleName() + "' *****");
+    VersionInfo.logVersion();
+    final Configuration conf = HBaseConfiguration.create();
+    parseCommandLine(args, conf);
+    RESTServer server = new RESTServer(conf);
+
+    try {
+      server.run();
+      server.join();
+    } catch (Exception e) {
       System.exit(1);
     }
+
     LOG.info("***** STOPPING service '" + RESTServer.class.getSimpleName() + "' *****");
   }
 }
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServletContainer.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServletContainer.java
index 3b5dc2b..2b6a6c7 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServletContainer.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServletContainer.java
@@ -24,14 +24,13 @@ import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
-import org.apache.yetus.audience.InterfaceAudience;
-
-import org.glassfish.jersey.servlet.ServletContainer;
-
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.apache.hadoop.conf.Configuration;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.glassfish.jersey.server.ResourceConfig;
+import org.glassfish.jersey.servlet.ServletContainer;
 
 /**
  * REST servlet container. It is used to get the remote request user
@@ -41,6 +40,10 @@ import org.apache.hadoop.conf.Configuration;
 public class RESTServletContainer extends ServletContainer {
   private static final long serialVersionUID = -2474255003443394314L;
 
+  public RESTServletContainer(ResourceConfig config) {
+    super(config);
+  }
+
   /**
    * This container is used only if authentication and
    * impersonation is enabled. The remote request user is used
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
index e617cd4..83a70ff 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/SchemaResource.java
@@ -28,6 +28,7 @@ import javax.ws.rs.GET;
 import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.CacheControl;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.Response;
@@ -137,6 +138,7 @@ public class SchemaResource extends ResourceBase {
       }
       return Response.created(uriInfo.getAbsolutePath()).build();
     } catch (Exception e) {
+      LOG.info("Caught exception", e);
       servlet.getMetrics().incrementFailedPutRequests(1);
       return processException(e);
     }
@@ -191,6 +193,10 @@ public class SchemaResource extends ResourceBase {
       }
     } catch (Exception e) {
       servlet.getMetrics().incrementFailedPutRequests(1);
+      // Avoid re-unwrapping the exception
+      if (e instanceof WebApplicationException) {
+        throw (WebApplicationException) e;
+      }
       return processException(e);
     }
   }
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/AuthFilter.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/AuthFilter.java
index 216a3fc..b9b8a00 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/AuthFilter.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/AuthFilter.java
@@ -28,6 +28,7 @@ import javax.servlet.FilterConfig;
 import javax.servlet.ServletException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.rest.RESTServer;
 import org.apache.hadoop.hbase.util.DNS;
 import org.apache.hadoop.hbase.util.Strings;
 import org.apache.hadoop.security.SecurityUtil;
@@ -57,7 +58,14 @@ public class AuthFilter extends AuthenticationFilter {
     //setting the cookie path to root '/' so it is used for all resources.
     props.setProperty(AuthenticationFilter.COOKIE_PATH, "/");
 
-    Configuration conf = HBaseConfiguration.create();
+    Configuration conf = null;
+    // Dirty hack to get at the RESTServer's configuration. These should be pulled out
+    // of the FilterConfig.
+    if (RESTServer.conf != null) {
+      conf = RESTServer.conf;
+    } else {
+      conf = HBaseConfiguration.create();
+    }
     for (Map.Entry<String, String> entry : conf) {
       String name = entry.getKey();
       if (name.startsWith(REST_PREFIX)) {
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 38c734d..ffa1835 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
@@ -18,90 +18,37 @@
  */
 package org.apache.hadoop.hbase.rest;
 
-import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider;
-import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.http.HttpServerUtil;
 import org.apache.hadoop.util.StringUtils;
 
-import org.eclipse.jetty.server.HttpConfiguration;
-import org.eclipse.jetty.server.HttpConnectionFactory;
-import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.servlet.ServletContextHandler;
-import org.eclipse.jetty.server.ServerConnector;
-import org.eclipse.jetty.servlet.ServletHolder;
-
-import org.glassfish.jersey.server.ResourceConfig;
-import org.glassfish.jersey.servlet.ServletContainer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.servlet.DispatcherType;
-import java.util.Arrays;
-import java.util.EnumSet;
-
 public class HBaseRESTTestingUtility {
 
   private static final Logger LOG = LoggerFactory.getLogger(HBaseRESTTestingUtility.class);
 
-  private int testServletPort;
-  private Server server;
+  private RESTServer server;
 
   public int getServletPort() {
-    return testServletPort;
+    return server.getPort();
   }
 
   public void startServletContainer(Configuration conf) throws Exception {
     if (server != null) {
-      LOG.error("ServletContainer already running");
+      LOG.error("RESTServer already running");
       return;
     }
 
-    // Inject the conf for the test by being first to make singleton
-    RESTServlet.getInstance(conf, UserProvider.instantiate(conf));
-
-    // set up the Jersey servlet container for Jetty
-    ResourceConfig app = new ResourceConfig().
-        packages("org.apache.hadoop.hbase.rest").register(JacksonJaxbJsonProvider.class);
-    ServletHolder sh = new ServletHolder(new ServletContainer(app));
-
-    // set up Jetty and run the embedded server
-    server = new Server(0);
-    LOG.info("configured " + ServletContainer.class.getName());
-
-    HttpConfiguration httpConfig = new HttpConfiguration();
-    httpConfig.setSendDateHeader(false);
-    httpConfig.setSendServerVersion(false);
-    ServerConnector serverConnector = new ServerConnector(server, new HttpConnectionFactory(httpConfig));
-    serverConnector.setPort(testServletPort);
-
-    server.addConnector(serverConnector);
-
-    // set up context
-    ServletContextHandler ctxHandler = new ServletContextHandler(server, "/", ServletContextHandler.SESSIONS);
-    ctxHandler.addServlet(sh, "/*");
-    // Load filters specified from configuration.
-    String[] filterClasses = conf.getStrings(Constants.FILTER_CLASSES,
-        ArrayUtils.EMPTY_STRING_ARRAY);
-    for (String filter : filterClasses) {
-      filter = filter.trim();
-      ctxHandler.addFilter(filter, "/*", EnumSet.of(DispatcherType.REQUEST));
-    }
-    LOG.info("Loaded filter classes :" + Arrays.toString(filterClasses));
-
-    conf.set(RESTServer.REST_CSRF_BROWSER_USERAGENTS_REGEX_KEY, ".*");
-    RESTServer.addCSRFFilter(ctxHandler, conf);
-
-    HttpServerUtil.constrainHttpMethods(ctxHandler, true);
+    conf.setInt("hbase.rest.port", 0);
+    conf.setInt("hbase.rest.info.port", -1);
+    conf.setBoolean(RESTServer.SKIP_LOGIN_KEY, true);
 
-    // start the server
-    server.start();
-    // get the port
-    testServletPort = ((ServerConnector)server.getConnectors()[0]).getLocalPort();
+    server = new RESTServer(conf);
+    server.run();
 
     LOG.info("started " + server.getClass().getName() + " on port " +
-      testServletPort);
+      server.getPort());
   }
 
   public void shutdownServletContainer() {
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
index a8af6c4..5d8602f 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
@@ -100,6 +100,9 @@ public class TestMultiRowResource {
   public static void setUpBeforeClass() throws Exception {
     conf = TEST_UTIL.getConfiguration();
     conf.setBoolean(RESTServer.REST_CSRF_ENABLED_KEY, csrfEnabled);
+    if (csrfEnabled) {
+      conf.set(RESTServer.REST_CSRF_BROWSER_USERAGENTS_REGEX_KEY, ".*");
+    }
     extraHdr = new BasicHeader(RESTServer.REST_CSRF_CUSTOM_HEADER_DEFAULT, "");
     TEST_UTIL.startMiniCluster();
     REST_TEST_UTIL.startServletContainer(conf);
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java
index ab9ac12..1c2a7b9 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java
@@ -92,6 +92,9 @@ public class TestSchemaResource {
   public static void setUpBeforeClass() throws Exception {
     conf = TEST_UTIL.getConfiguration();
     conf.setBoolean(RESTServer.REST_CSRF_ENABLED_KEY, csrfEnabled);
+    if (csrfEnabled) {
+      conf.set(RESTServer.REST_CSRF_BROWSER_USERAGENTS_REGEX_KEY, ".*");
+    }
     extraHdr = new BasicHeader(RESTServer.REST_CSRF_CUSTOM_HEADER_DEFAULT, "");
     TEST_UTIL.startMiniCluster();
     REST_TEST_UTIL.startServletContainer(conf);
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSecureRESTServer.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSecureRESTServer.java
new file mode 100644
index 0000000..58f7c10
--- /dev/null
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSecureRESTServer.java
@@ -0,0 +1,428 @@
+/**
+ * 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.rest;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider;
+
+import java.io.File;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.security.Principal;
+import java.security.PrivilegedExceptionAction;
+
+import javax.ws.rs.core.MediaType;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.StartMiniClusterOption;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.http.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.hbase.rest.model.CellModel;
+import org.apache.hadoop.hbase.rest.model.CellSetModel;
+import org.apache.hadoop.hbase.rest.model.RowModel;
+import org.apache.hadoop.hbase.security.HBaseKerberosUtils;
+import org.apache.hadoop.hbase.security.access.AccessControlClient;
+import org.apache.hadoop.hbase.security.access.AccessControlConstants;
+import org.apache.hadoop.hbase.security.access.AccessController;
+import org.apache.hadoop.hbase.security.access.Permission.Action;
+import org.apache.hadoop.hbase.security.token.TokenProvider;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.util.KerberosName;
+import org.apache.http.HttpEntity;
+import org.apache.http.HttpHost;
+import org.apache.http.auth.AuthSchemeProvider;
+import org.apache.http.auth.AuthScope;
+import org.apache.http.auth.Credentials;
+import org.apache.http.client.AuthCache;
+import org.apache.http.client.CredentialsProvider;
+import org.apache.http.client.config.AuthSchemes;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPut;
+import org.apache.http.client.protocol.HttpClientContext;
+import org.apache.http.config.Registry;
+import org.apache.http.config.RegistryBuilder;
+import org.apache.http.conn.HttpClientConnectionManager;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.auth.SPNegoSchemeFactory;
+import org.apache.http.impl.client.BasicAuthCache;
+import org.apache.http.impl.client.BasicCredentialsProvider;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
+import org.apache.http.util.EntityUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 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 TestSecureRESTServer {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestSecureRESTServer.class);
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestSecureRESTServer.class);
+  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  private static final HBaseRESTTestingUtility REST_TEST = new HBaseRESTTestingUtility();
+  private static MiniHBaseCluster CLUSTER;
+
+  private static final String HOSTNAME = "localhost";
+  private static final String CLIENT_PRINCIPAL = "client";
+  // The principal for accepting SPNEGO authn'ed requests (*must* be HTTP/fqdn)
+  private static final String SPNEGO_SERVICE_PRINCIPAL = "HTTP/" + HOSTNAME;
+  // The principal we use to connect to HBase
+  private static final String REST_SERVER_PRINCIPAL = "rest";
+  private static final String SERVICE_PRINCIPAL = "hbase/" + HOSTNAME;
+
+  private static URL baseUrl;
+  private static MiniKdc KDC;
+  private static RESTServer server;
+  private static File restServerKeytab;
+  private static File clientKeytab;
+  private static File serviceKeytab;
+
+  @BeforeClass
+  public static void setupServer() throws Exception {
+    final File target = new File(System.getProperty("user.dir"), "target");
+    assertTrue(target.exists());
+
+    /*
+     * Keytabs
+     */
+    File keytabDir = new File(target, TestSecureRESTServer.class.getSimpleName()
+        + "_keytabs");
+    if (keytabDir.exists()) {
+      FileUtils.deleteDirectory(keytabDir);
+    }
+    keytabDir.mkdirs();
+    // Keytab for HBase services (RS, Master)
+    serviceKeytab = new File(keytabDir, "hbase.service.keytab");
+    // The keytab for the REST server
+    restServerKeytab = new File(keytabDir, "spnego.keytab");
+    // Keytab for the client
+    clientKeytab = new File(keytabDir, CLIENT_PRINCIPAL + ".keytab");
+
+    /*
+     * Update UGI
+     */
+    Configuration conf = TEST_UTIL.getConfiguration();
+
+    /*
+     * Start KDC
+     */
+    KDC = TEST_UTIL.setupMiniKdc(serviceKeytab);
+    KDC.createPrincipal(clientKeytab, CLIENT_PRINCIPAL);
+    KDC.createPrincipal(serviceKeytab, SERVICE_PRINCIPAL);
+    // REST server's keytab contains keys for both principals REST uses
+    KDC.createPrincipal(restServerKeytab, SPNEGO_SERVICE_PRINCIPAL, REST_SERVER_PRINCIPAL);
+
+    // Set configuration for HBase
+    HBaseKerberosUtils.setPrincipalForTesting(SERVICE_PRINCIPAL + "@" + KDC.getRealm());
+    HBaseKerberosUtils.setKeytabFileForTesting(serviceKeytab.getAbsolutePath());
+    // Why doesn't `setKeytabFileForTesting` do this?
+    conf.set("hbase.master.keytab.file", serviceKeytab.getAbsolutePath());
+    conf.set("hbase.regionserver.hostname", "localhost");
+    conf.set("hbase.master.hostname", "localhost");
+    HBaseKerberosUtils.setSecuredConfiguration(conf,
+        SERVICE_PRINCIPAL+ "@" + KDC.getRealm(), SPNEGO_SERVICE_PRINCIPAL+ "@" + KDC.getRealm());
+    setHdfsSecuredConfiguration(conf);
+    conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
+        TokenProvider.class.getName(), AccessController.class.getName());
+    conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
+        AccessController.class.getName());
+    conf.setStrings(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY,
+        AccessController.class.getName());
+    // Enable EXEC permission checking
+    conf.setBoolean(AccessControlConstants.EXEC_PERMISSION_CHECKS_KEY, true);
+    conf.set("hbase.superuser", "hbase");
+    conf.set("hadoop.proxyuser.rest.hosts", "*");
+    conf.set("hadoop.proxyuser.rest.users", "*");
+    UserGroupInformation.setConfiguration(conf);
+
+    updateKerberosConfiguration(conf, REST_SERVER_PRINCIPAL, SPNEGO_SERVICE_PRINCIPAL,
+        restServerKeytab);
+
+    // Start HDFS
+    TEST_UTIL.startMiniCluster(StartMiniClusterOption.builder()
+        .numMasters(1)
+        .numRegionServers(1)
+        .numZkServers(1)
+        .build());
+
+    // Start REST
+    UserGroupInformation restUser = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+        REST_SERVER_PRINCIPAL, restServerKeytab.getAbsolutePath());
+    restUser.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        REST_TEST.startServletContainer(conf);
+        return null;
+      }
+    });
+    baseUrl = new URL("http://localhost:" + REST_TEST.getServletPort());
+
+    LOG.info("HTTP server started: "+ baseUrl);
+    TEST_UTIL.waitTableAvailable(TableName.valueOf("hbase:acl"));
+
+    // Let the REST server create, read, and write globally
+    UserGroupInformation superuser = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+        SERVICE_PRINCIPAL, serviceKeytab.getAbsolutePath());
+    superuser.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
+          AccessControlClient.grant(
+              conn, REST_SERVER_PRINCIPAL, Action.CREATE, Action.READ, Action.WRITE);
+        } catch (Throwable t) {
+          if (t instanceof Exception) {
+            throw (Exception) t;
+          } else {
+            throw new Exception(t);
+          }
+        }
+        return null;
+      }
+    });
+  }
+
+  @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 (CLUSTER != null) {
+        CLUSTER.shutdown();
+      }
+    } catch (Exception e) {
+      LOG.info("Failed to stop HBase cluster", e);
+    }
+    try {
+      if (null != KDC) {
+        KDC.stop();
+      }
+    } catch (Exception e) {
+      LOG.info("Failed to stop mini KDC", e);
+    }
+  }
+
+  private static void setHdfsSecuredConfiguration(Configuration conf) throws Exception {
+    // Set principal+keytab configuration for HDFS
+    conf.set(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY,
+        SERVICE_PRINCIPAL + "@" + KDC.getRealm());
+    conf.set(DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY, serviceKeytab.getAbsolutePath());
+    conf.set(DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY,
+        SERVICE_PRINCIPAL + "@" + KDC.getRealm());
+    conf.set(DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY, serviceKeytab.getAbsolutePath());
+    conf.set(DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY,
+        SPNEGO_SERVICE_PRINCIPAL + "@" + KDC.getRealm());
+    // Enable token access for HDFS blocks
+    conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true);
+    // Only use HTTPS (required because we aren't using "secure" ports)
+    conf.set(DFSConfigKeys.DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTPS_ONLY.name());
+    // Bind on localhost for spnego to have a chance at working
+    conf.set(DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY, "localhost:0");
+    conf.set(DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY, "localhost:0");
+
+    // Generate SSL certs
+    File keystoresDir = new File(TEST_UTIL.getDataTestDir("keystore").toUri().getPath());
+    keystoresDir.mkdirs();
+    String sslConfDir = KeyStoreTestUtil.getClasspathDir(TestSecureRESTServer.class);
+    KeyStoreTestUtil.setupSSLConfig(keystoresDir.getAbsolutePath(), sslConfDir, conf, false);
+
+    // Magic flag to tell hdfs to not fail on using ports above 1024
+    conf.setBoolean("ignore.secure.ports.for.testing", true);
+  }
+
+  private static void updateKerberosConfiguration(Configuration conf,
+      String serverPrincipal, String spnegoPrincipal, File serverKeytab) {
+    KerberosName.setRules("DEFAULT");
+
+    // Enable Kerberos (pre-req)
+    conf.set("hbase.security.authentication", "kerberos");
+    conf.set(RESTServer.REST_AUTHENTICATION_TYPE, "kerberos");
+    // User to talk to HBase as
+    conf.set(RESTServer.REST_KERBEROS_PRINCIPAL, serverPrincipal);
+    // User to accept SPNEGO-auth'd http calls as
+    conf.set("hbase.rest.authentication.kerberos.principal", spnegoPrincipal);
+    // Keytab for both principals above
+    conf.set(RESTServer.REST_KEYTAB_FILE, serverKeytab.getAbsolutePath());
+    conf.set("hbase.rest.authentication.kerberos.keytab", serverKeytab.getAbsolutePath());
+  }
+
+  @Test
+  public void testPositiveAuthorization() throws Exception {
+    // Create a table, write a row to it, grant read perms to the client
+    UserGroupInformation superuser = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+        SERVICE_PRINCIPAL, serviceKeytab.getAbsolutePath());
+    final TableName table = TableName.valueOf("publicTable");
+    superuser.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
+          TableDescriptor desc = TableDescriptorBuilder.newBuilder(table)
+              .setColumnFamily(ColumnFamilyDescriptorBuilder.of("f1"))
+              .build();
+          conn.getAdmin().createTable(desc);
+          try (Table t = conn.getTable(table)) {
+            Put p = new Put(Bytes.toBytes("a"));
+            p.addColumn(Bytes.toBytes("f1"), new byte[0], Bytes.toBytes("1"));
+            t.put(p);
+          }
+          AccessControlClient.grant(conn, CLIENT_PRINCIPAL, Action.READ);
+        } catch (Throwable e) {
+          if (e instanceof Exception) {
+            throw (Exception) e;
+          } else {
+            throw new Exception(e);
+          }
+        }
+        return null;
+      }
+    });
+
+    // Read that row as the client
+    Pair<CloseableHttpClient,HttpClientContext> pair = getClient();
+    CloseableHttpClient client = pair.getFirst();
+    HttpClientContext context = pair.getSecond();
+
+    HttpGet get = new HttpGet(new URL("http://localhost:"+ REST_TEST.getServletPort()).toURI()
+        + "/" + table + "/a");
+    get.addHeader("Accept", "application/json");
+    UserGroupInformation user = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+        CLIENT_PRINCIPAL, clientKeytab.getAbsolutePath());
+    String jsonResponse = user.doAs(new PrivilegedExceptionAction<String>() {
+      @Override
+      public String run() throws Exception {
+        try (CloseableHttpResponse response = client.execute(get, context)) {
+          final int statusCode = response.getStatusLine().getStatusCode();
+          assertEquals(response.getStatusLine().toString(), HttpURLConnection.HTTP_OK, statusCode);
+          HttpEntity entity = response.getEntity();
+          return EntityUtils.toString(entity);
+        }
+      }
+    });
+    ObjectMapper mapper = new JacksonJaxbJsonProvider()
+        .locateMapper(CellSetModel.class, MediaType.APPLICATION_JSON_TYPE);
+    CellSetModel model = mapper.readValue(jsonResponse, CellSetModel.class);
+    assertEquals(1, model.getRows().size());
+    RowModel row = model.getRows().get(0);
+    assertEquals("a", Bytes.toString(row.getKey()));
+    assertEquals(1, row.getCells().size());
+    CellModel cell = row.getCells().get(0);
+    assertEquals("1", Bytes.toString(cell.getValue()));
+  }
+
+  @Test
+  public void testNegativeAuthorization() throws Exception {
+    Pair<CloseableHttpClient,HttpClientContext> pair = getClient();
+    CloseableHttpClient client = pair.getFirst();
+    HttpClientContext context = pair.getSecond();
+
+    StringEntity entity = new StringEntity(
+        "{\"name\":\"test\", \"ColumnSchema\":[{\"name\":\"f\"}]}", ContentType.APPLICATION_JSON);
+    HttpPut put = new HttpPut("http://localhost:"+ REST_TEST.getServletPort() + "/test/schema");
+    put.setEntity(entity);
+
+
+    UserGroupInformation unprivileged = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+        CLIENT_PRINCIPAL, clientKeytab.getAbsolutePath());
+    unprivileged.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        try (CloseableHttpResponse response = client.execute(put, context)) {
+          final int statusCode = response.getStatusLine().getStatusCode();
+          HttpEntity entity = response.getEntity();
+          assertEquals("Got response: "+ EntityUtils.toString(entity),
+              HttpURLConnection.HTTP_FORBIDDEN, statusCode);
+        }
+        return null;
+      }
+    });
+  }
+
+  private Pair<CloseableHttpClient,HttpClientContext> getClient() {
+    HttpClientConnectionManager pool = new PoolingHttpClientConnectionManager();
+    HttpHost host = new HttpHost("localhost", REST_TEST.getServletPort());
+    Registry<AuthSchemeProvider> authRegistry =
+        RegistryBuilder.<AuthSchemeProvider>create().register(AuthSchemes.SPNEGO,
+            new SPNegoSchemeFactory(true, true)).build();
+    CredentialsProvider credentialsProvider = new BasicCredentialsProvider();
+    credentialsProvider.setCredentials(AuthScope.ANY, EmptyCredentials.INSTANCE);
+    AuthCache authCache = new BasicAuthCache();
+
+    CloseableHttpClient client = HttpClients.custom()
+        .setDefaultAuthSchemeRegistry(authRegistry)
+        .setConnectionManager(pool).build();
+
+    HttpClientContext context = HttpClientContext.create();
+    context.setTargetHost(host);
+    context.setCredentialsProvider(credentialsProvider);
+    context.setAuthSchemeRegistry(authRegistry);
+    context.setAuthCache(authCache);
+
+    return new Pair<>(client, context);
+  }
+
+  private static class EmptyCredentials implements Credentials {
+    public static final EmptyCredentials INSTANCE = new EmptyCredentials();
+
+    @Override public String getPassword() {
+      return null;
+    }
+    @Override public Principal getUserPrincipal() {
+      return null;
+    }
+  }
+}
diff --git a/hbase-rest/src/test/resources/log4j.properties b/hbase-rest/src/test/resources/log4j.properties
index c322699..4e5f014 100644
--- a/hbase-rest/src/test/resources/log4j.properties
+++ b/hbase-rest/src/test/resources/log4j.properties
@@ -66,3 +66,4 @@ 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
+log4j.logger.org.apache.directory=WARN