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/26 06:55:33 UTC

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

Repository: hbase
Updated Branches:
  refs/heads/master a8bb27b2e -> 0ac5d4a71


http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac5d4a7/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 be8e98f..84c1fea 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,6 +27,7 @@ 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;
@@ -63,35 +64,31 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.util.Shell;
-
-import org.eclipse.jetty.http.HttpVersion;
-import org.eclipse.jetty.server.Server;
-import org.eclipse.jetty.server.Handler;
-import org.eclipse.jetty.server.HttpConfiguration;
-import org.eclipse.jetty.server.HttpConnectionFactory;
-import org.eclipse.jetty.server.ServerConnector;
-import org.eclipse.jetty.server.SecureRequestCustomizer;
-import org.eclipse.jetty.server.SslConnectionFactory;
-import org.eclipse.jetty.server.handler.ContextHandlerCollection;
-import org.eclipse.jetty.server.handler.HandlerCollection;
-import org.eclipse.jetty.server.RequestLog;
-import org.eclipse.jetty.server.handler.RequestLogHandler;
-import org.eclipse.jetty.servlet.FilterMapping;
-import org.eclipse.jetty.servlet.ServletHandler;
-import org.eclipse.jetty.servlet.FilterHolder;
-import org.eclipse.jetty.servlet.ServletContextHandler;
-import org.eclipse.jetty.servlet.DefaultServlet;
-import org.eclipse.jetty.servlet.ServletHolder;
-import org.eclipse.jetty.util.MultiException;
-import org.eclipse.jetty.util.ssl.SslContextFactory;
-import org.eclipse.jetty.util.thread.QueuedThreadPool;
-import org.eclipse.jetty.webapp.WebAppContext;
-
-import org.glassfish.jersey.server.ResourceConfig;
-import org.glassfish.jersey.servlet.ServletContainer;
+import org.mortbay.io.Buffer;
+import org.mortbay.jetty.Connector;
+import org.mortbay.jetty.Handler;
+import org.mortbay.jetty.MimeTypes;
+import org.mortbay.jetty.RequestLog;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.handler.ContextHandler;
+import org.mortbay.jetty.handler.ContextHandlerCollection;
+import org.mortbay.jetty.handler.HandlerCollection;
+import org.mortbay.jetty.handler.RequestLogHandler;
+import org.mortbay.jetty.nio.SelectChannelConnector;
+import org.mortbay.jetty.security.SslSocketConnector;
+import org.mortbay.jetty.servlet.Context;
+import org.mortbay.jetty.servlet.DefaultServlet;
+import org.mortbay.jetty.servlet.FilterHolder;
+import org.mortbay.jetty.servlet.FilterMapping;
+import org.mortbay.jetty.servlet.ServletHandler;
+import org.mortbay.jetty.servlet.ServletHolder;
+import org.mortbay.jetty.webapp.WebAppContext;
+import org.mortbay.thread.QueuedThreadPool;
+import org.mortbay.util.MultiException;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import com.sun.jersey.spi.container.servlet.ServletContainer;
 
 /**
  * Create a Jetty embedded server to answer http requests. The primary goal
@@ -107,8 +104,6 @@ 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";
@@ -152,8 +147,8 @@ public class HttpServer implements FilterContainer {
      * listener in stop().
      */
     private final boolean isManaged;
