You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2019/01/04 06:40:28 UTC

[02/15] hbase git commit: HBASE-21652 Refactor ThriftServer making thrift2 server inherited from thrift1 server

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
index 5681569..fa3d39d 100644
--- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
+++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
@@ -18,355 +18,86 @@
  */
 package org.apache.hadoop.hbase.thrift2;
 
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-import java.security.PrivilegedAction;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
+import static org.apache.hadoop.hbase.thrift.Constants.READONLY_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_READONLY_ENABLED;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_READONLY_ENABLED_DEFAULT;
 
-import javax.security.auth.callback.Callback;
-import javax.security.auth.callback.UnsupportedCallbackException;
-import javax.security.sasl.AuthorizeCallback;
-import javax.security.sasl.SaslServer;
+import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.filter.ParseFilter;
-import org.apache.hadoop.hbase.http.InfoServer;
-import org.apache.hadoop.hbase.security.SaslUtil;
-import org.apache.hadoop.hbase.security.SecurityUtil;
 import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.thrift.CallQueue;
-import org.apache.hadoop.hbase.thrift.THBaseThreadPoolExecutor;
-import org.apache.hadoop.hbase.thrift.ThriftMetrics;
+import org.apache.hadoop.hbase.thrift.HBaseServiceHandler;
+import org.apache.hadoop.hbase.thrift.HbaseHandlerMetricsProxy;
 import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
-import org.apache.hadoop.hbase.util.DNS;
-import org.apache.hadoop.hbase.util.JvmPauseMonitor;
-import org.apache.hadoop.hbase.util.Strings;
-import org.apache.hadoop.security.SaslRpcServer.SaslGssCallbackHandler;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.thrift.TException;
 import org.apache.thrift.TProcessor;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.protocol.TProtocolFactory;
-import org.apache.thrift.server.THsHaServer;
-import org.apache.thrift.server.TNonblockingServer;
-import org.apache.thrift.server.TServer;
-import org.apache.thrift.server.TThreadPoolServer;
-import org.apache.thrift.server.TThreadedSelectorServer;
-import org.apache.thrift.transport.TFramedTransport;
-import org.apache.thrift.transport.TNonblockingServerSocket;
-import org.apache.thrift.transport.TNonblockingServerTransport;
-import org.apache.thrift.transport.TSaslServerTransport;
-import org.apache.thrift.transport.TServerSocket;
-import org.apache.thrift.transport.TServerTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.apache.thrift.transport.TTransportFactory;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.DefaultParser;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.OptionGroup;
 import org.apache.hbase.thirdparty.org.apache.commons.cli.Options;
-import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException;
 
 /**
  * ThriftServer - this class starts up a Thrift server which implements the HBase API specified in
  * the HbaseClient.thrift IDL file.
  */
+@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NM_SAME_SIMPLE_NAME_AS_SUPERCLASS",
+    justification = "Change the name will be an incompatible change, will do it later")
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
 @SuppressWarnings({ "rawtypes", "unchecked" })
