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

[1/2] hbase git commit: Revert "Revert "HBASE-12894 Upgrade to Jetty 9 for REST / Info Server / Thrift Http Server""

Repository: hbase
Updated Branches:
  refs/heads/master f576f4678 -> ffe7dac53


http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/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
index 84c1fea..be8e98f 100644
--- 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
@@ -27,7 +27,6 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.Enumeration;
 import java.util.HashMap;
@@ -64,31 +63,35 @@ 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.mortbay.io.Buffer;
-import org.mortbay.jetty.Connector;
-import org.mortbay.jetty.Handler;
-import org.mortbay.jetty.MimeTypes;
-import org.mortbay.jetty.RequestLog;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.handler.ContextHandler;
-import org.mortbay.jetty.handler.ContextHandlerCollection;
-import org.mortbay.jetty.handler.HandlerCollection;
-import org.mortbay.jetty.handler.RequestLogHandler;
-import org.mortbay.jetty.nio.SelectChannelConnector;
-import org.mortbay.jetty.security.SslSocketConnector;
-import org.mortbay.jetty.servlet.Context;
-import org.mortbay.jetty.servlet.DefaultServlet;
-import org.mortbay.jetty.servlet.FilterHolder;
-import org.mortbay.jetty.servlet.FilterMapping;
-import org.mortbay.jetty.servlet.ServletHandler;
-import org.mortbay.jetty.servlet.ServletHolder;
-import org.mortbay.jetty.webapp.WebAppContext;
-import org.mortbay.thread.QueuedThreadPool;
-import org.mortbay.util.MultiException;
+
+import 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 com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import com.sun.jersey.spi.container.servlet.ServletContainer;
 
 /**
  * Create a Jetty embedded server to answer http requests. The primary goal
@@ -104,6 +107,8 @@ 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";
@@ -147,8 +152,8 @@ public class HttpServer implements FilterContainer {
      * listener in stop().
      */
     private final boolean isManaged;