-    private final ServerConnector listener;
-    private ListenerInfo(boolean isManaged, ServerConnector listener) {
+    private final Connector listener;
+    private ListenerInfo(boolean isManaged, Connector listener) {
       this.isManaged = isManaged;
       this.listener = listener;
     }
@@ -163,8 +158,8 @@ public class HttpServer implements FilterContainer {
 
   protected final WebAppContext webAppContext;
   protected final boolean findPort;
-  protected final Map<ServletContextHandler, Boolean> defaultContexts =
-      new HashMap<ServletContextHandler, Boolean>();
+  protected final Map<Context, Boolean> defaultContexts =
+      new HashMap<Context, Boolean>();
   protected final List<String> filterNames = new ArrayList<String>();
   static final String STATE_DESCRIPTION_ALIVE = " - alive";
   static final String STATE_DESCRIPTION_NOT_LIVE = " - not live";
@@ -174,6 +169,7 @@ 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;
@@ -298,6 +294,11 @@ 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;
@@ -355,15 +356,17 @@ 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()) {
+      if (endpoints.isEmpty() && connector == null) {
         throw new HadoopIllegalArgumentException("No endpoints specified");
       }
 
       if (hostName == null) {
-        hostName = endpoints.get(0).getHost();
+        hostName = endpoints.isEmpty() ? connector.getHost() : endpoints.get(
+            0).getHost();
       }
 
       if (this.conf == null) {
@@ -377,53 +380,37 @@ public class HttpServer implements FilterContainer {
             signatureSecretFileKey);
       }
 
+      if (connector != null) {
+        server.addUnmanagedListener(connector);
+      }
+
       for (URI ep : endpoints) {
-        ServerConnector listener = null;
+        Connector listener = null;
         String scheme = ep.getScheme();
-        HttpConfiguration httpConfig = new HttpConfiguration();
-        httpConfig.setSecureScheme("https");
-        httpConfig.setHeaderCacheSize(DEFAULT_MAX_HEADER_SIZE);
-        httpConfig.setResponseHeaderSize(DEFAULT_MAX_HEADER_SIZE);
-        httpConfig.setRequestHeaderSize(DEFAULT_MAX_HEADER_SIZE);
-
         if ("http".equals(scheme)) {
-          listener = new ServerConnector(server.webServer, new HttpConnectionFactory(httpConfig));
+          listener = HttpServer.createDefaultChannelConnector();
         } else if ("https".equals(scheme)) {
-          HttpConfiguration httpsConfig = new HttpConfiguration(httpConfig);
-          httpsConfig.addCustomizer(new SecureRequestCustomizer());
-          SslContextFactory sslCtxFactory = new SslContextFactory();
-          sslCtxFactory.setNeedClientAuth(needsClientAuth);
-          sslCtxFactory.setKeyManagerPassword(keyPassword);
+          SslSocketConnector c = new SslSocketConnectorSecure();
+          c.setNeedClientAuth(needsClientAuth);
+          c.setKeyPassword(keyPassword);
 
           if (keyStore != null) {
-            sslCtxFactory.setKeyStorePath(keyStore);
-            sslCtxFactory.setKeyStoreType(keyStoreType);
-            sslCtxFactory.setKeyStorePassword(keyStorePassword);
+            c.setKeystore(keyStore);
+            c.setKeystoreType(keyStoreType);
+            c.setPassword(keyStorePassword);
           }
 
           if (trustStore != null) {
-            sslCtxFactory.setTrustStorePath(trustStore);
-            sslCtxFactory.setTrustStoreType(trustStoreType);
-            sslCtxFactory.setTrustStorePassword(trustStorePassword);
-
+            c.setTruststore(trustStore);
+            c.setTruststoreType(trustStoreType);
+            c.setTrustPassword(trustStorePassword);
           }
-          listener = new ServerConnector(server.webServer, new SslConnectionFactory(sslCtxFactory,
-              HttpVersion.HTTP_1_1.toString()), new HttpConnectionFactory(httpsConfig));
+          listener = c;
+
         } 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);
@@ -443,6 +430,12 @@ 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
@@ -460,7 +453,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, pathSpecs);
+    this(name, bindAddress, port, findPort, conf, null, null, pathSpecs);
   }
 
   /**
@@ -475,14 +468,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);
+    this(name, bindAddress, port, findPort, conf, null, null, null);
   }
 
   @Deprecated
   public HttpServer(String name, String bindAddress, int port,
       boolean findPort, Configuration conf, AccessControlList adminsAcl)
       throws IOException {
-    this(name, bindAddress, port, findPort, conf, adminsAcl, null);
+    this(name, bindAddress, port, findPort, conf, adminsAcl, null, null);
   }
 
   /**
@@ -495,33 +488,44 @@ 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,
-      String[] pathSpecs) throws IOException {
+      Connector connector, String[] pathSpecs) throws IOException {
     this(new Builder().setName(name)
         .addEndpoint(URI.create("http://" + bindAddress + ":" + port))
         .setFindPort(findPort).setConf(conf).setACL(adminsAcl)
-        .setPathSpec(pathSpecs));
+        .setConnector(connector).setPathSpec(pathSpecs));
   }
 
   private HttpServer(final Builder b) throws IOException {
     this.appDir = b.appDir;
     this.logDir = b.logDir;
     final String appDir = getWebAppsPath(b.name);
-
-
-    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.webServer = new Server();
     this.adminsAcl = b.adminsAcl;
     this.webAppContext = createWebAppContext(b.name, b.conf, adminsAcl, appDir);
     this.findPort = b.findPort;
@@ -534,6 +538,14 @@ 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);
 
@@ -549,7 +561,7 @@ public class HttpServer implements FilterContainer {
 
     final String appDir = getWebAppsPath(name);
 
-    webServer.setHandler(webAppContext);
+    webServer.addHandler(webAppContext);
 
     addDefaultApps(contexts, appDir, conf);
 
@@ -577,7 +589,11 @@ public class HttpServer implements FilterContainer {
     }
   }
 
-  private void addManagedListener(ServerConnector connector) {
+  private void addUnmanagedListener(Connector connector) {
+    listeners.add(new ListenerInfo(false, connector));
+  }
+
+  private void addManagedListener(Connector connector) {
     listeners.add(new ListenerInfo(true, connector));
   }
 
@@ -598,6 +614,33 @@ 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) {
@@ -629,10 +672,9 @@ public class HttpServer implements FilterContainer {
         logDir = System.getProperty("hadoop.log.dir");
     }
     if (logDir != null) {
-      ServletContextHandler logContext = new ServletContextHandler(parent, "/*");
-      logContext.addServlet(AdminAuthorizedServlet.class, "/*");
+      Context logContext = new Context(parent, "/logs");
       logContext.setResourceBase(logDir);
-
+      logContext.addServlet(AdminAuthorizedServlet.class, "/*");
       if (conf.getBoolean(
           ServerConfigurationKeys.HBASE_JETTY_LOGS_SERVE_ALIASES,
           ServerConfigurationKeys.DEFAULT_HBASE_JETTY_LOGS_SERVE_ALIASES)) {
@@ -647,7 +689,7 @@ public class HttpServer implements FilterContainer {
       defaultContexts.put(logContext, true);
     }
     // set up the context for "/static/*"
-    ServletContextHandler staticContext = new ServletContextHandler(parent, "/static");
+    Context staticContext = new Context(parent, "/static");
     staticContext.setResourceBase(appDir + "/static");
     staticContext.addServlet(DefaultServlet.class, "/*");
     staticContext.setDisplayName("static");
@@ -655,7 +697,7 @@ public class HttpServer implements FilterContainer {
     defaultContexts.put(staticContext, true);
   }
 
-  private void setContextAttributes(ServletContextHandler context, Configuration conf) {
+  private void setContextAttributes(Context context, Configuration conf) {
     context.getServletContext().setAttribute(CONF_CONTEXT_ATTRIBUTE, conf);
     context.getServletContext().setAttribute(ADMINS_ACL, adminsAcl);
   }
@@ -682,6 +724,30 @@ 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)".
@@ -701,9 +767,10 @@ public class HttpServer implements FilterContainer {
       final String pathSpec) {
     LOG.info("addJerseyResourcePackage: packageName=" + packageName
         + ", pathSpec=" + pathSpec);
-
-    ResourceConfig application = new ResourceConfig().packages(packageName);
-    final ServletHolder sh = new ServletHolder(new ServletContainer(application));
+    final ServletHolder sh = new ServletHolder(ServletContainer.class);
+    sh.setInitParameter("com.sun.jersey.config.property.resourceConfigClass",
+        "com.sun.jersey.api.core.PackagesResourceConfig");
+    sh.setInitParameter("com.sun.jersey.config.property.packages", packageName);
     webAppContext.addServlet(sh, pathSpec);
   }
 
@@ -761,7 +828,7 @@ public class HttpServer implements FilterContainer {
        FilterMapping fmap = new FilterMapping();
        fmap.setPathSpec(pathSpec);
        fmap.setFilterName(SPNEGO_FILTER);
-       fmap.setDispatches(FilterMapping.ALL);
+       fmap.setDispatches(Handler.ALL);
        handler.addFilterMapping(fmap);
     }
   }
@@ -775,12 +842,12 @@ public class HttpServer implements FilterContainer {
     LOG.info("Added filter " + name + " (class=" + classname
         + ") to context " + webAppContext.getDisplayName());
     final String[] ALL_URLS = { "/*" };
-    for (Map.Entry<ServletContextHandler, Boolean> e : defaultContexts.entrySet()) {
+    for (Map.Entry<Context, Boolean> e : defaultContexts.entrySet()) {
       if (e.getValue()) {
-        ServletContextHandler handler = e.getKey();
-        defineFilter(handler, name, classname, parameters, ALL_URLS);
+        Context ctx = e.getKey();
+        defineFilter(ctx, name, classname, parameters, ALL_URLS);
         LOG.info("Added filter " + name + " (class=" + classname
-            + ") to context " + handler.getDisplayName());
+            + ") to context " + ctx.getDisplayName());
       }
     }
     filterNames.add(name);
@@ -791,7 +858,7 @@ public class HttpServer implements FilterContainer {
       Map<String, String> parameters) {
     final String[] ALL_URLS = { "/*" };
     defineFilter(webAppContext, name, classname, parameters, ALL_URLS);
-    for (ServletContextHandler ctx : defaultContexts.keySet()) {
+    for (Context ctx : defaultContexts.keySet()) {
       defineFilter(ctx, name, classname, parameters, ALL_URLS);
     }
     LOG.info("Added global filter '" + name + "' (class=" + classname + ")");
@@ -800,20 +867,19 @@ public class HttpServer implements FilterContainer {
   /**
    * Define a filter for a context and set up default url mappings.
    */