-public class ThriftServer extends Configured implements Tool {
+public class ThriftServer extends org.apache.hadoop.hbase.thrift.ThriftServer {
   private static final Logger log = LoggerFactory.getLogger(ThriftServer.class);
 
-  /**
-   * Thrift quality of protection configuration key. Valid values can be:
-   * privacy: authentication, integrity and confidentiality checking
-   * integrity: authentication and integrity checking
-   * authentication: authentication only
-   *
-   * This is used to authenticate the callers and support impersonation.
-   * The thrift server and the HBase cluster must run in secure mode.
-   */
-  static final String THRIFT_QOP_KEY = "hbase.thrift.security.qop";
-
-  static final String BACKLOG_CONF_KEY = "hbase.regionserver.thrift.backlog";
 
-  public static final int DEFAULT_LISTEN_PORT = 9090;
-
-  private static final String READ_TIMEOUT_OPTION = "readTimeout";
-
-  /**
-   * Amount of time in milliseconds before a server thread will timeout
-   * waiting for client to send data on a connected socket. Currently,
-   * applies only to TBoundedThreadPoolServer
-   */
-  public static final String THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY =
-    "hbase.thrift.server.socket.read.timeout";
-  public static final int THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT = 60000;
-
-  public ThriftServer() {
+  public ThriftServer(Configuration conf) {
+    super(conf);
   }
 
-  private static void printUsage() {
+  @Override
+  protected void printUsageAndExit(Options options, int exitCode)
+      throws Shell.ExitCodeException {
     HelpFormatter formatter = new HelpFormatter();
-    formatter.printHelp("Thrift", null, getOptions(),
+    formatter.printHelp("Thrift", null, options,
         "To start the Thrift server run 'hbase-daemon.sh start thrift2' or " +
-        "'hbase thrift2'\n" +
-        "To shutdown the thrift server run 'hbase-daemon.sh stop thrift2' or" +
-        " send a kill signal to the thrift server pid",
+            "'hbase thrift2'\n" +
+            "To shutdown the thrift server run 'hbase-daemon.sh stop thrift2' or" +
+            " send a kill signal to the thrift server pid",
         true);
+    throw new Shell.ExitCodeException(exitCode, "");
   }
 
-  private static Options getOptions() {
-    Options options = new Options();
-    options.addOption("b", "bind", true,
-        "Address to bind the Thrift server to. [default: 0.0.0.0]");
-    options.addOption("p", "port", true, "Port to bind to [default: " + DEFAULT_LISTEN_PORT + "]");
-    options.addOption("f", "framed", false, "Use framed transport");
-    options.addOption("c", "compact", false, "Use the compact protocol");
-    options.addOption("w", "workers", true, "How many worker threads to use.");
-    options.addOption("s", "selectors", true, "How many selector threads to use.");
-    options.addOption("q", "callQueueSize", true,
-      "Max size of request queue (unbounded by default)");
-    options.addOption("h", "help", false, "Print help information");
-    options.addOption(null, "infoport", true, "Port for web UI");
-    options.addOption("t", READ_TIMEOUT_OPTION, true,
-      "Amount of time in milliseconds before a server thread will timeout " +
-      "waiting for client to send data on a connected socket. Currently, " +
-      "only applies to TBoundedThreadPoolServer");
-    options.addOption("ro", "readonly", false,
-      "Respond only to read method requests [default: false]");
-    OptionGroup servers = new OptionGroup();
-    servers.addOption(new Option("nonblocking", false,
-            "Use the TNonblockingServer. This implies the framed transport."));
-    servers.addOption(new Option("hsha", false,
-            "Use the THsHaServer. This implies the framed transport."));
-    servers.addOption(new Option("selector", false,
-            "Use the TThreadedSelectorServer. This implies the framed transport."));
-    servers.addOption(new Option("threadpool", false,
-            "Use the TThreadPoolServer. This is the default."));
-    options.addOptionGroup(servers);
-    return options;
-  }
-
-  private static CommandLine parseArguments(Configuration conf, Options options, String[] args)
-      throws ParseException, IOException {
-    CommandLineParser parser = new DefaultParser();
-    return parser.parse(options, args);
-  }
-
-  private static TProtocolFactory getTProtocolFactory(boolean isCompact) {
-    if (isCompact) {
-      log.debug("Using compact protocol");
-      return new TCompactProtocol.Factory();
-    } else {
-      log.debug("Using binary protocol");
-      return new TBinaryProtocol.Factory();
-    }
-  }
-
-  private static TTransportFactory getTTransportFactory(
-      SaslUtil.QualityOfProtection qop, String name, String host,
-      boolean framed, int frameSize) {
-    if (framed) {
-      if (qop != null) {
-        throw new RuntimeException("Thrift server authentication"
-          + " doesn't work with framed transport yet");
-      }
-      log.debug("Using framed transport");
-      return new TFramedTransport.Factory(frameSize);
-    } else if (qop == null) {
-      return new TTransportFactory();
-    } else {
-      Map<String, String> saslProperties = SaslUtil.initSaslProperties(qop.name());
-      TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory();
-      saslFactory.addServerDefinition("GSSAPI", name, host, saslProperties,
-        new SaslGssCallbackHandler() {
-          @Override
-          public void handle(Callback[] callbacks)
-              throws UnsupportedCallbackException {
-            AuthorizeCallback ac = null;
-            for (Callback callback : callbacks) {
-              if (callback instanceof AuthorizeCallback) {
-                ac = (AuthorizeCallback) callback;
-              } else {
-                throw new UnsupportedCallbackException(callback,
-                    "Unrecognized SASL GSSAPI Callback");
-              }
-            }
-            if (ac != null) {
-              String authid = ac.getAuthenticationID();
-              String authzid = ac.getAuthorizationID();
-              if (!authid.equals(authzid)) {
-                ac.setAuthorized(false);
-              } else {
-                ac.setAuthorized(true);
-                String userName = SecurityUtil.getUserFromPrincipal(authzid);
-                log.info("Effective user: " + userName);
-                ac.setAuthorizedID(userName);
-              }
-            }
-          }
-        });
-      return saslFactory;
-    }
-  }
-
-  /*
-   * If bindValue is null, we don't bind.
-   */
-  private static InetSocketAddress bindToPort(String bindValue, int listenPort)
-      throws UnknownHostException {
-    try {
-      if (bindValue == null) {
-        return new InetSocketAddress(listenPort);
-      } else {
-        return new InetSocketAddress(InetAddress.getByName(bindValue), listenPort);
-      }
-    } catch (UnknownHostException e) {
-      throw new RuntimeException("Could not bind to provided ip address", e);
-    }
-  }
-
-  private static TServer getTNonBlockingServer(TProtocolFactory protocolFactory,
-      TProcessor processor, TTransportFactory transportFactory, InetSocketAddress inetSocketAddress)
-          throws TTransportException {
-    TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(inetSocketAddress);
-    log.info("starting HBase Nonblocking Thrift server on " + inetSocketAddress.toString());
-    TNonblockingServer.Args serverArgs = new TNonblockingServer.Args(serverTransport);
-    serverArgs.processor(processor);
-    serverArgs.transportFactory(transportFactory);
-    serverArgs.protocolFactory(protocolFactory);
-    return new TNonblockingServer(serverArgs);
-  }
-
-  private static TServer getTHsHaServer(TProtocolFactory protocolFactory,
-      TProcessor processor, TTransportFactory transportFactory,
-      int workerThreads, int maxCallQueueSize,
-      InetSocketAddress inetSocketAddress, ThriftMetrics metrics)
-      throws TTransportException {
-    TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(inetSocketAddress);
-    log.info("starting HBase HsHA Thrift server on " + inetSocketAddress.toString());
-    THsHaServer.Args serverArgs = new THsHaServer.Args(serverTransport);
-    if (workerThreads > 0) {
-      // Could support the min & max threads, avoiding to preserve existing functionality.
-      serverArgs.minWorkerThreads(workerThreads).maxWorkerThreads(workerThreads);
-    }
-    ExecutorService executorService = createExecutor(
-        workerThreads, maxCallQueueSize, metrics);
-    serverArgs.executorService(executorService);
-    serverArgs.processor(processor);
-    serverArgs.transportFactory(transportFactory);
-    serverArgs.protocolFactory(protocolFactory);
-    return new THsHaServer(serverArgs);
-  }
-
-  private static TServer getTThreadedSelectorServer(TProtocolFactory protocolFactory,
-      TProcessor processor, TTransportFactory transportFactory,
-      int workerThreads, int selectorThreads, int maxCallQueueSize,
-      InetSocketAddress inetSocketAddress, ThriftMetrics metrics)
-      throws TTransportException {
-    TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(inetSocketAddress);
-    log.info("starting HBase ThreadedSelector Thrift server on " + inetSocketAddress.toString());
-    TThreadedSelectorServer.Args serverArgs = new TThreadedSelectorServer.Args(serverTransport);
-    if (workerThreads > 0) {
-      serverArgs.workerThreads(workerThreads);
-    }
-    if (selectorThreads > 0) {
-      serverArgs.selectorThreads(selectorThreads);
-    }
-
-    ExecutorService executorService = createExecutor(
-        workerThreads, maxCallQueueSize, metrics);
-    serverArgs.executorService(executorService);
-    serverArgs.processor(processor);
-    serverArgs.transportFactory(transportFactory);
-    serverArgs.protocolFactory(protocolFactory);
-    return new TThreadedSelectorServer(serverArgs);
+  @Override
+  protected HBaseServiceHandler createHandler(Configuration conf, UserProvider userProvider)
+      throws IOException {
+    return new ThriftHBaseServiceHandler(conf, userProvider);
   }
 
-  private static ExecutorService createExecutor(
-      int workerThreads, int maxCallQueueSize, ThriftMetrics metrics) {
-    CallQueue callQueue;
-    if (maxCallQueueSize > 0) {
-      callQueue = new CallQueue(new LinkedBlockingQueue<>(maxCallQueueSize), metrics);
-    } else {
-      callQueue = new CallQueue(new LinkedBlockingQueue<>(), metrics);
-    }
-
-    ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
-    tfb.setDaemon(true);
-    tfb.setNameFormat("thrift2-worker-%d");
-    ThreadPoolExecutor pool = new THBaseThreadPoolExecutor(workerThreads, workerThreads,
-            Long.MAX_VALUE, TimeUnit.SECONDS, callQueue, tfb.build(), metrics);
-    pool.prestartAllCoreThreads();
-    return pool;
+  @Override
+  protected TProcessor createProcessor() {
+    return new THBaseService.Processor<>(HbaseHandlerMetricsProxy
+        .newInstance((THBaseService.Iface) hbaseServiceHandler, metrics, conf));
   }
 
-  private static TServer getTThreadPoolServer(TProtocolFactory protocolFactory,
-                                              TProcessor processor,
-                                              TTransportFactory transportFactory,
-                                              int workerThreads,
-                                              InetSocketAddress inetSocketAddress,
-                                              int backlog,
-                                              int clientTimeout,
-                                              ThriftMetrics metrics)
-      throws TTransportException {
-    TServerTransport serverTransport = new TServerSocket(
-                                           new TServerSocket.ServerSocketTransportArgs().
-                                               bindAddr(inetSocketAddress).backlog(backlog).
-                                               clientTimeout(clientTimeout));
-    log.info("starting HBase ThreadPool Thrift server on " + inetSocketAddress.toString());
-    TThreadPoolServer.Args serverArgs = new TThreadPoolServer.Args(serverTransport);
-    serverArgs.processor(processor);
-    serverArgs.transportFactory(transportFactory);
-    serverArgs.protocolFactory(protocolFactory);
-    if (workerThreads > 0) {
-      serverArgs.maxWorkerThreads(workerThreads);
-    }
-    ThreadPoolExecutor executor = new THBaseThreadPoolExecutor(serverArgs.minWorkerThreads,
-        serverArgs.maxWorkerThreads, serverArgs.stopTimeoutVal, TimeUnit.SECONDS,
-        new SynchronousQueue<>(), metrics);
-    serverArgs.executorService(executor);
-
-    return new TThreadPoolServer(serverArgs);
+  @Override
+  protected void addOptions(Options options) {
+    super.addOptions(options);
+    options.addOption("ro", READONLY_OPTION, false,
+        "Respond only to read method requests [default: false]");
   }
 
-  /**
-   * Adds the option to pre-load filters at startup.
-   *
-   * @param conf  The current configuration instance.
-   */
-  protected static void registerFilters(Configuration conf) {
-    String[] filters = conf.getStrings("hbase.thrift.filters");
-    if(filters != null) {
-      for(String filterClass: filters) {
-        String[] filterPart = filterClass.split(":");
-        if(filterPart.length != 2) {
-          log.warn("Invalid filter specification " + filterClass + " - skipping");
-        } else {
-          ParseFilter.registerFilter(filterPart[0], filterPart[1]);
-        }
-      }
+  @Override
+  protected void parseCommandLine(CommandLine cmd, Options options) throws Shell.ExitCodeException {
+    super.parseCommandLine(cmd, options);
+    boolean readOnly = THRIFT_READONLY_ENABLED_DEFAULT;
+    if (cmd.hasOption(READONLY_OPTION)) {
+      readOnly = true;
     }
+    conf.setBoolean(THRIFT_READONLY_ENABLED, readOnly);
   }
 
   /**
@@ -375,249 +106,8 @@ public class ThriftServer extends Configured implements Tool {
   public static void main(String[] args) throws Exception {
     final Configuration conf = HBaseConfiguration.create();
     // for now, only time we return is on an argument error.
-    final int status = ToolRunner.run(conf, new ThriftServer(), args);
+    final int status = ToolRunner.run(conf, new ThriftServer(conf), args);
     System.exit(status);
   }
 
-  @Override
-  public int run(String[] args) throws Exception {
-    final Configuration conf = getConf();
-    Options options = getOptions();
-    CommandLine cmd = parseArguments(conf, options, args);
-    int workerThreads = 0;
-    int selectorThreads = 0;
-    int maxCallQueueSize = -1; // use unbounded queue by default
-
-    if (cmd.hasOption("help")) {
-      printUsage();
-      return 1;
-    }
-
-    // Get address to bind
-    String bindAddress = getBindAddress(conf, cmd);
-
-    // check if server should only process read requests, if so override the conf
-    if (cmd.hasOption("readonly")) {
-      conf.setBoolean("hbase.thrift.readonly", true);
-      if (log.isDebugEnabled()) {
-        log.debug("readonly set to true");
-      }
-    }
-
-    // Get read timeout
-    int readTimeout = getReadTimeout(conf, cmd);
-    // Get port to bind to
-    int listenPort = getListenPort(conf, cmd);
-    // Thrift's implementation uses '0' as a placeholder for 'use the default.'
-    int backlog = conf.getInt(BACKLOG_CONF_KEY, 0);
-
-    // Local hostname and user name, used only if QOP is configured.
-    String host = null;
-    String name = null;
-
-    UserProvider userProvider = UserProvider.instantiate(conf);
-    // login the server principal (if using secure Hadoop)
-    boolean securityEnabled = userProvider.isHadoopSecurityEnabled()
-      && userProvider.isHBaseSecurityEnabled();
-    if (securityEnabled) {
-      host = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
-        conf.get("hbase.thrift.dns.interface", "default"),
-        conf.get("hbase.thrift.dns.nameserver", "default")));
-      userProvider.login("hbase.thrift.keytab.file", "hbase.thrift.kerberos.principal", host);
-    }
-
-    UserGroupInformation realUser = userProvider.getCurrent().getUGI();
-    String stringQop = conf.get(THRIFT_QOP_KEY);
-    SaslUtil.QualityOfProtection qop = null;
-    if (stringQop != null) {
-      qop = SaslUtil.getQop(stringQop);
-      if (!securityEnabled) {
-        throw new IOException("Thrift server must run in secure mode to support authentication");
-      }
-      // Extract the name from the principal
-      name = SecurityUtil.getUserFromPrincipal(conf.get("hbase.thrift.kerberos.principal"));
-    }
-
-    boolean nonblocking = cmd.hasOption("nonblocking");
-    boolean hsha = cmd.hasOption("hsha");
-    boolean selector = cmd.hasOption("selector");
-
-    ThriftMetrics metrics = new ThriftMetrics(conf, ThriftMetrics.ThriftServerType.TWO);
-    final JvmPauseMonitor pauseMonitor = new JvmPauseMonitor(conf, metrics.getSource());
-
-    String implType = getImplType(nonblocking, hsha, selector);
-
-    conf.set("hbase.regionserver.thrift.server.type", implType);
-    conf.setInt("hbase.regionserver.thrift.port", listenPort);
-    registerFilters(conf);
-
-    // Construct correct ProtocolFactory
-    boolean compact = cmd.hasOption("compact") ||
-        conf.getBoolean("hbase.regionserver.thrift.compact", false);
-    TProtocolFactory protocolFactory = getTProtocolFactory(compact);
-    final ThriftHBaseServiceHandler hbaseHandler =
-      new ThriftHBaseServiceHandler(conf, userProvider);
-    THBaseService.Iface handler =
-      ThriftHBaseServiceHandler.newInstance(hbaseHandler, metrics);
-    final THBaseService.Processor p = new THBaseService.Processor(handler);
-    conf.setBoolean("hbase.regionserver.thrift.compact", compact);
-    TProcessor processor = p;
-
-    boolean framed = cmd.hasOption("framed") ||
-        conf.getBoolean("hbase.regionserver.thrift.framed", false) || nonblocking || hsha;
-    TTransportFactory transportFactory = getTTransportFactory(qop, name, host, framed,
-        conf.getInt("hbase.regionserver.thrift.framed.max_frame_size_in_mb", 2) * 1024 * 1024);
-    InetSocketAddress inetSocketAddress = bindToPort(bindAddress, listenPort);
-    conf.setBoolean("hbase.regionserver.thrift.framed", framed);
-    if (qop != null) {
-      // Create a processor wrapper, to get the caller
-      processor = new TProcessor() {
-        @Override
-        public boolean process(TProtocol inProt,
-            TProtocol outProt) throws TException {
-          TSaslServerTransport saslServerTransport =
-            (TSaslServerTransport)inProt.getTransport();
-          SaslServer saslServer = saslServerTransport.getSaslServer();
-          String principal = saslServer.getAuthorizationID();
-          hbaseHandler.setEffectiveUser(principal);
-          return p.process(inProt, outProt);
-        }
-      };
-    }
-
-    if (cmd.hasOption("w")) {
-      workerThreads = Integer.parseInt(cmd.getOptionValue("w"));
-    }
-    if (cmd.hasOption("s")) {
-      selectorThreads = Integer.parseInt(cmd.getOptionValue("s"));
-    }
-    if (cmd.hasOption("q")) {
-      maxCallQueueSize = Integer.parseInt(cmd.getOptionValue("q"));
-    }
-
-    // check for user-defined info server port setting, if so override the conf
-    try {
-      if (cmd.hasOption("infoport")) {
-        String val = cmd.getOptionValue("infoport");
-        conf.setInt("hbase.thrift.info.port", Integer.parseInt(val));
-        log.debug("Web UI port set to " + val);
-      }
-    } catch (NumberFormatException e) {
-      log.error("Could not parse the value provided for the infoport option", e);
-      printUsage();
-      System.exit(1);
-    }
-
-    // Put up info server.
-    startInfoServer(conf);
-
-    final TServer tserver = getServer(workerThreads, selectorThreads, maxCallQueueSize, readTimeout,
-            backlog, nonblocking, hsha, selector, metrics, protocolFactory, processor,
-            transportFactory, inetSocketAddress);
-
-    realUser.doAs(
-      new PrivilegedAction<Object>() {
-        @Override
-        public Object run() {
-          pauseMonitor.start();
-          try {
-            tserver.serve();
-            return null;
-          } finally {
-            pauseMonitor.stop();
-          }
-        }
-      });
-    // when tserver.stop eventually happens we'll get here.
-    return 0;
-  }
-
-  private String getImplType(boolean nonblocking, boolean hsha, boolean selector) {
-    String implType = "threadpool";
-
-    if (nonblocking) {
-      implType = "nonblocking";
-    } else if (hsha) {
-      implType = "hsha";
-    } else if (selector) {
-      implType = "selector";
-    }
-
-    return implType;
-  }
-
-  private String getBindAddress(Configuration conf, CommandLine cmd) {
-    String bindAddress;
-    if (cmd.hasOption("bind")) {
-      bindAddress = cmd.getOptionValue("bind");
-      conf.set("hbase.thrift.info.bindAddress", bindAddress);
-    } else {
-      bindAddress = conf.get("hbase.thrift.info.bindAddress");
-    }
-    return bindAddress;
-  }
-
-  private int getListenPort(Configuration conf, CommandLine cmd) {
-    int listenPort;
-    try {
-      if (cmd.hasOption("port")) {
-        listenPort = Integer.parseInt(cmd.getOptionValue("port"));
-      } else {
-        listenPort = conf.getInt("hbase.regionserver.thrift.port", DEFAULT_LISTEN_PORT);
-      }
-    } catch (NumberFormatException e) {
-      throw new RuntimeException("Could not parse the value provided for the port option", e);
-    }
-    return listenPort;
-  }
-
-  private int getReadTimeout(Configuration conf, CommandLine cmd) {
-    int readTimeout;
-    if (cmd.hasOption(READ_TIMEOUT_OPTION)) {
-      try {
-        readTimeout = Integer.parseInt(cmd.getOptionValue(READ_TIMEOUT_OPTION));
-      } catch (NumberFormatException e) {
-        throw new RuntimeException("Could not parse the value provided for the timeout option", e);
-      }
-    } else {
-      readTimeout = conf.getInt(THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY,
-        THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT);
-    }
-    return readTimeout;
-  }
-
-  private void startInfoServer(Configuration conf) throws IOException {
-    int port = conf.getInt("hbase.thrift.info.port", 9095);
-
-    if (port >= 0) {
-      conf.setLong("startcode", System.currentTimeMillis());
-      String a = conf.get("hbase.thrift.info.bindAddress", "0.0.0.0");
-      InfoServer infoServer = new InfoServer("thrift", a, port, false, conf);
-      infoServer.setAttribute("hbase.conf", conf);
-      infoServer.start();
-    }
-  }
-
-  private TServer getServer(int workerThreads, int selectorThreads, int maxCallQueueSize,
-        int readTimeout, int backlog, boolean nonblocking, boolean hsha, boolean selector,
-        ThriftMetrics metrics, TProtocolFactory protocolFactory, TProcessor processor,
-        TTransportFactory transportFactory, InetSocketAddress inetSocketAddress)
-          throws TTransportException {
-    TServer server;
-
-    if (nonblocking) {
-      server = getTNonBlockingServer(protocolFactory, processor, transportFactory,
-              inetSocketAddress);
-    } else if (hsha) {
-      server = getTHsHaServer(protocolFactory, processor, transportFactory, workerThreads,
-              maxCallQueueSize, inetSocketAddress, metrics);
-    } else if (selector) {
-      server = getTThreadedSelectorServer(protocolFactory, processor, transportFactory,
-              workerThreads, selectorThreads, maxCallQueueSize, inetSocketAddress, metrics);
-    } else {
-      server = getTThreadPoolServer(protocolFactory, processor, transportFactory, workerThreads,
-              inetSocketAddress, backlog, readTimeout, metrics);
-    }
-    return server;
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp b/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
index eb329fe..b0064b4 100644
--- a/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
+++ b/hbase-thrift/src/main/resources/hbase-webapps/thrift/thrift.jsp
@@ -20,10 +20,10 @@
 <%@ page contentType="text/html;charset=UTF-8"
   import="org.apache.hadoop.conf.Configuration"
   import="org.apache.hadoop.hbase.HBaseConfiguration"
-  import="org.apache.hadoop.hbase.thrift.ThriftServerRunner.ImplType"
   import="org.apache.hadoop.hbase.util.VersionInfo"
   import="java.util.Date"
 %>
+<%@ page import="org.apache.hadoop.hbase.thrift.ImplType" %>
 
 <%
 Configuration conf = (Configuration)getServletContext().getAttribute("hbase.conf");

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
index 0eb7641..fe6655e 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
@@ -17,8 +17,9 @@
  */
 package org.apache.hadoop.hbase.thrift;
 
+import static org.apache.hadoop.hbase.thrift.Constants.INFOPORT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.PORT_OPTION;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.fail;
 
 import java.net.HttpURLConnection;
@@ -65,7 +66,7 @@ public class TestThriftHttpServer {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestThriftHttpServer.class);
 
-  static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+  protected static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   private Thread httpServerThread;
   private volatile Exception httpServerException;
@@ -97,18 +98,17 @@ public class TestThriftHttpServer {
     conf.set("hbase.thrift.security.qop", "privacy");
     conf.setBoolean("hbase.thrift.ssl.enabled", false);
 
-    ThriftServerRunner runner = null;
+    ThriftServer server = null;
     ExpectedException thrown = ExpectedException.none();
     try {
       thrown.expect(IllegalArgumentException.class);
       thrown.expectMessage("Thrift HTTP Server's QoP is privacy, " +
           "but hbase.thrift.ssl.enabled is false");
-      runner = new ThriftServerRunner(conf);
+      server = new ThriftServer(conf);
+      server.run();
       fail("Thrift HTTP Server starts up even with wrong security configurations.");
     } catch (Exception e) {
     }
-
-    assertNull(runner);
   }
 
   private void startHttpServerThread(final String[] args) {
@@ -117,7 +117,7 @@ public class TestThriftHttpServer {
     httpServerException = null;
     httpServerThread = new Thread(() -> {
       try {
-        thriftServer.doMain(args);
+        thriftServer.run(args);
       } catch (Exception e) {
         httpServerException = e;
       }
@@ -145,6 +145,10 @@ public class TestThriftHttpServer {
     runThriftServer(1024 * 64);
   }
 
+  protected ThriftServer createThriftServer() {
+    return new ThriftServer(TEST_UTIL.getConfiguration());
+  }
+
   @Test
   public void testRunThriftServer() throws Exception {
     runThriftServer(0);
@@ -153,14 +157,14 @@ public class TestThriftHttpServer {
   void runThriftServer(int customHeaderSize) throws Exception {
     List<String> args = new ArrayList<>(3);
     port = HBaseTestingUtility.randomFreePort();
-    args.add("-" + ThriftServer.PORT_OPTION);
+    args.add("-" + PORT_OPTION);
     args.add(String.valueOf(port));
-    args.add("-" + ThriftServer.INFOPORT_OPTION);
+    args.add("-" + INFOPORT_OPTION);
     int infoPort = HBaseTestingUtility.randomFreePort();
     args.add(String.valueOf(infoPort));
     args.add("start");
 
-    thriftServer = new ThriftServer(TEST_UTIL.getConfiguration());
+    thriftServer = createThriftServer();
     startHttpServerThread(args.toArray(new String[args.size()]));
 
     // wait up to 10s for the server to start
@@ -195,9 +199,9 @@ public class TestThriftHttpServer {
     Assert.assertEquals(HttpURLConnection.HTTP_FORBIDDEN, conn.getResponseCode());
   }
 
-  static volatile boolean tableCreated = false;
+  protected static volatile boolean tableCreated = false;
 
-  void talkToThriftServer(String url, int customHeaderSize) throws Exception {
+  protected void talkToThriftServer(String url, int customHeaderSize) throws Exception {
     THttpClient httpClient = new THttpClient(url);
     httpClient.open();
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
index 05dc2ae..09d0605 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.thrift;
 
+import static org.apache.hadoop.hbase.thrift.Constants.COALESCE_INC_KEY;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -46,7 +47,6 @@ import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.test.MetricsAssertHelper;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.thrift.ThriftServerRunner.HBaseHandler;
 import org.apache.hadoop.hbase.thrift.generated.BatchMutation;
 import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
 import org.apache.hadoop.hbase.thrift.generated.Hbase;
@@ -71,7 +71,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Unit testing for ThriftServerRunner.HBaseHandler, a part of the
+ * Unit testing for ThriftServerRunner.HBaseServiceHandler, a part of the
  * org.apache.hadoop.hbase.thrift package.
  */
 @Category({ClientTests.class, LargeTests.class})
@@ -113,7 +113,7 @@ public class TestThriftServer {
 
   @BeforeClass
   public static void beforeClass() throws Exception {
-    UTIL.getConfiguration().setBoolean(ThriftServerRunner.COALESCE_INC_KEY, true);
+    UTIL.getConfiguration().setBoolean(COALESCE_INC_KEY, true);
     UTIL.getConfiguration().setBoolean("hbase.table.sanity.checks", false);
     UTIL.getConfiguration().setInt("hbase.client.retries.number", 3);
     UTIL.startMiniCluster();
@@ -152,8 +152,8 @@ public class TestThriftServer {
    * IllegalArgument exception.
    */
   public void doTestTableCreateDrop() throws Exception {
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     doTestTableCreateDrop(handler);
   }
@@ -163,7 +163,7 @@ public class TestThriftServer {
     dropTestTables(handler);
   }
 
-  public static final class MySlowHBaseHandler extends ThriftServerRunner.HBaseHandler
+  public static final class MySlowHBaseHandler extends ThriftHBaseServiceHandler
       implements Hbase.Iface {
 
     protected MySlowHBaseHandler(Configuration c)
@@ -230,7 +230,7 @@ public class TestThriftServer {
   private static Hbase.Iface getHandlerForMetricsTest(ThriftMetrics metrics, Configuration conf)
       throws Exception {
     Hbase.Iface handler = new MySlowHBaseHandler(conf);
-    return HbaseHandlerMetricsProxy.newInstance(handler, metrics, conf);
+    return HbaseHandlerMetricsProxy.newInstance((ThriftHBaseServiceHandler)handler, metrics, conf);
   }
 
   private static ThriftMetrics getMetrics(Configuration conf) throws Exception {
@@ -270,15 +270,15 @@ public class TestThriftServer {
   }
 
   public void doTestIncrements() throws Exception {
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     createTestTables(handler);
     doTestIncrements(handler);
     dropTestTables(handler);
   }
 
-  public static void doTestIncrements(HBaseHandler handler) throws Exception {
+  public static void doTestIncrements(ThriftHBaseServiceHandler handler) throws Exception {
     List<Mutation> mutations = new ArrayList<>(1);
     mutations.add(new Mutation(false, columnAAname, valueEname, true));
     mutations.add(new Mutation(false, columnAname, valueEname, true));
@@ -318,8 +318,8 @@ public class TestThriftServer {
    * versions.
    */
   public void doTestTableMutations() throws Exception {
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     doTestTableMutations(handler);
   }
@@ -395,8 +395,8 @@ public class TestThriftServer {
    */
   public void doTestTableTimestampsAndColumns() throws Exception {
     // Setup
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     handler.createTable(tableAname, getColumnDescriptors());
 
@@ -473,8 +473,8 @@ public class TestThriftServer {
    */
   public void doTestTableScanners() throws Exception {
     // Setup
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     handler.createTable(tableAname, getColumnDescriptors());
 
@@ -592,8 +592,8 @@ public class TestThriftServer {
    * Tests for GetTableRegions
    */
   public void doTestGetTableRegions() throws Exception {
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     doTestGetTableRegions(handler);
   }
@@ -620,7 +620,7 @@ public class TestThriftServer {
 
     conf.set("hbase.thrift.filters", "MyFilter:filterclass");
 
-    ThriftServerRunner.registerFilters(conf);
+    ThriftServer.registerFilters(conf);
 
     Map<String, String> registeredFilters = ParseFilter.getAllFilters();
 
@@ -628,8 +628,8 @@ public class TestThriftServer {
   }
 
   public void doTestGetRegionInfo() throws Exception {
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     doTestGetRegionInfo(handler);
   }
@@ -655,8 +655,8 @@ public class TestThriftServer {
    * Appends the value to a cell and checks that the cell value is updated properly.
    */
   public static void doTestAppend() throws Exception {
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     handler.createTable(tableAname, getColumnDescriptors());
     try {
@@ -687,8 +687,8 @@ public class TestThriftServer {
    * the checkAndPut succeeds.
    */
   public static void doTestCheckAndPut() throws Exception {
-    ThriftServerRunner.HBaseHandler handler =
-      new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler handler =
+      new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
         UserProvider.instantiate(UTIL.getConfiguration()));
     handler.createTable(tableAname, getColumnDescriptors());
     try {
@@ -729,8 +729,8 @@ public class TestThriftServer {
 
     Configuration conf = UTIL.getConfiguration();
     ThriftMetrics metrics = getMetrics(conf);
-    ThriftServerRunner.HBaseHandler hbaseHandler =
-        new ThriftServerRunner.HBaseHandler(UTIL.getConfiguration(),
+    ThriftHBaseServiceHandler hbaseHandler =
+        new ThriftHBaseServiceHandler(UTIL.getConfiguration(),
             UserProvider.instantiate(UTIL.getConfiguration()));
     Hbase.Iface handler = HbaseHandlerMetricsProxy.newInstance(hbaseHandler, metrics, conf);
 
@@ -863,10 +863,10 @@ public class TestThriftServer {
    * the scanner.
    *
    * @param scannerId the scanner to close
-   * @param handler the HBaseHandler interfacing to HBase
+   * @param handler the HBaseServiceHandler interfacing to HBase
    */
   private void closeScanner(
-      int scannerId, ThriftServerRunner.HBaseHandler handler) throws Exception {
+      int scannerId, ThriftHBaseServiceHandler handler) throws Exception {
     handler.scannerGet(scannerId);
     handler.scannerClose(scannerId);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java
index 9e5a103..08dae61 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java
@@ -17,6 +17,11 @@
  */
 package org.apache.hadoop.hbase.thrift;
 
+import static org.apache.hadoop.hbase.thrift.Constants.BIND_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.COMPACT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.FRAMED_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.INFOPORT_OPTION;
+import static org.apache.hadoop.hbase.thrift.Constants.PORT_OPTION;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -28,7 +33,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
-import org.apache.hadoop.hbase.thrift.ThriftServerRunner.ImplType;
 import org.apache.hadoop.hbase.thrift.generated.Hbase;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
@@ -68,12 +72,12 @@ public class TestThriftServerCmdLine {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestThriftServerCmdLine.class);
 
-  private final ImplType implType;
-  private boolean specifyFramed;
-  private boolean specifyBindIP;
-  private boolean specifyCompact;
+  protected final ImplType implType;
+  protected boolean specifyFramed;
+  protected boolean specifyBindIP;
+  protected boolean specifyCompact;
 
-  private static final HBaseTestingUtility TEST_UTIL =
+  protected static final HBaseTestingUtility TEST_UTIL =
       new HBaseTestingUtility();
 
   private Thread cmdLineThread;
@@ -81,8 +85,8 @@ public class TestThriftServerCmdLine {
 
   private Exception clientSideException;
 
-  private ThriftServer thriftServer;
-  private int port;
+  private volatile ThriftServer thriftServer;
+  protected int port;
 
   @Parameters
   public static Collection<Object[]> getParameters() {
@@ -143,8 +147,9 @@ public class TestThriftServerCmdLine {
       @Override
       public void run() {
         try {
-          thriftServer.doMain(args);
+          thriftServer.run(args);
         } catch (Exception e) {
+          LOG.error("Error when start thrift server", e);
           cmdLineException = e;
         }
       }
@@ -154,6 +159,10 @@ public class TestThriftServerCmdLine {
     cmdLineThread.start();
   }
 
+  protected ThriftServer createThriftServer() {
+    return new ThriftServer(TEST_UTIL.getConfiguration());
+  }
+
   @Test
   public void testRunThriftServer() throws Exception {
     List<String> args = new ArrayList<>();
@@ -163,37 +172,37 @@ public class TestThriftServerCmdLine {
       args.add(serverTypeOption);
     }
     port = HBaseTestingUtility.randomFreePort();
-    args.add("-" + ThriftServer.PORT_OPTION);
+    args.add("-" + PORT_OPTION);
     args.add(String.valueOf(port));
-    args.add("-" + ThriftServer.INFOPORT_OPTION);
+    args.add("-" + INFOPORT_OPTION);
     int infoPort = HBaseTestingUtility.randomFreePort();
     args.add(String.valueOf(infoPort));
 
     if (specifyFramed) {
-      args.add("-" + ThriftServer.FRAMED_OPTION);
+      args.add("-" + FRAMED_OPTION);
     }
     if (specifyBindIP) {
-      args.add("-" + ThriftServer.BIND_OPTION);
+      args.add("-" + BIND_OPTION);
       args.add(InetAddress.getLocalHost().getHostName());
     }
     if (specifyCompact) {
-      args.add("-" + ThriftServer.COMPACT_OPTION);
+      args.add("-" + COMPACT_OPTION);
     }
     args.add("start");
 
-    thriftServer = new ThriftServer(TEST_UTIL.getConfiguration());
+    thriftServer = createThriftServer();
     startCmdLineThread(args.toArray(new String[args.size()]));
 
     // wait up to 10s for the server to start
     for (int i = 0; i < 100
-        && (thriftServer.serverRunner == null || thriftServer.serverRunner.tserver == null); i++) {
+        && (thriftServer.tserver == null); i++) {
       Thread.sleep(100);
     }
 
     Class<? extends TServer> expectedClass = implType != null ?
         implType.serverClass : TBoundedThreadPoolServer.class;
     assertEquals(expectedClass,
-                 thriftServer.serverRunner.tserver.getClass());
+                 thriftServer.tserver.getClass());
 
     try {
       talkToThriftServer();
@@ -210,9 +219,9 @@ public class TestThriftServerCmdLine {
     }
   }
 
-  private static volatile boolean tableCreated = false;
+  protected static volatile boolean tableCreated = false;
 
-  private void talkToThriftServer() throws Exception {
+  protected void talkToThriftServer() throws Exception {
     TSocket sock = new TSocket(InetAddress.getLocalHost().getHostName(),
         port);
     TTransport transport = sock;
@@ -228,6 +237,7 @@ public class TestThriftServerCmdLine {
       } else {
         prot = new TBinaryProtocol(transport);
       }
+
       Hbase.Client client = new Hbase.Client(prot);
       if (!tableCreated){
         TestThriftServer.createTestTables(client);

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
index 49eafec..45abe79 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
@@ -18,12 +18,7 @@
  */
 package org.apache.hadoop.hbase.thrift;
 
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_KERBEROS_PRINCIPAL_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_KEYTAB_FILE_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_PRINCIPAL_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SUPPORT_PROXYUSER_KEY;
-import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.USE_HTTP_CONF_KEY;
+import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SUPPORT_PROXYUSER_KEY;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -133,14 +128,14 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
     HBaseKerberosUtils.setSecuredConfiguration(conf, serverPrincipal, spnegoServerPrincipal);
 
     conf.setBoolean(THRIFT_SUPPORT_PROXYUSER_KEY, true);
-    conf.setBoolean(USE_HTTP_CONF_KEY, true);
+    conf.setBoolean(Constants.USE_HTTP_CONF_KEY, true);
     conf.set("hadoop.proxyuser.hbase.hosts", "*");
     conf.set("hadoop.proxyuser.hbase.groups", "*");
 
-    conf.set(THRIFT_KERBEROS_PRINCIPAL_KEY, serverPrincipal);
-    conf.set(THRIFT_KEYTAB_FILE_KEY, serverKeytab.getAbsolutePath());
-    conf.set(THRIFT_SPNEGO_PRINCIPAL_KEY, spnegoServerPrincipal);
-    conf.set(THRIFT_SPNEGO_KEYTAB_FILE_KEY, spnegoServerKeytab.getAbsolutePath());
+    conf.set(Constants.THRIFT_KERBEROS_PRINCIPAL_KEY, serverPrincipal);
+    conf.set(Constants.THRIFT_KEYTAB_FILE_KEY, serverKeytab.getAbsolutePath());
+    conf.set(Constants.THRIFT_SPNEGO_PRINCIPAL_KEY, spnegoServerPrincipal);
+    conf.set(Constants.THRIFT_SPNEGO_KEYTAB_FILE_KEY, spnegoServerKeytab.getAbsolutePath());
   }
 
   @BeforeClass
@@ -170,7 +165,7 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
     spnegoServerKeytab = new File(keytabDir, spnegoServerPrincipal.replace('/', '_') + ".keytab");
     setupUser(kdc, spnegoServerKeytab, spnegoServerPrincipal);
 
-    TEST_UTIL.getConfiguration().setBoolean(USE_HTTP_CONF_KEY, true);
+    TEST_UTIL.getConfiguration().setBoolean(Constants.USE_HTTP_CONF_KEY, true);
     TEST_UTIL.getConfiguration().setBoolean("hbase.table.sanity.checks", false);
     addSecurityConfigurations(TEST_UTIL.getConfiguration());
 
@@ -191,7 +186,7 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
   }
 
   @Override
-  void talkToThriftServer(String url, int customHeaderSize) throws Exception {
+  protected void talkToThriftServer(String url, int customHeaderSize) throws Exception {
     // Close httpClient and THttpClient automatically on any failures
     try (
         CloseableHttpClient httpClient = createHttpClient();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2HttpServer.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2HttpServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2HttpServer.java
new file mode 100644
index 0000000..cf084c9
--- /dev/null
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2HttpServer.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.thrift2;
+
+import java.util.ArrayList;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.thrift.TestThriftHttpServer;
+import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
+import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
+import org.apache.hadoop.hbase.thrift2.generated.TTableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.THttpClient;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+@Category({ ClientTests.class, MediumTests.class})
+public class TestThrift2HttpServer extends TestThriftHttpServer {
+  private static final String TABLENAME = "TestThrift2HttpServerTable";
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestThrift2HttpServer.class);
+
+
+
+  @Override
+  protected ThriftServer createThriftServer() {
+    return new ThriftServer(TEST_UTIL.getConfiguration());
+  }
+
+  @Override
+  protected void talkToThriftServer(String url, int customHeaderSize) throws Exception {
+    THttpClient httpClient = new THttpClient(url);
+    httpClient.open();
+
+    if (customHeaderSize > 0) {
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < customHeaderSize; i++) {
+        sb.append("a");
+      }
+      httpClient.setCustomHeader("User-Agent", sb.toString());
+    }
+
+    try {
+      TProtocol prot;
+      prot = new TBinaryProtocol(httpClient);
+      THBaseService.Client client = new THBaseService.Client(prot);
+      TTableName tTableName = new TTableName();
+      tTableName.setNs(Bytes.toBytes(""));
+      tTableName.setQualifier(Bytes.toBytes(TABLENAME));
+      if (!tableCreated){
+        Assert.assertTrue(!client.tableExists(tTableName));
+        TTableDescriptor tTableDescriptor = new TTableDescriptor();
+        tTableDescriptor.setTableName(tTableName);
+        TColumnFamilyDescriptor columnFamilyDescriptor = new TColumnFamilyDescriptor();
+        columnFamilyDescriptor.setName(Bytes.toBytes(TABLENAME));
+        tTableDescriptor.addToColumns(columnFamilyDescriptor);
+        client.createTable(tTableDescriptor, new ArrayList<>());
+        tableCreated = true;
+      }
+      Assert.assertTrue(client.tableExists(tTableName));
+    } finally {
+      httpClient.close();
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2ServerCmdLine.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2ServerCmdLine.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2ServerCmdLine.java
new file mode 100644
index 0000000..7489bd7
--- /dev/null
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2ServerCmdLine.java
@@ -0,0 +1,99 @@
+/**
+ * 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.thrift2;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.thrift.ImplType;
+import org.apache.hadoop.hbase.thrift.TestThriftServerCmdLine;
+import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
+import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
+import org.apache.hadoop.hbase.thrift2.generated.TTableName;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.experimental.categories.Category;
+
+@Category({ ClientTests.class, MediumTests.class})
+public class TestThrift2ServerCmdLine extends TestThriftServerCmdLine {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestThrift2ServerCmdLine.class);
+
+  private static final String TABLENAME = "TestThrift2ServerCmdLineTable";
+
+
+  @Override
+  protected ThriftServer createThriftServer() {
+    return new ThriftServer(TEST_UTIL.getConfiguration());
+  }
+
+  public TestThrift2ServerCmdLine(ImplType implType, boolean specifyFramed,
+      boolean specifyBindIP, boolean specifyCompact) {
+    super(implType, specifyFramed, specifyBindIP, specifyCompact);
+  }
+
+  @Override
+  protected void talkToThriftServer() throws Exception {
+    TSocket sock = new TSocket(InetAddress.getLocalHost().getHostName(),
+        port);
+    TTransport transport = sock;
+    if (specifyFramed || implType.isAlwaysFramed()) {
+      transport = new TFramedTransport(transport);
+    }
+
+    sock.open();
+    try {
+      TProtocol tProtocol;
+      if (specifyCompact) {
+        tProtocol = new TCompactProtocol(transport);
+      } else {
+        tProtocol = new TBinaryProtocol(transport);
+      }
+      THBaseService.Client client = new THBaseService.Client(tProtocol);
+      TTableName tTableName = new TTableName();
+      tTableName.setNs(Bytes.toBytes(""));
+      tTableName.setQualifier(Bytes.toBytes(TABLENAME));
+      if (!tableCreated){
+        Assert.assertTrue(!client.tableExists(tTableName));
+        TTableDescriptor tTableDescriptor = new TTableDescriptor();
+        tTableDescriptor.setTableName(tTableName);
+        TColumnFamilyDescriptor columnFamilyDescriptor = new TColumnFamilyDescriptor();
+        columnFamilyDescriptor.setName(Bytes.toBytes(TABLENAME));
+        tTableDescriptor.addToColumns(columnFamilyDescriptor);
+        client.createTable(tTableDescriptor, new ArrayList<>());
+        tableCreated = true;
+      }
+      Assert.assertTrue(client.tableExists(tTableName));
+    } finally {
+      sock.close();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/e4b6b4af/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
----------------------------------------------------------------------
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
index 0734f85..844c1c0 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hbase.thrift2;
 
 import static java.nio.ByteBuffer.wrap;
+import static org.apache.hadoop.hbase.thrift.HBaseServiceHandler.CLEANUP_INTERVAL;
+import static org.apache.hadoop.hbase.thrift.HBaseServiceHandler.MAX_IDLETIME;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.deleteFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.getFromThrift;
 import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.incrementFromThrift;
@@ -71,6 +73,7 @@ import org.apache.hadoop.hbase.test.MetricsAssertHelper;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.thrift.ErrorThrowingGetObserver;
+import org.apache.hadoop.hbase.thrift.HbaseHandlerMetricsProxy;
 import org.apache.hadoop.hbase.thrift.ThriftMetrics;
 import org.apache.hadoop.hbase.thrift2.generated.TAppend;
 import org.apache.hadoop.hbase.thrift2.generated.TColumn;
@@ -115,7 +118,7 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 
 /**
- * Unit testing for ThriftServer.HBaseHandler, a part of the org.apache.hadoop.hbase.thrift2
+ * Unit testing for ThriftServer.HBaseServiceHandler, a part of the org.apache.hadoop.hbase.thrift2
  * package.
  */
 @Category({ClientTests.class, MediumTests.class})
@@ -749,8 +752,8 @@ public class TestThriftHBaseServiceHandler {
     int cleanUpInterval = 100;
     Configuration conf = new Configuration(UTIL.getConfiguration());
     // Set the ConnectionCache timeout to trigger halfway through the trials
-    conf.setInt(ThriftHBaseServiceHandler.MAX_IDLETIME, (numTrials / 2) * trialPause);
-    conf.setInt(ThriftHBaseServiceHandler.CLEANUP_INTERVAL, cleanUpInterval);
+    conf.setInt(MAX_IDLETIME, (numTrials / 2) * trialPause);
+    conf.setInt(CLEANUP_INTERVAL, cleanUpInterval);
     ThriftHBaseServiceHandler handler = new ThriftHBaseServiceHandler(conf,
         UserProvider.instantiate(conf));
 
@@ -1206,7 +1209,7 @@ public class TestThriftHBaseServiceHandler {
     ThriftMetrics metrics = getMetrics(conf);
     ThriftHBaseServiceHandler hbaseHandler = createHandler();
     THBaseService.Iface handler =
-        ThriftHBaseServiceHandler.newInstance(hbaseHandler, metrics);
+        HbaseHandlerMetricsProxy.newInstance(hbaseHandler, metrics,  conf);
     byte[] rowName = Bytes.toBytes("testMetrics");
     ByteBuffer table = wrap(tableAname);
 
@@ -1249,7 +1252,7 @@ public class TestThriftHBaseServiceHandler {
     ThriftHBaseServiceHandler hbaseHandler = createHandler();
     ThriftMetrics metrics = getMetrics(UTIL.getConfiguration());
     THBaseService.Iface handler =
-        ThriftHBaseServiceHandler.newInstance(hbaseHandler, metrics);
+        HbaseHandlerMetricsProxy.newInstance(hbaseHandler, metrics, null);
     ByteBuffer tTableName = wrap(tableName.getName());
 
     // check metrics increment with a successful get
@@ -1323,7 +1326,7 @@ public class TestThriftHBaseServiceHandler {
       ThriftHBaseServiceHandler hbaseHandler = createHandler();
       ThriftMetrics metrics = getMetrics(UTIL.getConfiguration());
       THBaseService.Iface handler =
-          ThriftHBaseServiceHandler.newInstance(hbaseHandler, metrics);
+          HbaseHandlerMetricsProxy.newInstance(hbaseHandler, metrics, null);
       ByteBuffer tTableName = wrap(tableName.getName());
 
       // check metrics latency with a successful get