-    private final Connector listener;
-    private ListenerInfo(boolean isManaged, Connector listener) {
+    private final ServerConnector listener;
+    private ListenerInfo(boolean isManaged, ServerConnector listener) {
       this.isManaged = isManaged;
       this.listener = listener;
     }
@@ -158,8 +163,8 @@ public class HttpServer implements FilterContainer {
 
   protected final WebAppContext webAppContext;
   protected final boolean findPort;
-  protected final Map<Context, Boolean> defaultContexts =
-      new HashMap<Context, Boolean>();
+  protected final Map<ServletContextHandler, Boolean> defaultContexts =
+      new HashMap<ServletContextHandler, Boolean>();
   protected final List<String> filterNames = new ArrayList<String>();
   static final String STATE_DESCRIPTION_ALIVE = " - alive";
   static final String STATE_DESCRIPTION_NOT_LIVE = " - not live";
@@ -169,7 +174,6 @@ public class HttpServer implements FilterContainer {
    */
   public static class Builder {
     private ArrayList<URI> endpoints = Lists.newArrayList();
-    private Connector connector;
     private Configuration conf;
     private String[] pathSpecs;
     private AccessControlList adminsAcl;
@@ -294,11 +298,6 @@ public class HttpServer implements FilterContainer {
       return this;
     }
 
-    public Builder setConnector(Connector connector) {
-      this.connector = connector;
-      return this;
-    }
-
     public Builder setPathSpec(String[] pathSpec) {
       this.pathSpecs = pathSpec;
       return this;
@@ -356,17 +355,15 @@ public class HttpServer implements FilterContainer {
         try {
           endpoints.add(0, new URI("http", "", bindAddress, port, "", "", ""));
         } catch (URISyntaxException e) {
-          throw new HadoopIllegalArgumentException("Invalid endpoint: "+ e);
-        }
+          throw new HadoopIllegalArgumentException("Invalid endpoint: "+ e); }
       }
 
-      if (endpoints.isEmpty() && connector == null) {
+      if (endpoints.isEmpty()) {
         throw new HadoopIllegalArgumentException("No endpoints specified");
       }
 
       if (hostName == null) {
-        hostName = endpoints.isEmpty() ? connector.getHost() : endpoints.get(
-            0).getHost();
+        hostName = endpoints.get(0).getHost();
       }
 
       if (this.conf == null) {
@@ -380,37 +377,53 @@ public class HttpServer implements FilterContainer {
             signatureSecretFileKey);
       }
 
-      if (connector != null) {
-        server.addUnmanagedListener(connector);
-      }
-
       for (URI ep : endpoints) {
-        Connector listener = null;
+        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 = HttpServer.createDefaultChannelConnector();
+          listener = new ServerConnector(server.webServer, new HttpConnectionFactory(httpConfig));
         } else if ("https".equals(scheme)) {
-          SslSocketConnector c = new SslSocketConnectorSecure();
-          c.setNeedClientAuth(needsClientAuth);
-          c.setKeyPassword(keyPassword);
+          HttpConfiguration httpsConfig = new HttpConfiguration(httpConfig);
+          httpsConfig.addCustomizer(new SecureRequestCustomizer());
+          SslContextFactory sslCtxFactory = new SslContextFactory();
+          sslCtxFactory.setNeedClientAuth(needsClientAuth);
+          sslCtxFactory.setKeyManagerPassword(keyPassword);
 
           if (keyStore != null) {
-            c.setKeystore(keyStore);
-            c.setKeystoreType(keyStoreType);
-            c.setPassword(keyStorePassword);
+            sslCtxFactory.setKeyStorePath(keyStore);
+            sslCtxFactory.setKeyStoreType(keyStoreType);
+            sslCtxFactory.setKeyStorePassword(keyStorePassword);
           }
 
           if (trustStore != null) {
-            c.setTruststore(trustStore);
-            c.setTruststoreType(trustStoreType);
-            c.setTrustPassword(trustStorePassword);
-          }
-          listener = c;
+            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);
@@ -430,12 +443,6 @@ public class HttpServer implements FilterContainer {
     this(name, bindAddress, port, findPort, new Configuration());
   }
 
-  @Deprecated
-  public HttpServer(String name, String bindAddress, int port,
-      boolean findPort, Configuration conf, Connector connector) throws IOException {
-    this(name, bindAddress, port, findPort, conf, null, connector, null);
-  }
-
   /**
    * Create a status server on the given port. Allows you to specify the
    * path specifications that this server will be serving so that they will be
@@ -453,7 +460,7 @@ public class HttpServer implements FilterContainer {
   @Deprecated
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, String[] pathSpecs) throws IOException {
-    this(name, bindAddress, port, findPort, conf, null, null, pathSpecs);
+    this(name, bindAddress, port, findPort, conf, null, pathSpecs);
   }
 
   /**
@@ -468,14 +475,14 @@ public class HttpServer implements FilterContainer {
   @Deprecated
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf) throws IOException {
-    this(name, bindAddress, port, findPort, conf, null, null, null);
+    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, null);
+    this(name, bindAddress, port, findPort, conf, adminsAcl, null);
   }
 
   /**
@@ -488,44 +495,33 @@ public class HttpServer implements FilterContainer {
    *        increment by 1 until it finds a free port.
    * @param conf Configuration
    * @param adminsAcl {@link AccessControlList} of the admins
-   * @param connector The jetty {@link Connector} to use
-   */
-  @Deprecated
-  public HttpServer(String name, String bindAddress, int port,
-      boolean findPort, Configuration conf, AccessControlList adminsAcl,
-      Connector connector) throws IOException {
-    this(name, bindAddress, port, findPort, conf, adminsAcl, connector, null);
-  }
-
-  /**
-   * Create a status server on the given port.
-   * The jsp scripts are taken from src/webapps/&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 connector A jetty connection listener
    * @param pathSpecs Path specifications that this httpserver will be serving.
    *        These will be added to any filters.
    */
   @Deprecated
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, AccessControlList adminsAcl,
-      Connector connector, String[] pathSpecs) throws IOException {
+      String[] pathSpecs) throws IOException {
     this(new Builder().setName(name)
         .addEndpoint(URI.create("http://" + bindAddress + ":" + port))
         .setFindPort(findPort).setConf(conf).setACL(adminsAcl)
-        .setConnector(connector).setPathSpec(pathSpecs));
+        .setPathSpec(pathSpecs));
   }
 
   private HttpServer(final Builder b) throws IOException {
     this.appDir = b.appDir;
     this.logDir = b.logDir;
     final String appDir = getWebAppsPath(b.name);
-    this.webServer = new Server();
+
+
+    int maxThreads = b.conf.getInt(HTTP_MAX_THREADS, -1);
+    // If HTTP_MAX_THREADS is not configured, QueueThreadPool() will use the
+    // default value (currently 250).
+    QueuedThreadPool threadPool = maxThreads == -1 ? new QueuedThreadPool()
+        : new QueuedThreadPool(maxThreads);
+    threadPool.setDaemon(true);
+    this.webServer = new Server(threadPool);
+
     this.adminsAcl = b.adminsAcl;
     this.webAppContext = createWebAppContext(b.name, b.conf, adminsAcl, appDir);
     this.findPort = b.findPort;
@@ -538,14 +534,6 @@ public class HttpServer implements FilterContainer {
 
     Preconditions.checkNotNull(webAppContext);
 
-    int maxThreads = conf.getInt(HTTP_MAX_THREADS, -1);
-    // If HTTP_MAX_THREADS is not configured, QueueThreadPool() will use the
-    // default value (currently 250).
-    QueuedThreadPool threadPool = maxThreads == -1 ? new QueuedThreadPool()
-        : new QueuedThreadPool(maxThreads);
-    threadPool.setDaemon(true);
-    webServer.setThreadPool(threadPool);
-
     ContextHandlerCollection contexts = new ContextHandlerCollection();
     RequestLog requestLog = HttpRequestLog.getRequestLog(name);
 
@@ -561,7 +549,7 @@ public class HttpServer implements FilterContainer {
 
     final String appDir = getWebAppsPath(name);
 
-    webServer.addHandler(webAppContext);
+    webServer.setHandler(webAppContext);
 
     addDefaultApps(contexts, appDir, conf);
 
@@ -589,11 +577,7 @@ public class HttpServer implements FilterContainer {
     }
   }
 
-  private void addUnmanagedListener(Connector connector) {
-    listeners.add(new ListenerInfo(false, connector));
-  }
-
-  private void addManagedListener(Connector connector) {
+  private void addManagedListener(ServerConnector connector) {
     listeners.add(new ListenerInfo(true, connector));
   }
 
@@ -614,33 +598,6 @@ public class HttpServer implements FilterContainer {
         Collections.<String, String> emptyMap(), new String[] { "/*" });
   }
 
-  /**
-   * Create a required listener for the Jetty instance listening on the port
-   * provided. This wrapper and all subclasses must create at least one
-   * listener.
-   */
-  public Connector createBaseListener(Configuration conf) throws IOException {
-    return HttpServer.createDefaultChannelConnector();
-  }
-
-  @InterfaceAudience.Private
-  public static Connector createDefaultChannelConnector() {
-    SelectChannelConnector ret = new SelectChannelConnector();
-    ret.setLowResourceMaxIdleTime(10000);
-    ret.setAcceptQueueSize(128);
-    ret.setResolveNames(false);
-    ret.setUseDirectBuffers(false);
-    if(Shell.WINDOWS) {
-      // result of setting the SO_REUSEADDR flag is different on Windows
-      // http://msdn.microsoft.com/en-us/library/ms740621(v=vs.85).aspx
-      // without this 2 NN's can start on the same machine and listen on
-      // the same port with indeterminate routing of incoming requests to them
-      ret.setReuseAddress(false);
-    }
-    ret.setHeaderBufferSize(1024*64);
-    return ret;
-  }
-
   /** Get an array of FilterConfiguration specified in the conf */
   private static FilterInitializer[] getFilterInitializers(Configuration conf) {
     if (conf == null) {
@@ -672,9 +629,10 @@ public class HttpServer implements FilterContainer {
         logDir = System.getProperty("hadoop.log.dir");
     }
     if (logDir != null) {
-      Context logContext = new Context(parent, "/logs");
-      logContext.setResourceBase(logDir);
+      ServletContextHandler logContext = new ServletContextHandler(parent, "/*");
       logContext.addServlet(AdminAuthorizedServlet.class, "/*");
+      logContext.setResourceBase(logDir);
+
       if (conf.getBoolean(
           ServerConfigurationKeys.HBASE_JETTY_LOGS_SERVE_ALIASES,
           ServerConfigurationKeys.DEFAULT_HBASE_JETTY_LOGS_SERVE_ALIASES)) {
@@ -689,7 +647,7 @@ public class HttpServer implements FilterContainer {
       defaultContexts.put(logContext, true);
     }
     // set up the context for "/static/*"
-    Context staticContext = new Context(parent, "/static");
+    ServletContextHandler staticContext = new ServletContextHandler(parent, "/static");
     staticContext.setResourceBase(appDir + "/static");
     staticContext.addServlet(DefaultServlet.class, "/*");
     staticContext.setDisplayName("static");
@@ -697,7 +655,7 @@ public class HttpServer implements FilterContainer {
     defaultContexts.put(staticContext, true);
   }
 
-  private void setContextAttributes(Context context, Configuration conf) {
+  private void setContextAttributes(ServletContextHandler context, Configuration conf) {
     context.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
     context.getServletContext().setAttribute(ADMINS_ACL, adminsAcl);
   }
@@ -724,30 +682,6 @@ public class HttpServer implements FilterContainer {
     addServlet("conf", "/conf", ConfServlet.class);
   }
 
-  public void addContext(Context ctxt, boolean isFiltered)
-      throws IOException {
-    webServer.addHandler(ctxt);
-    addNoCacheFilter(webAppContext);
-    defaultContexts.put(ctxt, isFiltered);
-  }
-
-  /**
-   * Add a context
-   * @param pathSpec The path spec for the context
-   * @param dir The directory containing the context
-   * @param isFiltered if true, the servlet is added to the filter path mapping
-   * @throws IOException
-   */
-  protected void addContext(String pathSpec, String dir, boolean isFiltered) throws IOException {
-    if (0 == webServer.getHandlers().length) {
-      throw new RuntimeException("Couldn't find handler");
-    }
-    WebAppContext webAppCtx = new WebAppContext();
-    webAppCtx.setContextPath(pathSpec);
-    webAppCtx.setWar(dir);
-    addContext(webAppCtx, true);
-  }
-
   /**
    * Set a value in the webapp context. These values are available to the jsp
    * pages as "application.getAttribute(name)".
@@ -767,10 +701,9 @@ public class HttpServer implements FilterContainer {
       final String pathSpec) {
     LOG.info("addJerseyResourcePackage: packageName=" + packageName
         + ", pathSpec=" + pathSpec);
-    final ServletHolder sh = new ServletHolder(ServletContainer.class);
-    sh.setInitParameter("com.sun.jersey.config.property.resourceConfigClass",
-        "com.sun.jersey.api.core.PackagesResourceConfig");
-    sh.setInitParameter("com.sun.jersey.config.property.packages", packageName);
+
+    ResourceConfig application = new ResourceConfig().packages(packageName);
+    final ServletHolder sh = new ServletHolder(new ServletContainer(application));
     webAppContext.addServlet(sh, pathSpec);
   }
 
@@ -828,7 +761,7 @@ public class HttpServer implements FilterContainer {
        FilterMapping fmap = new FilterMapping();
        fmap.setPathSpec(pathSpec);
        fmap.setFilterName(SPNEGO_FILTER);
-       fmap.setDispatches(Handler.ALL);
+       fmap.setDispatches(FilterMapping.ALL);
        handler.addFilterMapping(fmap);
     }
   }
@@ -842,12 +775,12 @@ public class HttpServer implements FilterContainer {
     LOG.info("Added filter " + name + " (class=" + classname
         + ") to context " + webAppContext.getDisplayName());
     final String[] ALL_URLS = { "/*" };
-    for (Map.Entry<Context, Boolean> e : defaultContexts.entrySet()) {
+    for (Map.Entry<ServletContextHandler, Boolean> e : defaultContexts.entrySet()) {
       if (e.getValue()) {
-        Context ctx = e.getKey();
-        defineFilter(ctx, name, classname, parameters, ALL_URLS);
+        ServletContextHandler handler = e.getKey();
+        defineFilter(handler, name, classname, parameters, ALL_URLS);
         LOG.info("Added filter " + name + " (class=" + classname
-            + ") to context " + ctx.getDisplayName());
+            + ") to context " + handler.getDisplayName());
       }
     }
     filterNames.add(name);
@@ -858,7 +791,7 @@ public class HttpServer implements FilterContainer {
       Map<String, String> parameters) {
     final String[] ALL_URLS = { "/*" };
     defineFilter(webAppContext, name, classname, parameters, ALL_URLS);
-    for (Context ctx : defaultContexts.keySet()) {
+    for (ServletContextHandler ctx : defaultContexts.keySet()) {
       defineFilter(ctx, name, classname, parameters, ALL_URLS);
     }
     LOG.info("Added global filter '" + name + "' (class=" + classname + ")");
@@ -867,19 +800,20 @@ public class HttpServer implements FilterContainer {
   /**
    * Define a filter for a context and set up default url mappings.
    */
-  public static void defineFilter(Context ctx, String name,
+  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);
-    holder.setInitParameters(parameters);
+    if (parameters != null) {
+      holder.setInitParameters(parameters);
+    }
     FilterMapping fmap = new FilterMapping();
     fmap.setPathSpecs(urls);
-    fmap.setDispatches(Handler.ALL);
+    fmap.setDispatches(FilterMapping.ALL);
     fmap.setFilterName(name);
-    ServletHandler handler = ctx.getServletHandler();
-    handler.addFilter(holder, fmap);
+    handler.getServletHandler().addFilter(holder, fmap);
   }
 
   /**
@@ -888,14 +822,13 @@ public class HttpServer implements FilterContainer {
    * @param webAppCtx The WebApplicationContext to add to
    */
   protected void addFilterPathMapping(String pathSpec,
-      Context webAppCtx) {
-    ServletHandler handler = webAppCtx.getServletHandler();
+      WebAppContext webAppCtx) {
     for(String name : filterNames) {
       FilterMapping fmap = new FilterMapping();
       fmap.setPathSpec(pathSpec);
       fmap.setFilterName(name);
-      fmap.setDispatches(Handler.ALL);
-      handler.addFilterMapping(fmap);
+      fmap.setDispatches(FilterMapping.ALL);
+      webAppCtx.getServletHandler().addFilterMapping(fmap);
     }
   }
 
@@ -937,7 +870,7 @@ public class HttpServer implements FilterContainer {
    */
   @Deprecated
   public int getPort() {
-    return webServer.getConnectors()[0].getLocalPort();
+    return ((ServerConnector)webServer.getConnectors()[0]).getLocalPort();
   }
 
   /**
@@ -951,9 +884,10 @@ public class HttpServer implements FilterContainer {
     if (index > webServer.getConnectors().length)
       return null;
 
-    Connector c = webServer.getConnectors()[index];
-    if (c.getLocalPort() == -1) {
-      // The connector is not bounded
+    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;
     }
 
@@ -1080,9 +1014,9 @@ public class HttpServer implements FilterContainer {
    */
   void openListeners() throws Exception {
     for (ListenerInfo li : listeners) {
-      Connector listener = li.listener;
-      if (!li.isManaged || li.listener.getLocalPort() != -1) {
-        // This listener is either started externally or has been bound
+      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();
@@ -1185,7 +1119,7 @@ public class HttpServer implements FilterContainer {
       StringBuilder sb = new StringBuilder("HttpServer (")
         .append(isAlive() ? STATE_DESCRIPTION_ALIVE : STATE_DESCRIPTION_NOT_LIVE).append("), listening at:");
       for (ListenerInfo li : listeners) {
-        Connector l = li.listener;
+        ServerConnector l = li.listener;
         sb.append(l.getHost()).append(":").append(l.getPort()).append("/,");
       }
       return sb.toString();
@@ -1442,12 +1376,8 @@ public class HttpServer implements FilterContainer {
      */
     private String inferMimeType(ServletRequest request) {
       String path = ((HttpServletRequest)request).getRequestURI();
-      ContextHandler.SContext sContext = (ContextHandler.SContext)config.getServletContext();
-      MimeTypes mimes = sContext.getContextHandler().getMimeTypes();
-      Buffer mimeBuffer = mimes.getMimeByExtension(path);
-      return (mimeBuffer == null) ? null : mimeBuffer.toString();
+      ServletContext context = config.getServletContext();
+      return context.getMimeType(path);
     }
-
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-server/src/main/java/org/apache/hadoop/hbase/jetty/SslSelectChannelConnectorSecure.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/jetty/SslSelectChannelConnectorSecure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/jetty/SslSelectChannelConnectorSecure.java
deleted file mode 100644
index 712b4f1..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/jetty/SslSelectChannelConnectorSecure.java
+++ /dev/null
@@ -1,36 +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.jetty;
-
-import java.io.IOException;
-import java.util.ArrayList;
-
-import javax.net.ssl.SSLEngine;
-
-import org.mortbay.jetty.security.SslSelectChannelConnector;
-
-/**
- * Avoid SSL V3.0 "Poodle" Vulnerability - CVE-2014-3566
- */
-public class SslSelectChannelConnectorSecure extends SslSelectChannelConnector {
-  @Override
-  protected SSLEngine createSSLEngine() throws IOException {
-    SSLEngine sslEngine = super.createSSLEngine();
-    ArrayList<String> secureProtocols = new ArrayList<String>();
-    for (String p : sslEngine.getEnabledProtocols()) {
-      if (!p.contains("SSLv3")) {
-        secureProtocols.add(p);
-      }
-    }
-    sslEngine.setEnabledProtocols(secureProtocols.toArray(new String[secureProtocols.size()]));
-    return sslEngine;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/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
index a66251f..cb93982 100644
--- 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
@@ -17,10 +17,10 @@
  */
 package org.apache.hadoop.hbase.util;
 
-import org.mortbay.jetty.security.Constraint;
-import org.mortbay.jetty.security.ConstraintMapping;
-import org.mortbay.jetty.security.SecurityHandler;
-import org.mortbay.jetty.servlet.Context;
+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.
@@ -28,9 +28,9 @@ import org.mortbay.jetty.servlet.Context;
 public class HttpServerUtil {
   /**
    * Add constraints to a Jetty Context to disallow undesirable Http methods.
-   * @param context The context to modify
+   * @param ctxHandler The context to modify
    */
-  public static void constrainHttpMethods(Context context) {
+  public static void constrainHttpMethods(ServletContextHandler ctxHandler) {
     Constraint c = new Constraint();
     c.setAuthenticate(true);
 
@@ -44,9 +44,9 @@ public class HttpServerUtil {
     cmo.setMethod("OPTIONS");
     cmo.setPathSpec("/*");
 
-    SecurityHandler sh = new SecurityHandler();
-    sh.setConstraintMappings(new ConstraintMapping[]{ cmt, cmo });
+    ConstraintSecurityHandler securityHandler = new ConstraintSecurityHandler();
+    securityHandler.setConstraintMappings(new ConstraintMapping[]{ cmt, cmo });
 
-    context.addHandler(sh);
+    ctxHandler.setSecurityHandler(securityHandler);
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/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
index 8fea254..b8d21d1 100644
--- 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
@@ -22,8 +22,9 @@ import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.log4j.Logger;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.mortbay.jetty.NCSARequestLog;
-import org.mortbay.jetty.RequestLog;
+
+import org.eclipse.jetty.server.RequestLog;
+import org.eclipse.jetty.server.NCSARequestLog;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/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
index 6642638..3a58adc 100644
--- 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
@@ -70,8 +70,9 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 import org.mockito.internal.util.reflection.Whitebox;
-import org.mortbay.jetty.Connector;
-import org.mortbay.util.ajax.JSON;
+
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.util.ajax.JSON;
 
 @Category({MiscTests.class, SmallTests.class})
 public class TestHttpServer extends HttpServerFunctionalTest {
@@ -564,7 +565,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
       // not bound, ephemeral should return requested port (0 for ephemeral)
       List<?> listeners = (List<?>) Whitebox.getInternalState(server,
           "listeners");
-      Connector listener = (Connector) Whitebox.getInternalState(
+      ServerConnector listener = (ServerConnector) Whitebox.getInternalState(
           listeners.get(0), "listener");
 
       assertEquals(port, listener.getPort());
@@ -623,16 +624,4 @@ public class TestHttpServer extends HttpServerFunctionalTest {
     assertEquals(conn.getHeaderField("Expires"), conn.getHeaderField("Date"));
     assertEquals("DENY", conn.getHeaderField("X-Frame-Options"));
   }
-
-  /**
-   * HTTPServer.Builder should proceed if a external connector is available.
-   */
-  @Test
-  public void testHttpServerBuilderWithExternalConnector() throws Exception {
-    Connector c = mock(Connector.class);
-    doReturn("localhost").when(c).getHost();
-    HttpServer s = new HttpServer.Builder().setName("test").setConnector(c)
-        .build();
-    s.stop();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/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
index 5ddc6d4..cfb5ff3 100644
--- 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
@@ -183,7 +183,7 @@ public class TestServletFilter extends HttpServerFunctionalTest {
       http.start();
       fail("expecting exception");
     } catch (IOException e) {
-      assertTrue( e.getMessage().contains("Problem in starting http server. Server handlers failed"));
+      GenericTestUtils.assertExceptionContains("Unable to initialize WebAppContext", e);
     }
   }
   

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-spark/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-spark/pom.xml b/hbase-spark/pom.xml
index 035dfcc..8417d2f 100644
--- a/hbase-spark/pom.xml
+++ b/hbase-spark/pom.xml
@@ -51,7 +51,6 @@
         <dependency>
             <groupId>javax.servlet</groupId>
             <artifactId>javax.servlet-api</artifactId>
-            <version>3.0.1</version>
             <scope>test</scope>
         </dependency>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-thrift/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-thrift/pom.xml b/hbase-thrift/pom.xml
index f3f20ae..dcc8dc7 100644
--- a/hbase-thrift/pom.xml
+++ b/hbase-thrift/pom.xml
@@ -279,38 +279,16 @@
       <artifactId>slf4j-api</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty-sslengine</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-servlet</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty-util</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>servlet-api-2.5</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jsp-api-2.1</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jsp-2.1</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
-      <groupId>tomcat</groupId>
-      <artifactId>jasper-compiler</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
-      <groupId>tomcat</groupId>
-      <artifactId>jasper-runtime</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-jsp</artifactId>
     </dependency>
   </dependencies>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
index 46ea7f8..169d42f 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
@@ -48,6 +48,7 @@ import javax.security.sasl.SaslServer;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionGroup;
+import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -80,7 +81,6 @@ import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.ParseFilter;
 import org.apache.hadoop.hbase.filter.PrefixFilter;
 import org.apache.hadoop.hbase.filter.WhileMatchFilter;
-import org.apache.hadoop.hbase.jetty.SslSelectChannelConnectorSecure;
 import org.apache.hadoop.hbase.security.SecurityUtil;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.thrift.CallQueue.Call;
@@ -123,16 +123,17 @@ import org.apache.thrift.transport.TSaslServerTransport;
 import org.apache.thrift.transport.TServerSocket;
 import org.apache.thrift.transport.TServerTransport;
 import org.apache.thrift.transport.TTransportFactory;
-import org.mortbay.jetty.Connector;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.nio.SelectChannelConnector;
-import org.mortbay.jetty.servlet.Context;
-import org.mortbay.jetty.servlet.ServletHolder;
-import org.mortbay.thread.QueuedThreadPool;
+
+import org.eclipse.jetty.http.HttpVersion;
+import org.eclipse.jetty.server.*;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Throwables;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.eclipse.jetty.util.thread.QueuedThreadPool;
 
 /**
  * ThriftServerRunner - this class starts up a Thrift server which implements
@@ -143,6 +144,8 @@ public class ThriftServerRunner implements Runnable {
 
   private static final Log LOG = LogFactory.getLog(ThriftServerRunner.class);
 
+  private static final int DEFAULT_HTTP_MAX_HEADER_SIZE = 64 * 1024; // 64k
+
   static final String SERVER_TYPE_CONF_KEY =
       "hbase.regionserver.thrift.server.type";
 
@@ -160,6 +163,10 @@ public class ThriftServerRunner implements Runnable {
   static final String THRIFT_SSL_KEYSTORE_STORE = "hbase.thrift.ssl.keystore.store";
   static final String THRIFT_SSL_KEYSTORE_PASSWORD = "hbase.thrift.ssl.keystore.password";
   static final String THRIFT_SSL_KEYSTORE_KEYPASSWORD = "hbase.thrift.ssl.keystore.keypassword";
+  static final String THRIFT_SSL_EXCLUDE_CIPHER_SUITES = "hbase.thrift.ssl.exclude.cipher.suites";
+  static final String THRIFT_SSL_INCLUDE_CIPHER_SUITES = "hbase.thrift.ssl.include.cipher.suites";
+  static final String THRIFT_SSL_EXCLUDE_PROTOCOLS = "hbase.thrift.ssl.exclude.protocols";
+  static final String THRIFT_SSL_INCLUDE_PROTOCOLS = "hbase.thrift.ssl.include.protocols";
 
   /**
    * Amount of time in milliseconds before a server thread will timeout
@@ -393,53 +400,85 @@ public class ThriftServerRunner implements Runnable {
     TServlet thriftHttpServlet = new ThriftHttpServlet(processor, protocolFactory, realUser,
         conf, hbaseHandler, securityEnabled, doAsEnabled);
 
-    httpServer = new Server();
+    // Set the default max thread number to 100 to limit
+    // the number of concurrent requests so that Thrfit HTTP server doesn't OOM easily.
+    // Jetty set the default max thread number to 250, if we don't set it.
+    //
+    // Our default min thread number 2 is the same as that used by Jetty.
+    int minThreads = conf.getInt(HTTP_MIN_THREADS, 2);
+    int maxThreads = conf.getInt(HTTP_MAX_THREADS, 100);
+    QueuedThreadPool threadPool = new QueuedThreadPool(maxThreads);
+    threadPool.setMinThreads(minThreads);
+    httpServer = new Server(threadPool);
+
     // Context handler
-    Context context = new Context(httpServer, "/", Context.SESSIONS);
-    context.setContextPath("/");
-    String httpPath = "/*";
-    httpServer.setHandler(context);
-    context.addServlet(new ServletHolder(thriftHttpServlet), httpPath);
+    ServletContextHandler ctxHandler = new ServletContextHandler(httpServer, "/", ServletContextHandler.SESSIONS);
+    ctxHandler.addServlet(new ServletHolder(thriftHttpServlet), "/*");
 
     // set up Jetty and run the embedded server
-    Connector connector = new SelectChannelConnector();
+    HttpConfiguration httpConfig = new HttpConfiguration();
+    httpConfig.setSecureScheme("https");
+    httpConfig.setSecurePort(listenPort);
+    httpConfig.setHeaderCacheSize(DEFAULT_HTTP_MAX_HEADER_SIZE);
+    httpConfig.setRequestHeaderSize(DEFAULT_HTTP_MAX_HEADER_SIZE);
+    httpConfig.setResponseHeaderSize(DEFAULT_HTTP_MAX_HEADER_SIZE);
+    httpConfig.setSendServerVersion(false);
+    httpConfig.setSendDateHeader(false);
+
+    ServerConnector serverConnector;
     if(conf.getBoolean(THRIFT_SSL_ENABLED, false)) {
-      SslSelectChannelConnectorSecure sslConnector = new SslSelectChannelConnectorSecure();
+      HttpConfiguration httpsConfig = new HttpConfiguration(httpConfig);
+      httpsConfig.addCustomizer(new SecureRequestCustomizer());
+
+      SslContextFactory sslCtxFactory = new SslContextFactory();
       String keystore = conf.get(THRIFT_SSL_KEYSTORE_STORE);
       String password = HBaseConfiguration.getPassword(conf,
           THRIFT_SSL_KEYSTORE_PASSWORD, null);
       String keyPassword = HBaseConfiguration.getPassword(conf,
           THRIFT_SSL_KEYSTORE_KEYPASSWORD, password);
-      sslConnector.setKeystore(keystore);
-      sslConnector.setPassword(password);
-      sslConnector.setKeyPassword(keyPassword);
-      connector = sslConnector;
+      sslCtxFactory.setKeyStorePath(keystore);
+      sslCtxFactory.setKeyStorePassword(password);
+      sslCtxFactory.setKeyManagerPassword(keyPassword);
+
+      String[] excludeCiphers = conf.getStrings(
+          THRIFT_SSL_EXCLUDE_CIPHER_SUITES, ArrayUtils.EMPTY_STRING_ARRAY);
+      if (excludeCiphers.length != 0) {
+        sslCtxFactory.setExcludeCipherSuites(excludeCiphers);
+      }
+      String[] includeCiphers = conf.getStrings(
+          THRIFT_SSL_INCLUDE_CIPHER_SUITES, ArrayUtils.EMPTY_STRING_ARRAY);
+      if (includeCiphers.length != 0) {
+        sslCtxFactory.setIncludeCipherSuites(includeCiphers);
+      }
+
+      // Disable SSLv3 by default due to "Poodle" Vulnerability - CVE-2014-3566
+      String[] excludeProtocols = conf.getStrings(
+          THRIFT_SSL_EXCLUDE_PROTOCOLS, "SSLv3");
+      if (excludeProtocols.length != 0) {
+        sslCtxFactory.setExcludeProtocols(excludeProtocols);
+      }
+      String[] includeProtocols = conf.getStrings(
+          THRIFT_SSL_INCLUDE_PROTOCOLS, ArrayUtils.EMPTY_STRING_ARRAY);
+      if (includeProtocols.length != 0) {
+        sslCtxFactory.setIncludeProtocols(includeProtocols);
+      }
+
+      serverConnector = new ServerConnector(httpServer,
+          new SslConnectionFactory(sslCtxFactory, HttpVersion.HTTP_1_1.toString()),
+          new HttpConnectionFactory(httpsConfig));
+    } else {
+      serverConnector = new ServerConnector(httpServer, new HttpConnectionFactory(httpConfig));
     }
+    serverConnector.setPort(listenPort);
     String host = getBindAddress(conf).getHostAddress();
-    connector.setPort(listenPort);
-    connector.setHost(host);
-    connector.setHeaderBufferSize(1024 * 64);
-    httpServer.addConnector(connector);
+    serverConnector.setHost(host);
+    httpServer.addConnector(serverConnector);
+    httpServer.setStopAtShutdown(true);
 
     if (doAsEnabled) {
       ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
     }
 
-    // Set the default max thread number to 100 to limit
-    // the number of concurrent requests so that Thrfit HTTP server doesn't OOM easily.
-    // Jetty set the default max thread number to 250, if we don't set it.
-    //
-    // Our default min thread number 2 is the same as that used by Jetty.
-    int minThreads = conf.getInt(HTTP_MIN_THREADS, 2);
-    int maxThreads = conf.getInt(HTTP_MAX_THREADS, 100);
-    QueuedThreadPool threadPool = new QueuedThreadPool(maxThreads);
-    threadPool.setMinThreads(minThreads);
-    httpServer.setThreadPool(threadPool);
-
-    httpServer.setSendServerVersion(false);
-    httpServer.setSendDateHeader(false);
-    httpServer.setStopAtShutdown(true);
-
     LOG.info("Starting Thrift HTTP Server on " + Integer.toString(listenPort));
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 637b236..6d837d6 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1235,12 +1235,16 @@
     <httpcore.version>4.4.4</httpcore.version>
     <metrics-core.version>3.1.2</metrics-core.version>
     <guava.version>12.0.1</guava.version>
-    <jackson.version>1.9.13</jackson.version>
+    <jackson.version>2.23.2</jackson.version>
     <jasper.version>5.5.23</jasper.version>
     <jaxb-api.version>2.2.2</jaxb-api.version>
-    <jetty.version>6.1.26</jetty.version>
+    <jetty.version>9.3.8.v20160314</jetty.version>
+    <jetty-jsp.version>9.2.19.v20160908</jetty-jsp.version>
+    <jackson1.version>1.9.13</jackson1.version>
+    <servlet.api.version>3.1.0</servlet.api.version>
+    <jersey.version>2.22.2</jersey.version>
+    <jetty6.version>6.1.26</jetty6.version>
     <jetty.jspapi.version>6.1.14</jetty.jspapi.version>
-    <jersey.version>1.9</jersey.version>
     <jruby.version>1.6.8</jruby.version>
     <junit.version>4.12</junit.version>
     <hamcrest.version>1.3</hamcrest.version>
@@ -1660,17 +1664,6 @@
         <version>${jruby.version}</version>
       </dependency>
       <dependency>
-        <groupId>org.mortbay.jetty</groupId>
-        <artifactId>jetty</artifactId>
-        <version>${jetty.version}</version>
-        <exclusions>
-          <exclusion>
-            <groupId>org.mortbay.jetty</groupId>
-            <artifactId>servlet-api</artifactId>
-          </exclusion>
-        </exclusions>
-      </dependency>
-      <dependency>
         <groupId>org.jruby.jcodings</groupId>
         <artifactId>jcodings</artifactId>
         <version>${jcodings.version}</version>
@@ -1680,63 +1673,28 @@
         <artifactId>joni</artifactId>
         <version>${joni.version}</version>
       </dependency>
-      <dependency>
-        <groupId>org.mortbay.jetty</groupId>
-        <artifactId>jetty-util</artifactId>
-        <version>${jetty.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.mortbay.jetty</groupId>
-        <artifactId>jetty-sslengine</artifactId>
-        <version>${jetty.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.mortbay.jetty</groupId>
-        <artifactId>jsp-2.1</artifactId>
-        <version>${jetty.jspapi.version}</version>
-        <exclusions>
-          <exclusion>
-            <groupId>org.eclipse.jdt</groupId>
-            <artifactId>core</artifactId>
-          </exclusion>
-          <exclusion>
-            <groupId>ant</groupId>
-            <artifactId>ant</artifactId>
-          </exclusion>
-        </exclusions>
-      </dependency>
-      <dependency>
-        <groupId>org.mortbay.jetty</groupId>
-        <artifactId>jsp-api-2.1</artifactId>
-        <version>${jetty.jspapi.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.mortbay.jetty</groupId>
-        <artifactId>servlet-api-2.5</artifactId>
-        <version>${jetty.jspapi.version}</version>
-      </dependency>
       <!-- While jackson is also a dependency of jersey it
            can bring in jars from different, incompatible versions. We force
            the same version with these dependencies -->
       <dependency>
         <groupId>org.codehaus.jackson</groupId>
         <artifactId>jackson-core-asl</artifactId>
-        <version>${jackson.version}</version>
+        <version>${jackson1.version}</version>
       </dependency>
       <dependency>
         <groupId>org.codehaus.jackson</groupId>
         <artifactId>jackson-mapper-asl</artifactId>
-        <version>${jackson.version}</version>
+        <version>${jackson1.version}</version>
       </dependency>
       <dependency>
         <groupId>org.codehaus.jackson</groupId>
         <artifactId>jackson-jaxrs</artifactId>
-        <version>${jackson.version}</version>
+        <version>${jackson1.version}</version>
       </dependency>
       <dependency>
         <groupId>org.codehaus.jackson</groupId>
         <artifactId>jackson-xc</artifactId>
-        <version>${jackson.version}</version>
+        <version>${jackson1.version}</version>
       </dependency>
       <dependency>
         <!--If this is not in the runtime lib, we get odd
@@ -1783,32 +1741,82 @@
       </dependency>
       <!-- REST dependencies -->
       <dependency>
-        <groupId>com.google.protobuf</groupId>
-        <artifactId>protobuf-java</artifactId>
-        <version>${external.protobuf.version}</version>
+        <groupId>javax.servlet</groupId>
+        <artifactId>javax.servlet-api</artifactId>
+        <version>${servlet.api.version}</version>
       </dependency>
       <dependency>
-        <groupId>com.sun.jersey</groupId>
-        <artifactId>jersey-core</artifactId>
-        <version>${jersey.version}</version>
+        <groupId>org.eclipse.jetty</groupId>
+        <artifactId>jetty-server</artifactId>
+        <version>${jetty.version}</version>
       </dependency>
       <dependency>
-        <groupId>com.sun.jersey</groupId>
-        <artifactId>jersey-json</artifactId>
-        <version>${jersey.version}</version>
+        <groupId>org.eclipse.jetty</groupId>
+        <artifactId>jetty-servlet</artifactId>
+        <version>${jetty.version}</version>
         <exclusions>
           <exclusion>
-            <groupId>stax</groupId>
-            <artifactId>stax-api</artifactId>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>servlet-api</artifactId>
           </exclusion>
         </exclusions>
       </dependency>
       <dependency>
-        <groupId>com.sun.jersey</groupId>
-        <artifactId>jersey-server</artifactId>
+        <groupId>org.eclipse.jetty</groupId>
+        <artifactId>jetty-security</artifactId>
+        <version>${jetty.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.eclipse.jetty</groupId>
+        <artifactId>jetty-http</artifactId>
+        <version>${jetty.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.eclipse.jetty</groupId>
+        <artifactId>jetty-util</artifactId>
+        <version>${jetty.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.eclipse.jetty</groupId>
+        <artifactId>jetty-io</artifactId>
+        <version>${jetty.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.eclipse.jetty</groupId>
+        <artifactId>jetty-jsp</artifactId>
+        <version>${jetty-jsp.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.eclipse.jetty</groupId>
+        <artifactId>jetty-jmx</artifactId>
+        <version>${jetty.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.eclipse.jetty</groupId>
+        <artifactId>jetty-webapp</artifactId>
+        <version>${jetty.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.eclipse.jetty</groupId>
+        <artifactId>jetty-util-ajax</artifactId>
+        <version>${jetty.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>com.google.protobuf</groupId>
+        <artifactId>protobuf-java</artifactId>
+        <version>${external.protobuf.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.glassfish.jersey.containers</groupId>
+        <artifactId>jersey-container-servlet-core</artifactId>
         <version>${jersey.version}</version>
       </dependency>
       <dependency>
+        <groupId>org.glassfish.jersey.media</groupId>
+        <artifactId>jersey-media-json-jackson1</artifactId>
+        <version>${jackson.version}</version>
+      </dependency>
+      <dependency>
         <groupId>javax.xml.bind</groupId>
         <artifactId>jaxb-api</artifactId>
         <version>${jaxb-api.version}</version>
@@ -2162,6 +2170,10 @@
                 <groupId>io.netty</groupId>
                 <artifactId>netty</artifactId>
               </exclusion>
+              <exclusion>
+                <groupId>javax.servlet</groupId>
+                <artifactId>servlet-api</artifactId>
+              </exclusion>
             </exclusions>
           </dependency>
           <dependency>
@@ -2173,6 +2185,10 @@
                 <groupId>io.netty</groupId>
                 <artifactId>netty</artifactId>
               </exclusion>
+              <exclusion>
+                <groupId>javax.servlet</groupId>
+                <artifactId>servlet-api</artifactId>
+              </exclusion>
             </exclusions>
           </dependency>
           <dependency>
@@ -2186,6 +2202,10 @@
                 <groupId>io.netty</groupId>
                 <artifactId>netty</artifactId>
               </exclusion>
+              <exclusion>
+                <groupId>javax.servlet</groupId>
+                <artifactId>servlet-api</artifactId>
+              </exclusion>
             </exclusions>
           </dependency>
           <dependency>


[2/2] hbase git commit: Revert "Revert "HBASE-12894 Upgrade to Jetty 9 for REST / Info Server / Thrift Http Server""

Posted by st...@apache.org.
Revert "Revert "HBASE-12894 Upgrade to Jetty 9 for REST / Info Server / Thrift Http Server""

This reverts commit 0ac5d4a7171833d33b35c09a3f5f3ca3ee5cd4d1.
This is a revert of a revert; i.e. a reapplication!
Fixes for breakage that comes in with this patch is in a follow-on.


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

Branch: refs/heads/master
Commit: ffe7dac53d7824b4fb97b2aa642108a58b23afc8
Parents: f576f46
Author: Michael Stack <st...@apache.org>
Authored: Mon Jan 30 11:54:54 2017 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Mon Jan 30 11:54:54 2017 -0800

----------------------------------------------------------------------
 hbase-common/pom.xml                            |   4 -
 hbase-it/pom.xml                                |   5 -
 .../src/main/resources/META-INF/LICENSE.vm      |   2 +-
 .../src/main/resources/META-INF/NOTICE.vm       |  14 +-
 .../src/main/resources/supplemental-models.xml  | 468 +++++++++++++++++++
 hbase-rest/pom.xml                              |  68 +--
 .../org/apache/hadoop/hbase/rest/Constants.java |  10 +
 .../apache/hadoop/hbase/rest/RESTServer.java    | 197 +++++---
 .../hadoop/hbase/rest/RESTServletContainer.java |   2 +-
 .../hbase/rest/filter/GZIPRequestStream.java    |  16 +
 .../hbase/rest/filter/GZIPResponseStream.java   |  11 +
 .../hadoop/hbase/rest/model/ScannerModel.java   |  45 +-
 .../rest/provider/JAXBContextResolver.java      |   6 +-
 .../hbase/rest/provider/JacksonProvider.java    |  31 --
 .../producer/PlainTextMessageBodyProducer.java  |  11 +-
 .../producer/ProtobufMessageBodyProducer.java   |  17 +-
 .../hbase/rest/HBaseRESTTestingUtility.java     |  66 ++-
 .../hadoop/hbase/rest/RowResourceBase.java      |   6 +-
 .../hadoop/hbase/rest/TestGzipFilter.java       |  15 -
 .../hadoop/hbase/rest/TestMultiRowResource.java |   6 +-
 .../rest/TestNamespacesInstanceResource.java    |   8 +-
 .../hbase/rest/TestScannersWithLabels.java      |   2 +-
 .../apache/hadoop/hbase/rest/TestTableScan.java |   8 +-
 hbase-server/pom.xml                            |  76 ++-
 .../hadoop/hbase/http/HttpRequestLog.java       |   5 +-
 .../apache/hadoop/hbase/http/HttpServer.java    | 308 +++++-------
 .../jetty/SslSelectChannelConnectorSecure.java  |  36 --
 .../hadoop/hbase/util/HttpServerUtil.java       |  20 +-
 .../hadoop/hbase/http/TestHttpRequestLog.java   |   5 +-
 .../hadoop/hbase/http/TestHttpServer.java       |  19 +-
 .../hadoop/hbase/http/TestServletFilter.java    |   2 +-
 hbase-spark/pom.xml                             |   1 -
 hbase-thrift/pom.xml                            |  34 +-
 .../hadoop/hbase/thrift/ThriftServerRunner.java | 115 +++--
 pom.xml                                         | 152 +++---
 35 files changed, 1087 insertions(+), 704 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-common/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml
index b46be6d..cf2772e 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -253,10 +253,6 @@
       <groupId>com.google.protobuf</groupId>
       <artifactId>protobuf-java</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty-util</artifactId>
-    </dependency>
     <!-- tracing Dependencies -->
     <dependency>
       <groupId>org.apache.htrace</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-it/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-it/pom.xml b/hbase-it/pom.xml
index 09118b1..55240a9 100644
--- a/hbase-it/pom.xml
+++ b/hbase-it/pom.xml
@@ -232,11 +232,6 @@
       <artifactId>guava</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.sun.jersey</groupId>
-      <artifactId>jersey-client</artifactId>
-      <version>${jersey.version}</version>
-    </dependency>
-    <dependency>
       <groupId>io.dropwizard.metrics</groupId>
       <artifactId>metrics-core</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
----------------------------------------------------------------------
diff --git a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm b/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
index 688310f..a9f0c81 100644
--- a/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
+++ b/hbase-resource-bundle/src/main/resources/META-INF/LICENSE.vm
@@ -1563,7 +1563,7 @@ You can redistribute it and/or modify it under either the terms of the GPL
 ## license mentions.
 ##
 ## See this FAQ link for justifications: https://www.apache.org/legal/resolved.html
-#set($non_aggregate_fine = [ 'Public Domain', 'New BSD license', 'BSD license', 'Mozilla Public License Version 2.0', 'Creative Commons Attribution License, Version 2.5', 'MPL 1.1'])
+#set($non_aggregate_fine = [ 'Public Domain', 'New BSD license', 'BSD license', 'Mozilla Public License Version 1.1', 'Mozilla Public License Version 2.0', 'Creative Commons Attribution License, Version 2.5', 'Eclipse Public License 1.0'])
 ## include LICENSE sections for anything not under ASL2.0
 #foreach( ${dep} in ${projects} )
 ## if there are no licenses we'll fail the build later, so

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-resource-bundle/src/main/resources/META-INF/NOTICE.vm
----------------------------------------------------------------------
diff --git a/hbase-resource-bundle/src/main/resources/META-INF/NOTICE.vm b/hbase-resource-bundle/src/main/resources/META-INF/NOTICE.vm
index 53fb51e..4be6017 100644
--- a/hbase-resource-bundle/src/main/resources/META-INF/NOTICE.vm
+++ b/hbase-resource-bundle/src/main/resources/META-INF/NOTICE.vm
@@ -101,21 +101,13 @@ noted in the LICENSE file.
 ## Leave out all the 'optionally depends on' bits.
 
 #end
-## NOTICE.txt from jetty 6.1.26, which isn't in the org.mortbay.jetty:jetty-util jar
-## Obtained from fedora's mirror of the 6.1.26 source
-## http://pkgs.fedoraproject.org/repo/pkgs/jetty/jetty-6.1.26-src.zip/
-##     4980757517064efbf655e025d005a0b8/jetty-6.1.26-src.zip
-## https://github.com/toddlipcon/jetty-hadoop-fix/tree/6.1.26
-##
-## updated for server side to include jsp bits from 6.1.14 (notice files are identical save
-## copyright span)
+
 #macro(jetty_notice $jetty-include-unixcrypt)
 --
 This product includes portions of 'The Jetty Web Container'
 
-Copyright 1995-2009 Mort Bay Consulting Pty Ltd
+Copyright 1995-2016 Mort Bay Consulting Pty Ltd.
 
-## javax.servlet paragraph is generated via org.mortbay.jetty:servlet-api-2.5
 #if(${jetty-include-unixcrypt})
 ## UnixCrypt.java paragraph only in server
 The UnixCrypt.java code ~Implements the one way cryptography used by
@@ -126,7 +118,7 @@ for non-commercial or commercial purposes and without fee is
 granted provided that the copyright notice appears in all copies.
 
 #end
-## jsp implementation paragraph is handled via org.mortbay.jetty:jsp-2.1
+## jsp implementation paragraph is handled via org.eclipse.jetty:jetty-jsp:jar
 
 Some portions of the code are Copyright:
   2006 Tim Vernum

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-resource-bundle/src/main/resources/supplemental-models.xml
----------------------------------------------------------------------
diff --git a/hbase-resource-bundle/src/main/resources/supplemental-models.xml b/hbase-resource-bundle/src/main/resources/supplemental-models.xml
index e1de3c7..ab99443 100644
--- a/hbase-resource-bundle/src/main/resources/supplemental-models.xml
+++ b/hbase-resource-bundle/src/main/resources/supplemental-models.xml
@@ -152,6 +152,214 @@ under the License.
   </supplement>
   <supplement>
     <project>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-http</artifactId>
+
+      <licenses>
+        <license>
+          <name>Apache License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-io</artifactId>
+
+      <licenses>
+        <license>
+          <name>Apache License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-jmx</artifactId>
+
+      <licenses>
+        <license>
+          <name>Apache License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
+
+      <licenses>
+        <license>
+          <name>Apache License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-servlet</artifactId>
+
+      <licenses>
+        <license>
+          <name>Apache License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-util</artifactId>
+
+      <licenses>
+        <license>
+          <name>Apache License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-util-ajax</artifactId>
+
+      <licenses>
+        <license>
+          <name>Apache License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-webapp</artifactId>
+
+      <licenses>
+        <license>
+          <name>Apache License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-xml</artifactId>
+
+      <licenses>
+        <license>
+          <name>Apache License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-jsp</artifactId>
+
+      <licenses>
+        <license>
+          <name>Apache License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-security</artifactId>
+
+      <licenses>
+        <license>
+          <name>Apache License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.eclipse.jetty.orbit</groupId>
+      <artifactId>javax.servlet.jsp.jstl</artifactId>
+
+      <licenses>
+        <license>
+          <name>Apache License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.eclipse.jetty.orbit</groupId>
+      <artifactId>org.eclipse.jdt.core</artifactId>
+
+      <licenses>
+        <license>
+          <name>Eclipse Public License 1.0</name>
+          <url>http://www.eclipse.org/legal/epl-v10.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.eclipse.jetty.toolchain</groupId>
+      <artifactId>jetty-schemas</artifactId>
+      <licenses>
+        <license>
+          <name>Common Development and Distribution License (CDDL) v1.0</name>
+          <url>https://glassfish.dev.java.net/public/CDDLv1.0.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.javassist</groupId>
+      <artifactId>javassist</artifactId>
+      <licenses>
+        <license>
+          <name>Apache License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
       <groupId>com.lmax</groupId>
       <artifactId>disruptor</artifactId>
 
@@ -589,6 +797,266 @@ under the License.
   </supplement>
   <supplement>
     <project>
+      <groupId>javax.annotation</groupId>
+      <artifactId>javax.annotation-api</artifactId>
+      <licenses>
+        <license>
+          <name>Common Development and Distribution License (CDDL) v1.0</name>
+          <url>https://glassfish.dev.java.net/public/CDDLv1.0.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>javax.servlet</groupId>
+      <artifactId>javax.servlet-api</artifactId>
+      <licenses>
+        <license>
+          <name>Common Development and Distribution License (CDDL) v1.0</name>
+          <url>https://glassfish.dev.java.net/public/CDDLv1.0.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>javax.servlet.jsp</groupId>
+      <artifactId>javax.servlet.jsp-api</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.glassfish</groupId>
+      <artifactId>javax.el</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.glassfish.hk2</groupId>
+      <artifactId>hk2-api</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.glassfish.hk2</groupId>
+      <artifactId>hk2-locator</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.glassfish.hk2</groupId>
+      <artifactId>hk2-utils</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.glassfish.hk2</groupId>
+      <artifactId>osgi-resource-locator</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.glassfish.hk2.external</groupId>
+      <artifactId>aopalliance-repackaged</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.glassfish.hk2.external</groupId>
+      <artifactId>javax.inject</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.glassfish.jersey.bundles.repackaged</groupId>
+      <artifactId>jersey-guava</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.glassfish.jersey.containers</groupId>
+      <artifactId>jersey-container-servlet-core</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.glassfish.jersey.core</groupId>
+      <artifactId>jersey-client</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.glassfish.jersey.core</groupId>
+      <artifactId>jersey-common</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.glassfish.jersey.core</groupId>
+      <artifactId>jersey-server</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.glassfish.jersey.media</groupId>
+      <artifactId>jersey-media-jaxb</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.glassfish.jersey.media</groupId>
+      <artifactId>jersey-media-json-jackson1</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.glassfish.web</groupId>
+      <artifactId>javax.servlet.jsp</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>org.glassfish.web</groupId>
+      <artifactId>javax.servlet.jsp.jstl</artifactId>
+      <licenses>
+        <license>
+          <name>CDDL 1.1</name>
+          <url>https://glassfish.java.net/public/CDDL+GPL_1_1.html</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
+      <groupId>javax.validation</groupId>
+      <artifactId>validation-api</artifactId>
+      <licenses>
+        <license>
+          <name>Apache License, Version 2.0</name>
+          <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+          <distribution>repo</distribution>
+        </license>
+      </licenses>
+    </project>
+  </supplement>
+  <supplement>
+    <project>
       <groupId>org.mortbay.jetty</groupId>
       <artifactId>jetty-util</artifactId>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index 53fbad4..18e9da5 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -235,20 +235,6 @@
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
-    <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jsp-2.1</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
-      <groupId>tomcat</groupId>
-      <artifactId>jasper-compiler</artifactId>
-      <scope>compile</scope>
-    </dependency>
-    <dependency>
-      <groupId>tomcat</groupId>
-      <artifactId>jasper-runtime</artifactId>
-    </dependency>
     <!-- REST dependencies -->
     <dependency>
       <groupId>com.google.guava</groupId>
@@ -259,18 +245,6 @@
       <artifactId>protobuf-java</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.sun.jersey</groupId>
-      <artifactId>jersey-core</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.sun.jersey</groupId>
-      <artifactId>jersey-json</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.sun.jersey</groupId>
-      <artifactId>jersey-server</artifactId>
-    </dependency>
-    <dependency>
       <groupId>commons-cli</groupId>
       <artifactId>commons-cli</artifactId>
     </dependency>
@@ -295,38 +269,44 @@
       <artifactId>jaxb-api</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.codehaus.jackson</groupId>
-      <artifactId>jackson-core-asl</artifactId>
+      <groupId>javax.servlet</groupId>
+      <artifactId>javax.servlet-api</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.codehaus.jackson</groupId>
-      <artifactId>jackson-jaxrs</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-servlet</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty-sslengine</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-util</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty-util</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-http</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jsp-api-2.1</artifactId>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-io</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>servlet-api-2.5</artifactId>
+    <groupId>org.eclipse.jetty</groupId>
+    <artifactId>jetty-jmx</artifactId>
     </dependency>
-    <!-- test dependencies -->
     <dependency>
-      <groupId>org.codehaus.jackson</groupId>
-      <artifactId>jackson-mapper-asl</artifactId>
-      <scope>test</scope>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-jsp</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.containers</groupId>
+      <artifactId>jersey-container-servlet-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.media</groupId>
+      <artifactId>jersey-media-json-jackson1</artifactId>
     </dependency>
   </dependencies>
   <profiles>

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/Constants.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/Constants.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/Constants.java
index 54ce117..ea8bc9c 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/Constants.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/Constants.java
@@ -54,6 +54,16 @@ public interface Constants {
   String REST_SSL_KEYSTORE_STORE = "hbase.rest.ssl.keystore.store";
   String REST_SSL_KEYSTORE_PASSWORD = "hbase.rest.ssl.keystore.password";
   String REST_SSL_KEYSTORE_KEYPASSWORD = "hbase.rest.ssl.keystore.keypassword";
+  String REST_SSL_EXCLUDE_CIPHER_SUITES = "hbase.rest.ssl.exclude.cipher.suites";
+  String REST_SSL_INCLUDE_CIPHER_SUITES = "hbase.rest.ssl.include.cipher.suites";
+  String REST_SSL_EXCLUDE_PROTOCOLS = "hbase.rest.ssl.exclude.protocols";
+  String REST_SSL_INCLUDE_PROTOCOLS = "hbase.rest.ssl.include.protocols";
+
+  String REST_THREAD_POOL_THREADS_MAX = "hbase.rest.threads.max";
+  String REST_THREAD_POOL_THREADS_MIN = "hbase.rest.threads.min";
+  String REST_THREAD_POOL_TASK_QUEUE_SIZE = "hbase.rest.task.queue.size";
+  String REST_THREAD_POOL_THREAD_IDLE_TIMEOUT = "hbase.rest.thread.idle.timeout";
+  String REST_CONNECTOR_ACCEPT_QUEUE_SIZE = "hbase.rest.connector.accept.queue.size";
 
   String REST_DNS_NAMESERVER = "hbase.rest.dns.nameserver";
   String REST_DNS_INTERFACE = "hbase.rest.dns.interface";

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/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 7cec152..4ab194c 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
@@ -18,12 +18,14 @@
 
 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.Map.Entry;
 import java.util.Set;
+import java.util.EnumSet;
+import java.util.concurrent.ArrayBlockingQueue;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.HelpFormatter;
@@ -37,9 +39,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.http.HttpServer;
 import org.apache.hadoop.hbase.http.InfoServer;
-import org.apache.hadoop.hbase.jetty.SslSelectChannelConnectorSecure;
 import org.apache.hadoop.hbase.rest.filter.AuthFilter;
 import org.apache.hadoop.hbase.rest.filter.RestCsrfPreventionFilter;
 import org.apache.hadoop.hbase.security.UserProvider;
@@ -49,17 +49,28 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Strings;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.util.StringUtils;
-import org.mortbay.jetty.Connector;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.nio.SelectChannelConnector;
-import org.mortbay.jetty.servlet.Context;
-import org.mortbay.jetty.servlet.FilterHolder;
-import org.mortbay.jetty.servlet.ServletHolder;
-import org.mortbay.thread.QueuedThreadPool;
 
 import com.google.common.base.Preconditions;
-import com.sun.jersey.api.json.JSONConfiguration;
-import com.sun.jersey.spi.container.servlet.ServletContainer;
+
+import org.eclipse.jetty.http.HttpVersion;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.HttpConnectionFactory;
+import org.eclipse.jetty.server.SslConnectionFactory;
+import org.eclipse.jetty.server.HttpConfiguration;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.server.SecureRequestCustomizer;
+import org.eclipse.jetty.util.ssl.SslContextFactory;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.util.thread.QueuedThreadPool;
+import org.eclipse.jetty.jmx.MBeanContainer;
+import org.eclipse.jetty.servlet.FilterHolder;
+
+import org.glassfish.jersey.jackson1.Jackson1Feature;
+import org.glassfish.jersey.server.ResourceConfig;
+import org.glassfish.jersey.servlet.ServletContainer;
+
+import javax.servlet.DispatcherType;
 
 /**
  * Main class for launching REST gateway as a servlet hosted by Jetty.
@@ -82,6 +93,8 @@ public class RESTServer implements Constants {
   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";
 
+  private static final String PATH_SPEC_ANY = "/*";
+
   private static void printUsageAndExit(Options options, int exitCode) {
     HelpFormatter formatter = new HelpFormatter();
     formatter.printHelp("hbase rest start", "", options,
@@ -109,17 +122,19 @@ public class RESTServer implements Constants {
   }
 
   static String REST_CSRF_BROWSER_USERAGENTS_REGEX_KEY = "hbase.rest-csrf.browser-useragents-regex";
-  static void addCSRFFilter(Context context, Configuration conf) {
+  static void addCSRFFilter(ServletContextHandler ctxHandler, Configuration conf) {
     restCSRFEnabled = conf.getBoolean(REST_CSRF_ENABLED_KEY, REST_CSRF_ENABLED_DEFAULT);
     if (restCSRFEnabled) {
-      String[] urls = { "/*" };
       Set<String> restCsrfMethodsToIgnore = new HashSet<>();
       restCsrfMethodsToIgnore.addAll(getTrimmedStringList(conf,
-        REST_CSRF_METHODS_TO_IGNORE_KEY, REST_CSRF_METHODS_TO_IGNORE_DEFAULT));
+          REST_CSRF_METHODS_TO_IGNORE_KEY, REST_CSRF_METHODS_TO_IGNORE_DEFAULT));
       Map<String, String> restCsrfParams = RestCsrfPreventionFilter
           .getFilterParams(conf, "hbase.rest-csrf.");
-      HttpServer.defineFilter(context, "csrf", RestCsrfPreventionFilter.class.getName(),
-        restCsrfParams, urls);
+      FilterHolder holder = new FilterHolder();
+      holder.setName("csrf");
+      holder.setClassName(RestCsrfPreventionFilter.class.getName());
+      holder.setInitParameters(restCsrfParams);
+      ctxHandler.addFilter(holder, PATH_SPEC_ANY, EnumSet.allOf(DispatcherType.class));
     }
   }
 
@@ -143,10 +158,10 @@ public class RESTServer implements Constants {
         FilterHolder authFilter = new FilterHolder();
         authFilter.setClassName(AuthFilter.class.getName());
         authFilter.setName("AuthenticationFilter");
-        return new Pair<FilterHolder, Class<? extends ServletContainer>>(authFilter,containerClass);
+        return new Pair<>(authFilter,containerClass);
       }
     }
-    return new Pair<FilterHolder, Class<? extends ServletContainer>>(null, containerClass);
+    return new Pair<>(null, containerClass);
   }
 
   private static void parseCommandLine(String[] args, RESTServlet servlet) {
@@ -220,76 +235,106 @@ public class RESTServer implements Constants {
     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
-    ServletHolder sh = new ServletHolder(containerClass);
-    sh.setInitParameter(
-      "com.sun.jersey.config.property.resourceConfigClass",
-      ResourceConfig.class.getCanonicalName());
-    sh.setInitParameter("com.sun.jersey.config.property.packages",
-      "jetty");
-    // The servlet holder below is instantiated to only handle the case
-    // of the /status/cluster returning arrays of nodes (live/dead). Without
-    // this servlet holder, the problem is that the node arrays in the response
-    // are collapsed to single nodes. We want to be able to treat the
-    // node lists as POJO in the response to /status/cluster servlet call,
-    // but not change the behavior for any of the other servlets
-    // Hence we don't use the servlet holder for all servlets / paths
-    ServletHolder shPojoMap = new ServletHolder(containerClass);
-    @SuppressWarnings("unchecked")
-    Map<String, String> shInitMap = sh.getInitParameters();
-    for (Entry<String, String> e : shInitMap.entrySet()) {
-      shPojoMap.setInitParameter(e.getKey(), e.getValue());
-    }
-    shPojoMap.setInitParameter(JSONConfiguration.FEATURE_POJO_MAPPING, "true");
+    ResourceConfig application = new ResourceConfig().
+        packages("org.apache.hadoop.hbase.rest").register(Jackson1Feature.class);
+    ServletHolder sh = new ServletHolder(new ServletContainer(application));
+
+    // Set the default max thread number to 100 to limit
+    // the number of concurrent requests so that REST server doesn't OOM easily.
+    // Jetty set the default max thread number to 250, if we don't set it.
+    //
+    // Our default min thread number 2 is the same as that used by Jetty.
+    int maxThreads = servlet.getConfiguration().getInt(REST_THREAD_POOL_THREADS_MAX, 100);
+    int minThreads = servlet.getConfiguration().getInt(REST_THREAD_POOL_THREADS_MIN, 2);
+    // Use the default queue (unbounded with Jetty 9.3) if the queue size is negative, otherwise use
+    // bounded {@link ArrayBlockingQueue} with the given size
+    int queueSize = servlet.getConfiguration().getInt(REST_THREAD_POOL_TASK_QUEUE_SIZE, -1);
+    int idleTimeout = servlet.getConfiguration().getInt(REST_THREAD_POOL_THREAD_IDLE_TIMEOUT, 60000);
+    QueuedThreadPool threadPool = queueSize > 0 ?
+        new QueuedThreadPool(maxThreads, minThreads, idleTimeout, new ArrayBlockingQueue<Runnable>(queueSize)) :
+        new QueuedThreadPool(maxThreads, minThreads, idleTimeout);
 
-    // set up Jetty and run the embedded server
+    Server server = new Server(threadPool);
 
-    Server server = new Server();
+    // Setup JMX
+    MBeanContainer mbContainer=new MBeanContainer(ManagementFactory.getPlatformMBeanServer());
+    server.addEventListener(mbContainer);
+    server.addBean(mbContainer);
 
-    Connector connector = new SelectChannelConnector();
-    if(conf.getBoolean(REST_SSL_ENABLED, false)) {
-      SslSelectChannelConnectorSecure sslConnector = new SslSelectChannelConnectorSecure();
+
+    String host = servlet.getConfiguration().get("hbase.rest.host", "0.0.0.0");
+    int servicePort = servlet.getConfiguration().getInt("hbase.rest.port", 8080);
+    HttpConfiguration httpConfig = new HttpConfiguration();
+    httpConfig.setSecureScheme("https");
+    httpConfig.setSecurePort(servicePort);
+    httpConfig.setSendServerVersion(false);
+    httpConfig.setSendDateHeader(false);
+
+    ServerConnector serverConnector;
+    if (conf.getBoolean(REST_SSL_ENABLED, false)) {
+      HttpConfiguration httpsConfig = new HttpConfiguration(httpConfig);
+      httpsConfig.addCustomizer(new SecureRequestCustomizer());
+
+      SslContextFactory sslCtxFactory = new SslContextFactory();
       String keystore = conf.get(REST_SSL_KEYSTORE_STORE);
       String password = HBaseConfiguration.getPassword(conf,
-        REST_SSL_KEYSTORE_PASSWORD, null);
+          REST_SSL_KEYSTORE_PASSWORD, null);
       String keyPassword = HBaseConfiguration.getPassword(conf,
-        REST_SSL_KEYSTORE_KEYPASSWORD, password);
-      sslConnector.setKeystore(keystore);
-      sslConnector.setPassword(password);
-      sslConnector.setKeyPassword(keyPassword);
-      connector = sslConnector;
+          REST_SSL_KEYSTORE_KEYPASSWORD, password);
+      sslCtxFactory.setKeyStorePath(keystore);
+      sslCtxFactory.setKeyStorePassword(password);
+      sslCtxFactory.setKeyManagerPassword(keyPassword);
+
+      String[] excludeCiphers = servlet.getConfiguration().getStrings(
+          REST_SSL_EXCLUDE_CIPHER_SUITES, ArrayUtils.EMPTY_STRING_ARRAY);
+      if (excludeCiphers.length != 0) {
+        sslCtxFactory.setExcludeCipherSuites(excludeCiphers);
+      }
+      String[] includeCiphers = servlet.getConfiguration().getStrings(
+          REST_SSL_INCLUDE_CIPHER_SUITES, ArrayUtils.EMPTY_STRING_ARRAY);
+      if (includeCiphers.length != 0) {
+        sslCtxFactory.setIncludeCipherSuites(includeCiphers);
+      }
+
+      String[] excludeProtocols = servlet.getConfiguration().getStrings(
+          REST_SSL_EXCLUDE_PROTOCOLS, ArrayUtils.EMPTY_STRING_ARRAY);
+      if (excludeProtocols.length != 0) {
+        sslCtxFactory.setExcludeProtocols(excludeProtocols);
+      }
+      String[] includeProtocols = servlet.getConfiguration().getStrings(
+          REST_SSL_INCLUDE_PROTOCOLS, ArrayUtils.EMPTY_STRING_ARRAY);
+      if (includeProtocols.length != 0) {
+        sslCtxFactory.setIncludeProtocols(includeProtocols);
+      }
+
+      serverConnector = new ServerConnector(server,
+          new SslConnectionFactory(sslCtxFactory, HttpVersion.HTTP_1_1.toString()),
+          new HttpConnectionFactory(httpsConfig));
+    } else {
+      serverConnector = new ServerConnector(server, new HttpConnectionFactory(httpConfig));
     }
-    connector.setPort(servlet.getConfiguration().getInt("hbase.rest.port", DEFAULT_LISTEN_PORT));
-    connector.setHost(servlet.getConfiguration().get("hbase.rest.host", "0.0.0.0"));
-    connector.setHeaderBufferSize(65536);
 
-    server.addConnector(connector);
+    int acceptQueueSize = servlet.getConfiguration().getInt(REST_CONNECTOR_ACCEPT_QUEUE_SIZE, -1);
+    if (acceptQueueSize >= 0) {
+      serverConnector.setAcceptQueueSize(acceptQueueSize);
+    }
 
-    // Set the default max thread number to 100 to limit
-    // the number of concurrent requests so that REST server doesn't OOM easily.
-    // Jetty set the default max thread number to 250, if we don't set it.
-    //
-    // Our default min thread number 2 is the same as that used by Jetty.
-    int maxThreads = servlet.getConfiguration().getInt("hbase.rest.threads.max", 100);
-    int minThreads = servlet.getConfiguration().getInt("hbase.rest.threads.min", 2);
-    QueuedThreadPool threadPool = new QueuedThreadPool(maxThreads);
-    threadPool.setMinThreads(minThreads);
-    server.setThreadPool(threadPool);
-
-    server.setSendServerVersion(false);
-    server.setSendDateHeader(false);
+    serverConnector.setPort(servicePort);
+    serverConnector.setHost(host);
+
+    server.addConnector(serverConnector);
     server.setStopAtShutdown(true);
-      // set up context
-    Context context = new Context(server, "/", Context.SESSIONS);
-    context.addServlet(shPojoMap, "/status/cluster");
-    context.addServlet(sh, "/*");
+
+    // set up context
+    ServletContextHandler ctxHandler = new ServletContextHandler(server, "/", ServletContextHandler.SESSIONS);
+    ctxHandler.addServlet(sh, PATH_SPEC_ANY);
     if (authFilter != null) {
-      context.addFilter(authFilter, "/*", 1);
+      ctxHandler.addFilter(authFilter, PATH_SPEC_ANY, EnumSet.of(DispatcherType.REQUEST));
     }
 
     // Load filters from configuration.
@@ -297,10 +342,10 @@ public class RESTServer implements Constants {
       ArrayUtils.EMPTY_STRING_ARRAY);
     for (String filter : filterClasses) {
       filter = filter.trim();
-      context.addFilter(Class.forName(filter), "/*", 0);
+      ctxHandler.addFilter(filter, PATH_SPEC_ANY, EnumSet.of(DispatcherType.REQUEST));
     }
-    addCSRFFilter(context, conf);
-    HttpServerUtil.constrainHttpMethods(context);
+    addCSRFFilter(ctxHandler, conf);
+    HttpServerUtil.constrainHttpMethods(ctxHandler);
 
     // Put up info server.
     int port = conf.getInt("hbase.rest.info.port", 8085);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/RESTServletContainer.java
----------------------------------------------------------------------
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 b5ecb35..dfc4549 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
@@ -26,7 +26,7 @@ import javax.servlet.http.HttpServletResponse;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
-import com.sun.jersey.spi.container.servlet.ServletContainer;
+import org.glassfish.jersey.servlet.ServletContainer;
 
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AuthorizationException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GZIPRequestStream.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GZIPRequestStream.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GZIPRequestStream.java
index 02957e9..faa8eea 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GZIPRequestStream.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GZIPRequestStream.java
@@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.rest.filter;
 import java.io.IOException;
 import java.util.zip.GZIPInputStream;
 
+import javax.servlet.ReadListener;
 import javax.servlet.ServletInputStream;
 import javax.servlet.http.HttpServletRequest;
 
@@ -55,4 +56,19 @@ public class GZIPRequestStream extends ServletInputStream
   public void close() throws IOException {
     in.close();
   }
+
+  @Override
+  public boolean isFinished() {
+    throw new UnsupportedOperationException("Asynchonous operation is not supported.");
+  }
+
+  @Override
+  public boolean isReady() {
+    throw new UnsupportedOperationException("Asynchonous operation is not supported.");
+  }
+
+  @Override
+  public void setReadListener(ReadListener listener) {
+    throw new UnsupportedOperationException("Asynchonous operation is not supported.");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GZIPResponseStream.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GZIPResponseStream.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GZIPResponseStream.java
index cc74f9c..0150d24 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GZIPResponseStream.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/filter/GZIPResponseStream.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.util.zip.GZIPOutputStream;
 
 import javax.servlet.ServletOutputStream;
+import javax.servlet.WriteListener;
 import javax.servlet.http.HttpServletResponse;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -75,4 +76,14 @@ public class GZIPResponseStream extends ServletOutputStream
   public void finish() throws IOException {
     out.finish();
   }
+
+  @Override
+  public boolean isReady() {
+    throw new UnsupportedOperationException("Asynchonous operation is not supported.");
+  }
+
+  @Override
+  public void setWriteListener(WriteListener writeListener) {
+    throw new UnsupportedOperationException("Asynchonous operation is not supported.");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java
index d484633..5c8d618 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/model/ScannerModel.java
@@ -28,6 +28,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
 
+import javax.ws.rs.core.MediaType;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
@@ -78,10 +79,8 @@ import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.protobuf.ByteString;
-import com.sun.jersey.api.json.JSONConfiguration;
-import com.sun.jersey.api.json.JSONJAXBContext;
-import com.sun.jersey.api.json.JSONMarshaller;
-import com.sun.jersey.api.json.JSONUnmarshaller;
+
+import org.codehaus.jackson.jaxrs.JacksonJaxbJsonProvider;
 
 /**
  * A representation of Scanner parameters.
@@ -120,6 +119,14 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
   private List<String> labels = new ArrayList<String>();
   private boolean cacheBlocks = true;
 
+  /**
+   * Implement lazily-instantiated singleton as per recipe
+   * here: http://literatejava.com/jvm/fastest-threadsafe-singleton-jvm/
+   */
+  private static class JaxbJsonProviderHolder {
+    static final JacksonJaxbJsonProvider INSTANCE = new JacksonJaxbJsonProvider();
+  }
+
   @XmlRootElement
   static class FilterModel {
 
@@ -466,18 +473,27 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
 
   }
 
+  // The singleton <code>JacksonJaxbJsonProvider</code> instance
+  private static JacksonJaxbJsonProvider jsonProvider;
+  private static final Object jsonProviderLock = new Object();
+
+  /**
+   * Get the <code>JacksonJaxbJsonProvider</code> instance;
+   *
+   * @return A <code>JacksonJaxbJsonProvider</code>.
+   */
+  private static JacksonJaxbJsonProvider getJasonProvider() {
+    return JaxbJsonProviderHolder.INSTANCE;
+  }
+
   /**
    * @param s the JSON representation of the filter
    * @return the filter
    * @throws Exception
    */
   public static Filter buildFilter(String s) throws Exception {
-    JSONJAXBContext context =
-      new JSONJAXBContext(JSONConfiguration.natural().build(),
-        FilterModel.class);
-    JSONUnmarshaller unmarshaller = context.createJSONUnmarshaller();
-    FilterModel model = unmarshaller.unmarshalFromJSON(new StringReader(s),
-      FilterModel.class);
+    FilterModel model = getJasonProvider().locateMapper(FilterModel.class,
+        MediaType.APPLICATION_JSON_TYPE).readValue(s, FilterModel.class);
     return model.build();
   }
 
@@ -487,13 +503,8 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
    * @throws Exception
    */
   public static String stringifyFilter(final Filter filter) throws Exception {
-    JSONJAXBContext context =
-      new JSONJAXBContext(JSONConfiguration.natural().build(),
-        FilterModel.class);
-    JSONMarshaller marshaller = context.createJSONMarshaller();
-    StringWriter writer = new StringWriter();
-    marshaller.marshallToJSON(new FilterModel(filter), writer);
-    return writer.toString();
+    return getJasonProvider().locateMapper(FilterModel.class,
+        MediaType.APPLICATION_JSON_TYPE).writeValueAsString(new FilterModel(filter));
   }
 
   private static final byte[] COLUMN_DIVIDER = Bytes.toBytes(":");

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
index ba91519..d8e8ede 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JAXBContextResolver.java
@@ -44,9 +44,6 @@ import org.apache.hadoop.hbase.rest.model.TableRegionModel;
 import org.apache.hadoop.hbase.rest.model.TableSchemaModel;
 import org.apache.hadoop.hbase.rest.model.VersionModel;
 
-import com.sun.jersey.api.json.JSONConfiguration;
-import com.sun.jersey.api.json.JSONJAXBContext;
-
 /**
  * Plumbing for hooking up Jersey's JSON entity body encoding and decoding
  * support to JAXB. Modify how the context is created (by using e.g. a 
@@ -82,8 +79,7 @@ public class JAXBContextResolver implements ContextResolver<JAXBContext> {
   @SuppressWarnings("unchecked")
   public JAXBContextResolver() throws Exception {
     this.types = new HashSet(Arrays.asList(cTypes));
-    this.context = new JSONJAXBContext(JSONConfiguration.natural().build(),
-      cTypes);
+    context = JAXBContext.newInstance(cTypes);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JacksonProvider.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JacksonProvider.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JacksonProvider.java
deleted file mode 100644
index 7791d02..0000000
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JacksonProvider.java
+++ /dev/null
@@ -1,31 +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.rest.provider;
-
-import org.codehaus.jackson.jaxrs.JacksonJaxbJsonProvider;
-
-import javax.ws.rs.ext.Provider;
-
-//create a class in the defined resource package name
-//so it gets activated
-//Use jackson to take care of json
-//since it has better support for object
-//deserializaiton and less clunky to deal with
-@Provider
-public class JacksonProvider extends JacksonJaxbJsonProvider {
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/PlainTextMessageBodyProducer.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/PlainTextMessageBodyProducer.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/PlainTextMessageBodyProducer.java
index fca4544..2a77683 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/PlainTextMessageBodyProducer.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/PlainTextMessageBodyProducer.java
@@ -46,8 +46,6 @@ import org.apache.hadoop.hbase.rest.Constants;
 public class PlainTextMessageBodyProducer 
   implements MessageBodyWriter<Object> {
 
-  private ThreadLocal<byte[]> buffer = new ThreadLocal<byte[]>();
-
   @Override
   public boolean isWriteable(Class<?> arg0, Type arg1, Annotation[] arg2,
       MediaType arg3) {
@@ -57,9 +55,8 @@ public class PlainTextMessageBodyProducer
   @Override
   public long getSize(Object object, Class<?> type, Type genericType,
       Annotation[] annotations, MediaType mediaType) {
-    byte[] bytes = object.toString().getBytes(); 
-    buffer.set(bytes);
-    return bytes.length;
+    // deprecated by JAX-RS 2.0 and ignored by Jersey runtime
+    return -1;
   }
 
   @Override
@@ -67,8 +64,6 @@ public class PlainTextMessageBodyProducer
       Annotation[] annotations, MediaType mediaType,
       MultivaluedMap<String, Object> httpHeaders, OutputStream outStream)
       throws IOException, WebApplicationException {
-    byte[] bytes = buffer.get();
-    outStream.write(bytes);
-    buffer.remove();
+    outStream.write(object.toString().getBytes());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/ProtobufMessageBodyProducer.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/ProtobufMessageBodyProducer.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/ProtobufMessageBodyProducer.java
index 12171a4..d0b1d5c 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/ProtobufMessageBodyProducer.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/producer/ProtobufMessageBodyProducer.java
@@ -48,8 +48,6 @@ import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
 public class ProtobufMessageBodyProducer
   implements MessageBodyWriter<ProtobufMessageHandler> {
 
-  private ThreadLocal<byte[]> buffer = new ThreadLocal<byte[]>();
-
   @Override
   public boolean isWriteable(Class<?> type, Type genericType, 
       Annotation[] annotations, MediaType mediaType) {
@@ -59,23 +57,14 @@ public class ProtobufMessageBodyProducer
   @Override
   public long getSize(ProtobufMessageHandler m, Class<?> type, Type genericType,
       Annotation[] annotations, MediaType mediaType) {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    try {
-      baos.write(m.createProtobufOutput());
-    } catch (IOException e) {
-      return -1;
-    }
-    byte[] bytes = baos.toByteArray();
-    buffer.set(bytes);
-    return bytes.length;
+    // deprecated by JAX-RS 2.0 and ignored by Jersey runtime
+    return -1;
   }
 
   public void writeTo(ProtobufMessageHandler m, Class<?> type, Type genericType,
       Annotation[] annotations, MediaType mediaType, 
       MultivaluedMap<String, Object> httpHeaders, OutputStream entityStream) 
       throws IOException, WebApplicationException {
-    byte[] bytes = buffer.get();
-    entityStream.write(bytes);
-    buffer.remove();
+    entityStream.write(m.createProtobufOutput());
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/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 7c3e1fd..c5cc121 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
@@ -25,11 +25,21 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.util.HttpServerUtil;
 import org.apache.hadoop.util.StringUtils;
-import org.mortbay.jetty.Server;
-import org.mortbay.jetty.servlet.Context;
-import org.mortbay.jetty.servlet.ServletHolder;
 
-import com.sun.jersey.spi.container.servlet.ServletContainer;
+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.eclipse.jetty.servlet.ServletContextHandler;
+
+import org.glassfish.jersey.jackson1.Jackson1Feature;
+import org.glassfish.jersey.server.ResourceConfig;
+import org.glassfish.jersey.servlet.ServletContainer;
+
+import javax.servlet.DispatcherType;
+import java.util.EnumSet;
 
 public class HBaseRESTTestingUtility {
 
@@ -52,37 +62,43 @@ public class HBaseRESTTestingUtility {
     RESTServlet.getInstance(conf, UserProvider.instantiate(conf));
 
     // set up the Jersey servlet container for Jetty
-    ServletHolder sh = new ServletHolder(ServletContainer.class);
-    sh.setInitParameter(
-      "com.sun.jersey.config.property.resourceConfigClass",
-      ResourceConfig.class.getCanonicalName());
-    sh.setInitParameter("com.sun.jersey.config.property.packages",
-      "jetty");
+    ResourceConfig app = new ResourceConfig().
+        packages("org.apache.hadoop.hbase.rest").register(Jackson1Feature.class);
+    ServletHolder sh = new ServletHolder(new ServletContainer(app));
 
-    LOG.info("configured " + ServletContainer.class.getName());
-    
     // set up Jetty and run the embedded server
     server = new Server(0);
-    server.setSendServerVersion(false);
-    server.setSendDateHeader(false);
-      // set up context
-    Context context = new Context(server, "/", Context.SESSIONS);
-    context.addServlet(sh, "/*");
+    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);
+        ArrayUtils.EMPTY_STRING_ARRAY);
     for (String filter : filterClasses) {
       filter = filter.trim();
-      context.addFilter(Class.forName(filter), "/*", 0);
+      ctxHandler.addFilter(filter, "/*", EnumSet.of(DispatcherType.REQUEST));
     }
-    conf.set(RESTServer.REST_CSRF_BROWSER_USERAGENTS_REGEX_KEY, ".*");
-    RESTServer.addCSRFFilter(context, conf);
-    HttpServerUtil.constrainHttpMethods(context);
     LOG.info("Loaded filter classes :" + filterClasses);
-      // start the server
+
+    conf.set(RESTServer.REST_CSRF_BROWSER_USERAGENTS_REGEX_KEY, ".*");
+    RESTServer.addCSRFFilter(ctxHandler, conf);
+
+    HttpServerUtil.constrainHttpMethods(ctxHandler);
+
+    // start the server
     server.start();
-      // get the port
-    testServletPort = server.getConnectors()[0].getLocalPort();
+    // get the port
+    testServletPort = ((ServerConnector)server.getConnectors()[0]).getLocalPort();
 
     LOG.info("started " + server.getClass().getName() + " on port " + 
       testServletPort);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
index fc8290f..b4b00a9 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/RowResourceBase.java
@@ -43,8 +43,8 @@ import org.apache.hadoop.hbase.rest.client.Response;
 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.rest.provider.JacksonProvider;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.codehaus.jackson.jaxrs.JacksonJaxbJsonProvider;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -93,7 +93,7 @@ public class RowResourceBase {
         RowModel.class);
     xmlMarshaller = context.createMarshaller();
     xmlUnmarshaller = context.createUnmarshaller();
-    jsonMapper = new JacksonProvider()
+    jsonMapper = new JacksonJaxbJsonProvider()
     .locateMapper(CellSetModel.class, MediaType.APPLICATION_JSON_TYPE);
     client = new Client(new Cluster().add("localhost",
       REST_TEST_UTIL.getServletPort()));
@@ -511,7 +511,7 @@ public class RowResourceBase {
     Response response = getValueJson(table, row, column);
     assertEquals(response.getCode(), 200);
     assertEquals(Constants.MIMETYPE_JSON, response.getHeader("content-type"));
-    ObjectMapper mapper = new JacksonProvider()
+    ObjectMapper mapper = new JacksonJaxbJsonProvider()
     .locateMapper(CellSetModel.class, MediaType.APPLICATION_JSON_TYPE);
     CellSetModel cellSet = mapper.readValue(response.getBody(), CellSetModel.class);
     RowModel rowModel = cellSet.getRows().get(0);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
index 1d245b3..9797594 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
@@ -126,21 +126,6 @@ public class TestGzipFilter {
     testScannerResultCodes();
   }
 
-  @Test
-  public void testErrorNotGzipped() throws Exception {
-    Header[] headers = new Header[2];
-    headers[0] = new BasicHeader("Accept", Constants.MIMETYPE_BINARY);
-    headers[1] = new BasicHeader("Accept-Encoding", "gzip");
-    Response response = client.get("/" + TABLE + "/" + ROW_1 + "/" + COLUMN_2, headers);
-    assertEquals(response.getCode(), 404);
-    String contentEncoding = response.getHeader("Content-Encoding");
-    assertTrue(contentEncoding == null || !contentEncoding.contains("gzip"));
-    response = client.get("/" + TABLE, headers);
-    assertEquals(response.getCode(), 405);
-    contentEncoding = response.getHeader("Content-Encoding");
-    assertTrue(contentEncoding == null || !contentEncoding.contains("gzip"));
-  }
-
   void testScannerResultCodes() throws Exception {
     Header[] headers = new Header[3];
     headers[0] = new BasicHeader("Content-Type", Constants.MIMETYPE_XML);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
----------------------------------------------------------------------
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 dfcc8af..2ecba6a 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
@@ -29,10 +29,10 @@ import org.apache.hadoop.hbase.rest.client.Response;
 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.rest.provider.JacksonProvider;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RestTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.codehaus.jackson.jaxrs.JacksonJaxbJsonProvider;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -204,7 +204,7 @@ public class TestMultiRowResource {
     Response response = client.get(path.toString(), Constants.MIMETYPE_JSON);
     assertEquals(response.getCode(), 200);
     ObjectMapper mapper =
-        new JacksonProvider().locateMapper(CellSetModel.class, MediaType.APPLICATION_JSON_TYPE);
+        new JacksonJaxbJsonProvider().locateMapper(CellSetModel.class, MediaType.APPLICATION_JSON_TYPE);
     CellSetModel cellSet = (CellSetModel) mapper.readValue(response.getBody(), CellSetModel.class);
     assertEquals(2, cellSet.getRows().size());
     assertEquals(ROW_1, Bytes.toString(cellSet.getRows().get(0).getKey()));
@@ -232,7 +232,7 @@ public class TestMultiRowResource {
     client.post(row_5_url, Constants.MIMETYPE_BINARY, Bytes.toBytes(VALUE_1), extraHdr);
     Response response = client.get(path.toString(), Constants.MIMETYPE_JSON);
     assertEquals(response.getCode(), 200);
-    ObjectMapper mapper = new JacksonProvider().locateMapper(CellSetModel.class,
+    ObjectMapper mapper = new JacksonJaxbJsonProvider().locateMapper(CellSetModel.class,
       MediaType.APPLICATION_JSON_TYPE);
     CellSetModel cellSet = (CellSetModel) mapper.readValue(response.getBody(), CellSetModel.class);
     assertEquals(1, cellSet.getRows().size());

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java
index 7f75a22..2058f50 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestNamespacesInstanceResource.java
@@ -45,10 +45,10 @@ import org.apache.hadoop.hbase.rest.model.NamespacesInstanceModel;
 import org.apache.hadoop.hbase.rest.model.TableListModel;
 import org.apache.hadoop.hbase.rest.model.TableModel;
 import org.apache.hadoop.hbase.rest.model.TestNamespacesInstanceModel;
-import org.apache.hadoop.hbase.rest.provider.JacksonProvider;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RestTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.codehaus.jackson.jaxrs.JacksonJaxbJsonProvider;
 import org.codehaus.jackson.map.ObjectMapper;
 
 import static org.junit.Assert.*;
@@ -87,8 +87,8 @@ public class TestNamespacesInstanceResource {
       REST_TEST_UTIL.getServletPort()));
     testNamespacesInstanceModel = new TestNamespacesInstanceModel();
     context = JAXBContext.newInstance(NamespacesInstanceModel.class, TableListModel.class);
-    jsonMapper = new JacksonProvider()
-    .locateMapper(NamespacesInstanceModel.class, MediaType.APPLICATION_JSON_TYPE);
+    jsonMapper = new JacksonJaxbJsonProvider()
+      .locateMapper(NamespacesInstanceModel.class, MediaType.APPLICATION_JSON_TYPE);
     NAMESPACE1_PROPS.put("key1", "value1");
     NAMESPACE2_PROPS.put("key2a", "value2a");
     NAMESPACE2_PROPS.put("key2b", "value2b");
@@ -265,7 +265,7 @@ public class TestNamespacesInstanceResource {
 
     // Try REST post and puts with invalid content.
     response = client.post(namespacePath1, Constants.MIMETYPE_JSON, toXML(model1));
-    assertEquals(500, response.getCode());
+    assertEquals(400, response.getCode());
     String jsonString = jsonMapper.writeValueAsString(model2);
     response = client.put(namespacePath2, Constants.MIMETYPE_XML, Bytes.toBytes(jsonString));
     assertEquals(400, response.getCode());

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java
index 1a86ce9..2d5a0c6 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannersWithLabels.java
@@ -207,7 +207,7 @@ public class TestScannersWithLabels {
     response = client.get(scannerURI, Constants.MIMETYPE_XML);
     // Respond with 204 as there are no cells to be retrieved
     assertEquals(response.getCode(), 204);
-    assertEquals(Constants.MIMETYPE_XML, response.getHeader("content-type"));
+    // With no content in the payload, the 'Content-Type' header is not echo back
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java
index 32f508a..96b9c4a 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestTableScan.java
@@ -59,13 +59,13 @@ import org.apache.hadoop.hbase.rest.client.Response;
 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.rest.provider.JacksonProvider;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.RestTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.codehaus.jackson.JsonFactory;
 import org.codehaus.jackson.JsonParser;
 import org.codehaus.jackson.JsonToken;
+import org.codehaus.jackson.jaxrs.JacksonJaxbJsonProvider;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -207,7 +207,7 @@ public class TestTableScan {
       Constants.MIMETYPE_JSON);
     assertEquals(200, response.getCode());
     assertEquals(Constants.MIMETYPE_JSON, response.getHeader("content-type"));
-    ObjectMapper mapper = new JacksonProvider()
+    ObjectMapper mapper = new JacksonJaxbJsonProvider()
         .locateMapper(CellSetModel.class, MediaType.APPLICATION_JSON_TYPE);
     CellSetModel model = mapper.readValue(response.getStream(), CellSetModel.class);
     int count = TestScannerResource.countCellSet(model);
@@ -317,7 +317,7 @@ public class TestTableScan {
       Constants.MIMETYPE_JSON);
     assertEquals(200, response.getCode());
     assertEquals(Constants.MIMETYPE_JSON, response.getHeader("content-type"));
-    ObjectMapper mapper = new JacksonProvider()
+    ObjectMapper mapper = new JacksonJaxbJsonProvider()
         .locateMapper(CellSetModel.class, MediaType.APPLICATION_JSON_TYPE);
     CellSetModel model = mapper.readValue(response.getStream(), CellSetModel.class);
     int count = TestScannerResource.countCellSet(model);
@@ -460,7 +460,7 @@ public class TestTableScan {
       Constants.MIMETYPE_JSON);
     assertEquals(200, response.getCode());
     assertEquals(Constants.MIMETYPE_JSON, response.getHeader("content-type"));
-    ObjectMapper mapper = new JacksonProvider().locateMapper(CellSetModel.class,
+    ObjectMapper mapper = new JacksonJaxbJsonProvider().locateMapper(CellSetModel.class,
       MediaType.APPLICATION_JSON_TYPE);
     CellSetModel model = mapper.readValue(response.getStream(), CellSetModel.class);
     int count = TestScannerResource.countCellSet(model);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 55fc256..09a4512 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -432,6 +432,38 @@
       <type>test-jar</type>
       <scope>test</scope>
     </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-jsp</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-webapp</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.containers</groupId>
+      <artifactId>jersey-container-servlet-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.glassfish.jersey.media</groupId>
+      <artifactId>jersey-media-json-jackson1</artifactId>
+    </dependency>
     <!-- General dependencies -->
     <dependency>
        <groupId>com.github.stephenc.findbugs</groupId>
@@ -451,14 +483,6 @@
       <artifactId>protobuf-java</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.sun.jersey</groupId>
-      <artifactId>jersey-core</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.sun.jersey</groupId>
-      <artifactId>jersey-server</artifactId>
-    </dependency>
-    <dependency>
       <groupId>commons-cli</groupId>
       <artifactId>commons-cli</artifactId>
     </dependency>
@@ -487,42 +511,6 @@
       <artifactId>zookeeper</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty-util</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jetty-sslengine</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jsp-2.1</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>jsp-api-2.1</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.mortbay.jetty</groupId>
-      <artifactId>servlet-api-2.5</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.codehaus.jackson</groupId>
-      <artifactId>jackson-core-asl</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.codehaus.jackson</groupId>
-      <artifactId>jackson-mapper-asl</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.codehaus.jackson</groupId>
-      <artifactId>jackson-jaxrs</artifactId>
-    </dependency>
-    <dependency>
       <groupId>tomcat</groupId>
       <artifactId>jasper-compiler</artifactId>
       <scope>compile</scope>

http://git-wip-us.apache.org/repos/asf/hbase/blob/ffe7dac5/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
index 422db44c..de53bd9 100644
--- 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
@@ -25,8 +25,9 @@ import org.apache.commons.logging.LogConfigurationException;
 import org.apache.commons.logging.LogFactory;
 import org.apache.log4j.Appender;
 import org.apache.log4j.Logger;
-import org.mortbay.jetty.NCSARequestLog;
-import org.mortbay.jetty.RequestLog;
+
+import org.eclipse.jetty.server.RequestLog;
+import org.eclipse.jetty.server.NCSARequestLog;
 
 /**
  * RequestLog object for use with Http