-  public static void defineFilter(ServletContextHandler handler, String name,
+  public static void defineFilter(Context ctx, String name,
       String classname, Map<String,String> parameters, String[] urls) {
 
     FilterHolder holder = new FilterHolder();
     holder.setName(name);
     holder.setClassName(classname);
-    if (parameters != null) {
-      holder.setInitParameters(parameters);
-    }
+    holder.setInitParameters(parameters);
     FilterMapping fmap = new FilterMapping();
     fmap.setPathSpecs(urls);
-    fmap.setDispatches(FilterMapping.ALL);
+    fmap.setDispatches(Handler.ALL);
     fmap.setFilterName(name);
-    handler.getServletHandler().addFilter(holder, fmap);
+    ServletHandler handler = ctx.getServletHandler();
+    handler.addFilter(holder, fmap);
   }
 
   /**
@@ -822,13 +888,14 @@ public class HttpServer implements FilterContainer {
    * @param webAppCtx The WebApplicationContext to add to
    */
   protected void addFilterPathMapping(String pathSpec,
-      WebAppContext webAppCtx) {
+      Context webAppCtx) {
+    ServletHandler handler = webAppCtx.getServletHandler();
     for(String name : filterNames) {
       FilterMapping fmap = new FilterMapping();
       fmap.setPathSpec(pathSpec);
       fmap.setFilterName(name);
-      fmap.setDispatches(FilterMapping.ALL);
-      webAppCtx.getServletHandler().addFilterMapping(fmap);
+      fmap.setDispatches(Handler.ALL);
+      handler.addFilterMapping(fmap);
     }
   }
 
@@ -870,7 +937,7 @@ public class HttpServer implements FilterContainer {
    */
   @Deprecated
   public int getPort() {
-    return ((ServerConnector)webServer.getConnectors()[0]).getLocalPort();
+    return webServer.getConnectors()[0].getLocalPort();
   }
 
   /**
@@ -884,10 +951,9 @@ public class HttpServer implements FilterContainer {
     if (index > webServer.getConnectors().length)
       return null;
 
-    ServerConnector c = (ServerConnector)webServer.getConnectors()[index];
-    if (c.getLocalPort() == -1 || c.getLocalPort() == -2) {
-      // -1 if the connector has not been opened
-      // -2 if it has been closed
+    Connector c = webServer.getConnectors()[index];
+    if (c.getLocalPort() == -1) {
+      // The connector is not bounded
       return null;
     }
 
@@ -1014,9 +1080,9 @@ public class HttpServer implements FilterContainer {
    */
   void openListeners() throws Exception {
     for (ListenerInfo li : listeners) {
-      ServerConnector listener = li.listener;
-      if (!li.isManaged || (li.listener.getLocalPort() != -1 && li.listener.getLocalPort() != -2)) {
-        // This listener is either started externally, or has not been opened, or has been closed
+      Connector listener = li.listener;
+      if (!li.isManaged || li.listener.getLocalPort() != -1) {
+        // This listener is either started externally or has been bound
         continue;
       }
       int port = listener.getPort();
@@ -1119,7 +1185,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) {
-        ServerConnector l = li.listener;
+        Connector l = li.listener;
         sb.append(l.getHost()).append(":").append(l.getPort()).append("/,");
       }
       return sb.toString();
@@ -1376,8 +1442,12 @@ public class HttpServer implements FilterContainer {
      */
     private String inferMimeType(ServletRequest request) {
       String path = ((HttpServletRequest)request).getRequestURI();
-      ServletContext context = config.getServletContext();
-      return context.getMimeType(path);
+      ContextHandler.SContext sContext = (ContextHandler.SContext)config.getServletContext();
+      MimeTypes mimes = sContext.getContextHandler().getMimeTypes();
+      Buffer mimeBuffer = mimes.getMimeByExtension(path);
+      return (mimeBuffer == null) ? null : mimeBuffer.toString();
     }
+
   }
+
 }

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

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac5d4a7/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 3a58adc..6642638 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,9 +70,8 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 import org.mockito.internal.util.reflection.Whitebox;
-
-import org.eclipse.jetty.server.ServerConnector;
-import org.eclipse.jetty.util.ajax.JSON;
+import org.mortbay.jetty.Connector;
+import org.mortbay.util.ajax.JSON;
 
 @Category({MiscTests.class, SmallTests.class})
 public class TestHttpServer extends HttpServerFunctionalTest {
@@ -565,7 +564,7 @@ public class TestHttpServer extends HttpServerFunctionalTest {
       // not bound, ephemeral should return requested port (0 for ephemeral)
       List<?> listeners = (List<?>) Whitebox.getInternalState(server,
           "listeners");
-      ServerConnector listener = (ServerConnector) Whitebox.getInternalState(
+      Connector listener = (Connector) Whitebox.getInternalState(
           listeners.get(0), "listener");
 
       assertEquals(port, listener.getPort());
@@ -624,4 +623,16 @@ 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/0ac5d4a7/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 cfb5ff3..5ddc6d4 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) {
-      GenericTestUtils.assertExceptionContains("Unable to initialize WebAppContext", e);
+      assertTrue( e.getMessage().contains("Problem in starting http server. Server handlers failed"));
     }
   }
   

http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac5d4a7/hbase-spark/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-spark/pom.xml b/hbase-spark/pom.xml
index 8417d2f..035dfcc 100644
--- a/hbase-spark/pom.xml
+++ b/hbase-spark/pom.xml
@@ -51,6 +51,7 @@
         <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/0ac5d4a7/hbase-thrift/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-thrift/pom.xml b/hbase-thrift/pom.xml
index 4b1946b..d097f38 100644
--- a/hbase-thrift/pom.xml
+++ b/hbase-thrift/pom.xml
@@ -305,16 +305,38 @@
       <artifactId>slf4j-api</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-server</artifactId>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-servlet</artifactId>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty-sslengine</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-jsp</artifactId>
+      <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>
     </dependency>
   </dependencies>
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac5d4a7/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 169d42f..46ea7f8 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,7 +48,6 @@ 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;
@@ -81,6 +80,7 @@ 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,17 +123,16 @@ 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.eclipse.jetty.http.HttpVersion;
-import org.eclipse.jetty.server.*;
+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 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
@@ -144,8 +143,6 @@ 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";
 
@@ -163,10 +160,6 @@ 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
@@ -400,85 +393,53 @@ public class ThriftServerRunner implements Runnable {
     TServlet thriftHttpServlet = new ThriftHttpServlet(processor, protocolFactory, realUser,
         conf, hbaseHandler, securityEnabled, doAsEnabled);
 
-    // 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);
-
+    httpServer = new Server();
     // Context handler
-    ServletContextHandler ctxHandler = new ServletContextHandler(httpServer, "/", ServletContextHandler.SESSIONS);
-    ctxHandler.addServlet(new ServletHolder(thriftHttpServlet), "/*");
+    Context context = new Context(httpServer, "/", Context.SESSIONS);
+    context.setContextPath("/");
+    String httpPath = "/*";
+    httpServer.setHandler(context);
+    context.addServlet(new ServletHolder(thriftHttpServlet), httpPath);
 
     // set up Jetty and run the embedded server
-    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;
+    Connector connector = new SelectChannelConnector();
     if(conf.getBoolean(THRIFT_SSL_ENABLED, false)) {
-      HttpConfiguration httpsConfig = new HttpConfiguration(httpConfig);
-      httpsConfig.addCustomizer(new SecureRequestCustomizer());
-
-      SslContextFactory sslCtxFactory = new SslContextFactory();
+      SslSelectChannelConnectorSecure sslConnector = new SslSelectChannelConnectorSecure();
       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);
-      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));
+      sslConnector.setKeystore(keystore);
+      sslConnector.setPassword(password);
+      sslConnector.setKeyPassword(keyPassword);
+      connector = sslConnector;
     }
-    serverConnector.setPort(listenPort);
     String host = getBindAddress(conf).getHostAddress();
-    serverConnector.setHost(host);
-    httpServer.addConnector(serverConnector);
-    httpServer.setStopAtShutdown(true);
+    connector.setPort(listenPort);
+    connector.setHost(host);
+    connector.setHeaderBufferSize(1024 * 64);
+    httpServer.addConnector(connector);
 
     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/0ac5d4a7/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6d837d6..637b236 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1235,16 +1235,12 @@
     <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>2.23.2</jackson.version>
+    <jackson.version>1.9.13</jackson.version>
     <jasper.version>5.5.23</jasper.version>
     <jaxb-api.version>2.2.2</jaxb-api.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.version>6.1.26</jetty.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>
@@ -1664,6 +1660,17 @@
         <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>
@@ -1673,28 +1680,63 @@
         <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>${jackson1.version}</version>
+        <version>${jackson.version}</version>
       </dependency>
       <dependency>
         <groupId>org.codehaus.jackson</groupId>
         <artifactId>jackson-mapper-asl</artifactId>
-        <version>${jackson1.version}</version>
+        <version>${jackson.version}</version>
       </dependency>
       <dependency>
         <groupId>org.codehaus.jackson</groupId>
         <artifactId>jackson-jaxrs</artifactId>
-        <version>${jackson1.version}</version>
+        <version>${jackson.version}</version>
       </dependency>
       <dependency>
         <groupId>org.codehaus.jackson</groupId>
         <artifactId>jackson-xc</artifactId>
-        <version>${jackson1.version}</version>
+        <version>${jackson.version}</version>
       </dependency>
       <dependency>
         <!--If this is not in the runtime lib, we get odd
@@ -1741,82 +1783,32 @@
       </dependency>
       <!-- REST dependencies -->
       <dependency>
-        <groupId>javax.servlet</groupId>
-        <artifactId>javax.servlet-api</artifactId>
-        <version>${servlet.api.version}</version>
+        <groupId>com.google.protobuf</groupId>
+        <artifactId>protobuf-java</artifactId>
+        <version>${external.protobuf.version}</version>
       </dependency>
       <dependency>
-        <groupId>org.eclipse.jetty</groupId>
-        <artifactId>jetty-server</artifactId>
-        <version>${jetty.version}</version>
+        <groupId>com.sun.jersey</groupId>
+        <artifactId>jersey-core</artifactId>
+        <version>${jersey.version}</version>
       </dependency>
       <dependency>
-        <groupId>org.eclipse.jetty</groupId>
-        <artifactId>jetty-servlet</artifactId>
-        <version>${jetty.version}</version>
+        <groupId>com.sun.jersey</groupId>
+        <artifactId>jersey-json</artifactId>
+        <version>${jersey.version}</version>
         <exclusions>
           <exclusion>
-            <groupId>org.eclipse.jetty</groupId>
-            <artifactId>servlet-api</artifactId>
+            <groupId>stax</groupId>
+            <artifactId>stax-api</artifactId>
           </exclusion>
         </exclusions>
       </dependency>
       <dependency>
-        <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>
+        <groupId>com.sun.jersey</groupId>
+        <artifactId>jersey-server</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>
@@ -2170,10 +2162,6 @@
                 <groupId>io.netty</groupId>
                 <artifactId>netty</artifactId>
               </exclusion>
-              <exclusion>
-                <groupId>javax.servlet</groupId>
-                <artifactId>servlet-api</artifactId>
-              </exclusion>
             </exclusions>
           </dependency>
           <dependency>
@@ -2185,10 +2173,6 @@
                 <groupId>io.netty</groupId>
                 <artifactId>netty</artifactId>
               </exclusion>
-              <exclusion>
-                <groupId>javax.servlet</groupId>
-                <artifactId>servlet-api</artifactId>
-              </exclusion>
             </exclusions>
           </dependency>
           <dependency>
@@ -2202,10 +2186,6 @@
                 <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 "HBASE-12894 Upgrade to Jetty 9 for REST / Info Server / Thrift Http Server"

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

This reverts commit 59fd6eb7f68c6d616a645db078939c6d32da7aaf.


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

Branch: refs/heads/master
Commit: 0ac5d4a7171833d33b35c09a3f5f3ca3ee5cd4d1
Parents: a8bb27b
Author: Michael Stack <st...@apache.org>
Authored: Wed Jan 25 22:55:18 2017 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Wed Jan 25 22:55:18 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, 704 insertions(+), 1087 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac5d4a7/hbase-common/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml
index b742da4..03f1682 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -266,6 +266,10 @@
       <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/0ac5d4a7/hbase-it/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-it/pom.xml b/hbase-it/pom.xml
index 55240a9..09118b1 100644
--- a/hbase-it/pom.xml
+++ b/hbase-it/pom.xml
@@ -232,6 +232,11 @@
       <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/0ac5d4a7/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 a9f0c81..688310f 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 1.1', 'Mozilla Public License Version 2.0', 'Creative Commons Attribution License, Version 2.5', 'Eclipse Public License 1.0'])
+#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'])
 ## 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/0ac5d4a7/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 4be6017..53fb51e 100644
--- a/hbase-resource-bundle/src/main/resources/META-INF/NOTICE.vm
+++ b/hbase-resource-bundle/src/main/resources/META-INF/NOTICE.vm
@@ -101,13 +101,21 @@ 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-2016 Mort Bay Consulting Pty Ltd.
+Copyright 1995-2009 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
@@ -118,7 +126,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.eclipse.jetty:jetty-jsp:jar
+## jsp implementation paragraph is handled via org.mortbay.jetty:jsp-2.1
 
 Some portions of the code are Copyright:
   2006 Tim Vernum

http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac5d4a7/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 ab99443..e1de3c7 100644
--- a/hbase-resource-bundle/src/main/resources/supplemental-models.xml
+++ b/hbase-resource-bundle/src/main/resources/supplemental-models.xml
@@ -152,214 +152,6 @@ 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>
 
@@ -797,266 +589,6 @@ 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/0ac5d4a7/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index 18e9da5..53fbad4 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -235,6 +235,20 @@
       <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>
@@ -245,6 +259,18 @@
       <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>
@@ -269,44 +295,38 @@
       <artifactId>jaxb-api</artifactId>
     </dependency>
     <dependency>
-      <groupId>javax.servlet</groupId>
-      <artifactId>javax.servlet-api</artifactId>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-core-asl</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-server</artifactId>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-jaxrs</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-servlet</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-util</artifactId>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-http</artifactId>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty-sslengine</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-io</artifactId>
-    </dependency>
-    <dependency>
-    <groupId>org.eclipse.jetty</groupId>
-    <artifactId>jetty-jmx</artifactId>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty-util</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.eclipse.jetty</groupId>
-      <artifactId>jetty-jsp</artifactId>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jsp-api-2.1</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.glassfish.jersey.containers</groupId>
-      <artifactId>jersey-container-servlet-core</artifactId>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>servlet-api-2.5</artifactId>
     </dependency>
+    <!-- test dependencies -->
     <dependency>
-      <groupId>org.glassfish.jersey.media</groupId>
-      <artifactId>jersey-media-json-jackson1</artifactId>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+      <scope>test</scope>
     </dependency>
   </dependencies>
   <profiles>

http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac5d4a7/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 ea8bc9c..54ce117 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,16 +54,6 @@ 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/0ac5d4a7/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 4ab194c..7cec152 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,14 +18,12 @@
 
 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;
@@ -39,7 +37,9 @@ 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,28 +49,17 @@ 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 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;
+import com.sun.jersey.api.json.JSONConfiguration;
+import com.sun.jersey.spi.container.servlet.ServletContainer;
 
 /**
  * Main class for launching REST gateway as a servlet hosted by Jetty.
@@ -93,8 +82,6 @@ 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,
@@ -122,19 +109,17 @@ public class RESTServer implements Constants {
   }
 
   static String REST_CSRF_BROWSER_USERAGENTS_REGEX_KEY = "hbase.rest-csrf.browser-useragents-regex";
-  static void addCSRFFilter(ServletContextHandler ctxHandler, Configuration conf) {
+  static void addCSRFFilter(Context context, 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.");
-      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));
+      HttpServer.defineFilter(context, "csrf", RestCsrfPreventionFilter.class.getName(),
+        restCsrfParams, urls);
     }
   }
 
@@ -158,10 +143,10 @@ public class RESTServer implements Constants {
         FilterHolder authFilter = new FilterHolder();
         authFilter.setClassName(AuthFilter.class.getName());
         authFilter.setName("AuthenticationFilter");
-        return new Pair<>(authFilter,containerClass);
+        return new Pair<FilterHolder, Class<? extends ServletContainer>>(authFilter,containerClass);
       }
     }
-    return new Pair<>(null, containerClass);
+    return new Pair<FilterHolder, Class<? extends ServletContainer>>(null, containerClass);
   }
 
   private static void parseCommandLine(String[] args, RESTServlet servlet) {
@@ -235,106 +220,76 @@ 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
-    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);
-
-    Server server = new Server(threadPool);
-
-    // Setup JMX
-    MBeanContainer mbContainer=new MBeanContainer(ManagementFactory.getPlatformMBeanServer());
-    server.addEventListener(mbContainer);
-    server.addBean(mbContainer);
-
+    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");
 
-    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);
+    // set up Jetty and run the embedded server
 
-    ServerConnector serverConnector;
-    if (conf.getBoolean(REST_SSL_ENABLED, false)) {
-      HttpConfiguration httpsConfig = new HttpConfiguration(httpConfig);
-      httpsConfig.addCustomizer(new SecureRequestCustomizer());
+    Server server = new Server();
 
-      SslContextFactory sslCtxFactory = new SslContextFactory();
+    Connector connector = new SelectChannelConnector();
+    if(conf.getBoolean(REST_SSL_ENABLED, false)) {
+      SslSelectChannelConnectorSecure sslConnector = new SslSelectChannelConnectorSecure();
       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);
-      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));
+        REST_SSL_KEYSTORE_KEYPASSWORD, password);
+      sslConnector.setKeystore(keystore);
+      sslConnector.setPassword(password);
+      sslConnector.setKeyPassword(keyPassword);
+      connector = sslConnector;
     }
+    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);
 
-    int acceptQueueSize = servlet.getConfiguration().getInt(REST_CONNECTOR_ACCEPT_QUEUE_SIZE, -1);
-    if (acceptQueueSize >= 0) {
-      serverConnector.setAcceptQueueSize(acceptQueueSize);
-    }
+    server.addConnector(connector);
 
-    serverConnector.setPort(servicePort);
-    serverConnector.setHost(host);
-
-    server.addConnector(serverConnector);
+    // 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);
     server.setStopAtShutdown(true);
-
-    // set up context
-    ServletContextHandler ctxHandler = new ServletContextHandler(server, "/", ServletContextHandler.SESSIONS);
-    ctxHandler.addServlet(sh, PATH_SPEC_ANY);
+      // set up context
+    Context context = new Context(server, "/", Context.SESSIONS);
+    context.addServlet(shPojoMap, "/status/cluster");
+    context.addServlet(sh, "/*");
     if (authFilter != null) {
-      ctxHandler.addFilter(authFilter, PATH_SPEC_ANY, EnumSet.of(DispatcherType.REQUEST));
+      context.addFilter(authFilter, "/*", 1);
     }
 
     // Load filters from configuration.
@@ -342,10 +297,10 @@ public class RESTServer implements Constants {
       ArrayUtils.EMPTY_STRING_ARRAY);
     for (String filter : filterClasses) {
       filter = filter.trim();
-      ctxHandler.addFilter(filter, PATH_SPEC_ANY, EnumSet.of(DispatcherType.REQUEST));
+      context.addFilter(Class.forName(filter), "/*", 0);
     }
-    addCSRFFilter(ctxHandler, conf);
-    HttpServerUtil.constrainHttpMethods(ctxHandler);
+    addCSRFFilter(context, conf);
+    HttpServerUtil.constrainHttpMethods(context);
 
     // Put up info server.
     int port = conf.getInt("hbase.rest.info.port", 8085);

http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac5d4a7/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 dfc4549..b5ecb35 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 org.glassfish.jersey.servlet.ServletContainer;
+import com.sun.jersey.spi.container.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/0ac5d4a7/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 faa8eea..02957e9 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,7 +22,6 @@ 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;
 
@@ -56,19 +55,4 @@ 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/0ac5d4a7/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 0150d24..cc74f9c 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,7 +23,6 @@ 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;
@@ -76,14 +75,4 @@ 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/0ac5d4a7/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 5c8d618..d484633 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,7 +28,6 @@ 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;
@@ -79,8 +78,10 @@ import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import com.google.protobuf.ByteString;
-
-import org.codehaus.jackson.jaxrs.JacksonJaxbJsonProvider;
+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;
 
 /**
  * A representation of Scanner parameters.
@@ -119,14 +120,6 @@ 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 {
 
@@ -473,27 +466,18 @@ 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 {
-    FilterModel model = getJasonProvider().locateMapper(FilterModel.class,
-        MediaType.APPLICATION_JSON_TYPE).readValue(s, FilterModel.class);
+    JSONJAXBContext context =
+      new JSONJAXBContext(JSONConfiguration.natural().build(),
+        FilterModel.class);
+    JSONUnmarshaller unmarshaller = context.createJSONUnmarshaller();
+    FilterModel model = unmarshaller.unmarshalFromJSON(new StringReader(s),
+      FilterModel.class);
     return model.build();
   }
 
@@ -503,8 +487,13 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
    * @throws Exception
    */
   public static String stringifyFilter(final Filter filter) throws Exception {
-    return getJasonProvider().locateMapper(FilterModel.class,
-        MediaType.APPLICATION_JSON_TYPE).writeValueAsString(new FilterModel(filter));
+    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();
   }
 
   private static final byte[] COLUMN_DIVIDER = Bytes.toBytes(":");

http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac5d4a7/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 d8e8ede..ba91519 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,6 +44,9 @@ 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 
@@ -79,7 +82,8 @@ public class JAXBContextResolver implements ContextResolver<JAXBContext> {
   @SuppressWarnings("unchecked")
   public JAXBContextResolver() throws Exception {
     this.types = new HashSet(Arrays.asList(cTypes));
-    context = JAXBContext.newInstance(cTypes);
+    this.context = new JSONJAXBContext(JSONConfiguration.natural().build(),
+      cTypes);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac5d4a7/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
new file mode 100644
index 0000000..7791d02
--- /dev/null
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/provider/JacksonProvider.java
@@ -0,0 +1,31 @@
+/**
+ * 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/0ac5d4a7/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 2a77683..fca4544 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,6 +46,8 @@ 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) {
@@ -55,8 +57,9 @@ public class PlainTextMessageBodyProducer
   @Override
   public long getSize(Object object, Class<?> type, Type genericType,
       Annotation[] annotations, MediaType mediaType) {
-    // deprecated by JAX-RS 2.0 and ignored by Jersey runtime
-    return -1;
+    byte[] bytes = object.toString().getBytes(); 
+    buffer.set(bytes);
+    return bytes.length;
   }
 
   @Override
@@ -64,6 +67,8 @@ public class PlainTextMessageBodyProducer
       Annotation[] annotations, MediaType mediaType,
       MultivaluedMap<String, Object> httpHeaders, OutputStream outStream)
       throws IOException, WebApplicationException {
-    outStream.write(object.toString().getBytes());
+    byte[] bytes = buffer.get();
+    outStream.write(bytes);
+    buffer.remove();
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac5d4a7/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 d0b1d5c..12171a4 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,6 +48,8 @@ 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) {
@@ -57,14 +59,23 @@ public class ProtobufMessageBodyProducer
   @Override
   public long getSize(ProtobufMessageHandler m, Class<?> type, Type genericType,
       Annotation[] annotations, MediaType mediaType) {
-    // deprecated by JAX-RS 2.0 and ignored by Jersey runtime
-    return -1;
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    try {
+      baos.write(m.createProtobufOutput());
+    } catch (IOException e) {
+      return -1;
+    }
+    byte[] bytes = baos.toByteArray();
+    buffer.set(bytes);
+    return bytes.length;
   }
 
   public void writeTo(ProtobufMessageHandler m, Class<?> type, Type genericType,
       Annotation[] annotations, MediaType mediaType, 
       MultivaluedMap<String, Object> httpHeaders, OutputStream entityStream) 
       throws IOException, WebApplicationException {
-    entityStream.write(m.createProtobufOutput());
+    byte[] bytes = buffer.get();
+    entityStream.write(bytes);
+    buffer.remove();
   }
 }

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac5d4a7/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 0ab95ec..61e650b 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 JacksonJaxbJsonProvider()
+    jsonMapper = new JacksonProvider()
     .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 JacksonJaxbJsonProvider()
+    ObjectMapper mapper = new JacksonProvider()
     .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/0ac5d4a7/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 6e76bc4..5097454 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,6 +126,21 @@ 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/0ac5d4a7/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 21996cc..c1087b5 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 JacksonJaxbJsonProvider().locateMapper(CellSetModel.class, MediaType.APPLICATION_JSON_TYPE);
+        new JacksonProvider().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 JacksonJaxbJsonProvider().locateMapper(CellSetModel.class,
+    ObjectMapper mapper = new JacksonProvider().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/0ac5d4a7/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 3676313..d7649b0 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 JacksonJaxbJsonProvider()
-      .locateMapper(NamespacesInstanceModel.class, MediaType.APPLICATION_JSON_TYPE);
+    jsonMapper = new JacksonProvider()
+    .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(400, response.getCode());
+    assertEquals(500, 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/0ac5d4a7/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 130e752..8b0b5b2 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);
-    // With no content in the payload, the 'Content-Type' header is not echo back
+    assertEquals(Constants.MIMETYPE_XML, response.getHeader("content-type"));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/0ac5d4a7/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 35fe112..7924aed 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 JacksonJaxbJsonProvider()
+    ObjectMapper mapper = new JacksonProvider()
         .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 JacksonJaxbJsonProvider()
+    ObjectMapper mapper = new JacksonProvider()
         .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 JacksonJaxbJsonProvider().locateMapper(CellSetModel.class,
+    ObjectMapper mapper = new JacksonProvider().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/0ac5d4a7/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 09a4512..55fc256 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -432,38 +432,6 @@
       <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>
@@ -483,6 +451,14 @@
       <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>
@@ -511,6 +487,42 @@
       <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/0ac5d4a7/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 de53bd9..422db44c 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,9 +25,8 @@ import org.apache.commons.logging.LogConfigurationException;
 import org.apache.commons.logging.LogFactory;
 import org.apache.log4j.Appender;
 import org.apache.log4j.Logger;
-
-import org.eclipse.jetty.server.RequestLog;
-import org.eclipse.jetty.server.NCSARequestLog;
+import org.mortbay.jetty.NCSARequestLog;
+import org.mortbay.jetty.RequestLog;
 
 /**
  * RequestLog object for use with Http