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 2020/05/01 15:38:59 UTC

[hbase] branch branch-2 updated: HBASE-24294 [Flakey Tests] TestThriftHttpServer BindException (#1619)

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

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


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 4cd893a  HBASE-24294 [Flakey Tests] TestThriftHttpServer BindException (#1619)
4cd893a is described below

commit 4cd893ad4743616271ebd0cc5db5d80cb8e6b1e9
Author: Michael Stack <sa...@users.noreply.github.com>
AuthorDate: Fri May 1 08:38:44 2020 -0700

    HBASE-24294 [Flakey Tests] TestThriftHttpServer BindException (#1619)
    
    Refactor Thrift tests so resilient when launch of ThriftServer runs
    into BindException.
    
    hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestBindExceptionHandling.java
    hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftHttpServer.java
    hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServerCmdLine.java
    hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2HttpServer.java
    hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift2/TestThrift2ServerCmdLine.java
     Refactor to use new ThriftServerRunner and ThriftServerSupplier
    
    hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java
     Log list of existing tables on assert.
    
    hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpFallbackServer.java
    hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpServer.java
     Add Ignore for testRunThriftServerWithHeaderBufferLength. Its tested in
     superclass. No need to test again here. Besides, its destructive test
     leaving behind dregs that mess up the retry.
    
    hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
     Utility. Thread to host ThriftServer and exception if one after run
     completes.
---
 .../hbase/thrift/TestBindExceptionHandling.java    |  32 ++---
 .../hadoop/hbase/thrift/TestThriftHttpServer.java  | 119 +++++-------------
 .../hadoop/hbase/thrift/TestThriftServer.java      |   7 +-
 .../hbase/thrift/TestThriftServerCmdLine.java      | 136 +++++++++++----------
 .../thrift/TestThriftSpnegoHttpFallbackServer.java |  18 +++
 .../hbase/thrift/TestThriftSpnegoHttpServer.java   |  34 +++++-
 .../hadoop/hbase/thrift/ThriftServerRunner.java    |  66 ++++++++++
 .../hbase/thrift2/TestThrift2HttpServer.java       |  11 +-
 .../hbase/thrift2/TestThrift2ServerCmdLine.java    |  16 +--
 9 files changed, 248 insertions(+), 191 deletions(-)

diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestBindExceptionHandling.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestBindExceptionHandling.java
index fb8b127..f5bc214 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestBindExceptionHandling.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestBindExceptionHandling.java
@@ -19,11 +19,13 @@ package org.apache.hadoop.hbase.thrift;
 
 import static org.junit.Assert.assertNotNull;
 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.MediumTests;
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import java.io.IOException;
 
 @Category({ ClientTests.class, MediumTests.class})
 public class TestBindExceptionHandling {
@@ -31,20 +33,16 @@ public class TestBindExceptionHandling {
   public static final HBaseClassTestRule CLASS_RULE =
     HBaseClassTestRule.forClass(TestBindExceptionHandling.class);
 
+  private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
+
   /**
    * See if random port choosing works around protocol port clashes
    */
   @Test
-  public void testProtocolPortClash() {
-    ThriftServer thriftServer = null;
-    try {
-      thriftServer = new TestThriftServerCmdLine(null, false, false, false).
-        createBoundServer(true, false);
-      assertNotNull(thriftServer.tserver);
-    } finally {
-      if (thriftServer != null) {
-        thriftServer.stop();
-      }
+  public void testProtocolPortClash() throws Exception {
+    try (ThriftServerRunner tsr = TestThriftServerCmdLine.
+        createBoundServer(() -> new ThriftServer(HTU.getConfiguration()), true, false)) {
+      assertNotNull(tsr.getThriftServer());
     }
   }
 
@@ -52,16 +50,10 @@ public class TestBindExceptionHandling {
    * See if random port choosing works around protocol port clashes
    */
   @Test
-  public void testInfoPortClash() {
-    ThriftServer thriftServer = null;
-    try {
-      thriftServer = new TestThriftServerCmdLine(null, false, false, false).
-        createBoundServer(false, true);
-      assertNotNull(thriftServer.tserver);
-    } finally {
-      if (thriftServer != null) {
-        thriftServer.stop();
-      }
+  public void testInfoPortClash() throws Exception {
+    try (ThriftServerRunner tsr = TestThriftServerCmdLine.
+        createBoundServer(() -> new ThriftServer(HTU.getConfiguration()), false, true)) {
+      assertNotNull(tsr.getThriftServer());
     }
   }
 }
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 ffb9a5f..c291171 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
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,16 +17,14 @@
  */
 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.apache.hadoop.hbase.thrift.TestThriftServerCmdLine.createBoundServer;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
 
-import java.net.BindException;
+import java.io.IOException;
 import java.net.HttpURLConnection;
 import java.net.URL;
-import java.util.ArrayList;
-import java.util.List;
+import java.util.function.Supplier;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -70,17 +68,9 @@ public class TestThriftHttpServer {
 
   protected static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
-  private Thread httpServerThread;
-  private volatile Exception httpServerException;
-
-  private Exception clientSideException;
-
-  private ThriftServer thriftServer;
-  int port;
-
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    TEST_UTIL.getConfiguration().setBoolean("hbase.regionserver.thrift.http", true);
+    TEST_UTIL.getConfiguration().setBoolean(Constants.USE_HTTP_CONF_KEY, true);
     TEST_UTIL.getConfiguration().setBoolean(TableDescriptorChecker.TABLE_SANITY_CHECKS, false);
     TEST_UTIL.startMiniCluster();
     //ensure that server time increments every time we do an operation, otherwise
@@ -95,37 +85,25 @@ public class TestThriftHttpServer {
   }
 
   @Test
-  public void testExceptionThrownWhenMisConfigured() throws Exception {
+  public void testExceptionThrownWhenMisConfigured() throws IOException {
     Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
     conf.set("hbase.thrift.security.qop", "privacy");
     conf.setBoolean("hbase.thrift.ssl.enabled", false);
-
-    ThriftServer server = null;
     ExpectedException thrown = ExpectedException.none();
+    ThriftServerRunner tsr = null;
     try {
       thrown.expect(IllegalArgumentException.class);
       thrown.expectMessage("Thrift HTTP Server's QoP is privacy, " +
           "but hbase.thrift.ssl.enabled is false");
-      server = new ThriftServer(conf);
-      server.run();
+      tsr = TestThriftServerCmdLine.createBoundServer(() -> new ThriftServer(conf));
       fail("Thrift HTTP Server starts up even with wrong security configurations.");
     } catch (Exception e) {
-    }
-  }
-
-  private void startHttpServerThread(final String[] args) {
-    LOG.info("Starting HBase Thrift server with HTTP server: " + Joiner.on(" ").join(args));
-
-    httpServerException = null;
-    httpServerThread = new Thread(() -> {
-      try {
-        thriftServer.run(args);
-      } catch (Exception e) {
-        httpServerException = e;
+      LOG.info("Expected!", e);
+    } finally {
+      if (tsr != null) {
+        tsr.close();
       }
-    });
-    httpServerThread.setName(ThriftServer.class.getSimpleName() + "-httpServer");
-    httpServerThread.start();
+    }
   }
 
   @Rule
@@ -133,7 +111,6 @@ public class TestThriftHttpServer {
 
   @Test
   public void testRunThriftServerWithHeaderBufferLength() throws Exception {
-
     // Test thrift server with HTTP header length less than 64k
     try {
       runThriftServer(1024 * 63);
@@ -147,8 +124,8 @@ public class TestThriftHttpServer {
     runThriftServer(1024 * 64);
   }
 
-  protected ThriftServer createThriftServer() {
-    return new ThriftServer(TEST_UTIL.getConfiguration());
+  protected Supplier<ThriftServer> getThriftServerSupplier() {
+    return () -> new ThriftServer(TEST_UTIL.getConfiguration());
   }
 
   @Test
@@ -157,48 +134,33 @@ public class TestThriftHttpServer {
   }
 
   void runThriftServer(int customHeaderSize) throws Exception {
-    for (int i = 0; i < 100; i++) {
-      List<String> args = new ArrayList<>(3);
-      port = HBaseTestingUtility.randomFreePort();
-      args.add("-" + PORT_OPTION);
-      args.add(String.valueOf(port));
-      args.add("-" + INFOPORT_OPTION);
-      int infoPort = HBaseTestingUtility.randomFreePort();
-      args.add(String.valueOf(infoPort));
-      args.add("start");
-
-      thriftServer = createThriftServer();
-      startHttpServerThread(args.toArray(new String[args.size()]));
-
-      // wait up to 10s for the server to start
-      HBaseTestingUtility.waitForHostPort(HConstants.LOCALHOST, port);
-
-      String url = "http://" + HConstants.LOCALHOST + ":" + port;
+    // Add retries in case we see stuff like connection reset
+    Exception clientSideException =  null;
+    for (int i = 0; i < 10; i++) {
+      clientSideException =  null;
+      ThriftServerRunner tsr = createBoundServer(getThriftServerSupplier());
+      String url = "http://" + HConstants.LOCALHOST + ":" + tsr.getThriftServer().listenPort;
       try {
         checkHttpMethods(url);
         talkToThriftServer(url, customHeaderSize);
+        break;
       } catch (Exception ex) {
         clientSideException = ex;
+        LOG.info("Client-side Exception", ex);
       } finally {
-        stopHttpServerThread();
-      }
-
-      if (clientSideException != null) {
-        LOG.error("Thrift client threw an exception " + clientSideException);
-        if (clientSideException instanceof TTransportException) {
-          if (clientSideException.getCause() != null &&
-            clientSideException.getCause() instanceof BindException) {
-            continue;
-          }
-          throw clientSideException;
-        } else {
-          throw new Exception(clientSideException);
+        tsr.close();
+        tsr.join();
+        if (tsr.getRunException() != null) {
+          LOG.error("Invocation of HBase Thrift server threw exception", tsr.getRunException());
+          throw tsr.getRunException();
         }
-      } else {
-        // Done.
-        break;
       }
     }
+
+    if (clientSideException != null) {
+      LOG.error("Thrift Client", clientSideException);
+      throw clientSideException;
+    }
   }
 
   private void checkHttpMethods(String url) throws Exception {
@@ -238,19 +200,4 @@ public class TestThriftHttpServer {
       httpClient.close();
     }
   }
-
-  private void stopHttpServerThread() throws Exception {
-    LOG.debug("Stopping Thrift HTTP server {}", thriftServer);
-    if (thriftServer != null) {
-      thriftServer.stop();
-    }
-    if (httpServerThread != null) {
-      httpServerThread.join();
-    }
-    if (httpServerException != null) {
-      LOG.error("Command-line invocation of HBase Thrift server threw an " +
-          "exception", httpServerException);
-      throw new Exception(httpServerException);
-    }
-  }
 }
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 794ebe5..02c2118 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
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -32,6 +32,7 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CompatibilityFactory;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -254,7 +255,9 @@ public class TestThriftServer {
 
   public static void createTestTables(Hbase.Iface handler) throws Exception {
     // Create/enable/disable/delete tables, ensure methods act correctly
-    assertEquals(0, handler.getTableNames().size());
+    List<java.nio.ByteBuffer> bbs = handler.getTableNames();
+    assertEquals(bbs.stream().map(b -> Bytes.toString(b.array())).
+      collect(Collectors.joining(",")), 0, bbs.size());
     handler.createTable(tableAname, getColumnDescriptors());
     assertEquals(1, handler.getTableNames().size());
     assertEquals(2, handler.getColumnDescriptors(tableAname).size());
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 db997fe..0d02d97 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
@@ -32,6 +32,7 @@ import java.net.ServerSocket;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import java.util.function.Supplier;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
@@ -84,14 +85,6 @@ public class TestThriftServerCmdLine {
   protected static final HBaseTestingUtility TEST_UTIL =
       new HBaseTestingUtility();
 
-  private Thread cmdLineThread;
-  private volatile Exception cmdLineException;
-
-  private Exception clientSideException;
-
-  private volatile ThriftServer thriftServer;
-  protected int port;
-
   @Parameters
   public static Collection<Object[]> getParameters() {
     Collection<Object[]> parameters = new ArrayList<>();
@@ -142,43 +135,43 @@ public class TestThriftServerCmdLine {
     EnvironmentEdgeManager.reset();
   }
 
-  private void startCmdLineThread(final String[] args) {
+  static ThriftServerRunner startCmdLineThread(Supplier<ThriftServer> supplier,
+      final String[] args) {
     LOG.info("Starting HBase Thrift server with command line: " + Joiner.on(" ").join(args));
+    ThriftServerRunner tsr = new ThriftServerRunner(supplier.get(), args);
+    tsr.setName(ThriftServer.class.getSimpleName() + "-cmdline");
+    tsr.start();
+    return tsr;
+  }
 
-    cmdLineException = null;
-    cmdLineThread = new Thread(new Runnable() {
-      @Override
-      public void run() {
-        try {
-          thriftServer.run(args);
-        } catch (Exception e) {
-          LOG.error("Error when start thrift server", e);
-          cmdLineException = e;
-        }
-      }
-    });
-    cmdLineThread.setName(ThriftServer.class.getSimpleName() +
-        "-cmdline");
-    cmdLineThread.start();
+  static int getRandomPort() {
+    return HBaseTestingUtility.randomFreePort();
   }
 
-  protected ThriftServer createThriftServer() {
-    return new ThriftServer(TEST_UTIL.getConfiguration());
+  protected Supplier<ThriftServer> getThriftServerSupplier() {
+    return () -> new ThriftServer(TEST_UTIL.getConfiguration());
   }
 
-  private int getRandomPort() {
-    return HBaseTestingUtility.randomFreePort();
+  static ThriftServerRunner createBoundServer(Supplier<ThriftServer> thriftServerSupplier)
+      throws Exception {
+    return createBoundServer(thriftServerSupplier, false, false);
   }
 
-  /**
-   * Server can fail to bind if clashing address. Add retrying until we get a good server.
-   */
-  ThriftServer createBoundServer() {
-    return createBoundServer(false, false);
+  static ThriftServerRunner createBoundServer(Supplier<ThriftServer> thriftServerSupplier,
+      boolean protocolPortClash, boolean infoPortClash) throws Exception {
+    return createBoundServer(thriftServerSupplier, null, false, false, false, protocolPortClash, infoPortClash);
   }
 
-  private ServerSocket getBoundSocket() {
+  static ThriftServerRunner createBoundServer(Supplier<ThriftServer> thriftServerSupplier,
+      ImplType implType, boolean specifyFramed, boolean specifyCompact, boolean specifyBindIP)
+      throws Exception {
+    return createBoundServer(thriftServerSupplier, implType, specifyFramed, specifyCompact,
+      specifyBindIP, false, false);
+  }
+
+  static ServerSocket getBoundSocket() {
     ServerSocket ss = null;
+    int port = -1;
     while (true) {
       port = getRandomPort();
       try {
@@ -202,7 +195,9 @@ public class TestThriftServerCmdLine {
    *   the code does the right thing when this happens during actual test runs. Ugly but works.
    * @see TestBindExceptionHandling#testProtocolPortClash()
    */
-  ThriftServer createBoundServer(boolean protocolPortClash, boolean infoPortClash) {
+  static ThriftServerRunner createBoundServer(Supplier<ThriftServer> thriftServerSupplier,
+      ImplType implType, boolean specifyFramed, boolean specifyCompact, boolean specifyBindIP,
+      boolean protocolPortClash, boolean infoPortClash) throws Exception {
     if (protocolPortClash && infoPortClash) {
       throw new RuntimeException("Can't set both at same time");
     }
@@ -210,6 +205,8 @@ public class TestThriftServerCmdLine {
     boolean testClashOfFirstInfoPort = infoPortClash;
     List<String> args = new ArrayList<>();
     ServerSocket ss = null;
+    int port = -1;
+    ThriftServerRunner tsr = null;
     for (int i = 0; i < 100; i++) {
       args.clear();
       if (implType != null) {
@@ -251,16 +248,20 @@ public class TestThriftServerCmdLine {
       }
       args.add("start");
 
-      thriftServer = createThriftServer();
-      startCmdLineThread(args.toArray(new String[args.size()]));
+      tsr = startCmdLineThread(thriftServerSupplier, args.toArray(new String[args.size()]));
       // wait up to 10s for the server to start
-      for (int ii = 0; ii < 100 && (thriftServer.tserver == null); ii++) {
+      for (int ii = 0; ii < 100 && (tsr.getThriftServer().tserver == null &&
+          tsr.getRunException() == null); ii++) {
         Threads.sleep(100);
       }
-      if (isBindException(cmdLineException)) {
-        LOG.info("BindException; trying new port", cmdLineException);
-        cmdLineException =  null;
-        thriftServer.stop();
+      if (isBindException(tsr.getRunException())) {
+        LOG.info("BindException; trying new port", tsr.getRunException());
+        try {
+          tsr.close();
+          tsr.join();
+        } catch (IOException | InterruptedException ioe) {
+          LOG.warn("Exception closing", ioe);
+        }
         continue;
       }
       break;
@@ -272,14 +273,18 @@ public class TestThriftServerCmdLine {
         LOG.warn("Failed close", ioe);
       }
     }
-    Class<? extends TServer> expectedClass = implType != null ?
-      implType.serverClass : TBoundedThreadPoolServer.class;
-    assertEquals(expectedClass, thriftServer.tserver.getClass());
-    LOG.info("Server={}", args);
-    return thriftServer;
+    if (tsr.getRunException() != null) {
+      throw tsr.getRunException();
+    }
+    if (tsr.getThriftServer().tserver != null) {
+      Class<? extends TServer> expectedClass =
+        implType != null ? implType.serverClass : TBoundedThreadPoolServer.class;
+      assertEquals(expectedClass, tsr.getThriftServer().tserver.getClass());
+    }
+    return tsr;
   }
 
-  private boolean isBindException(Exception cmdLineException) {
+  private static boolean isBindException(Exception cmdLineException) {
     if (cmdLineException == null) {
       return false;
     }
@@ -295,32 +300,40 @@ public class TestThriftServerCmdLine {
 
   @Test
   public void testRunThriftServer() throws Exception {
-    ThriftServer thriftServer = createBoundServer();
     // Add retries in case we see stuff like connection reset
+    Exception clientSideException =  null;
     for (int i = 0; i < 10; i++) {
+      clientSideException =  null;
+      ThriftServerRunner thriftServerRunner = createBoundServer(getThriftServerSupplier(),
+        this.implType, this.specifyFramed, this.specifyCompact, this.specifyBindIP);
       try {
-        talkToThriftServer();
+        talkToThriftServer(thriftServerRunner.getThriftServer().listenPort);
         break;
       } catch (Exception ex) {
         clientSideException = ex;
         LOG.info("Exception", ex);
       } finally {
-        stopCmdLineThread();
-        thriftServer.stop();
+        LOG.debug("Stopping " + this.implType.simpleClassName() + " Thrift server");
+        thriftServerRunner.close();
+        thriftServerRunner.join();
+        if (thriftServerRunner.getRunException() != null) {
+           LOG.error("Command-line invocation of HBase Thrift server threw exception",
+             thriftServerRunner.getRunException());
+           throw thriftServerRunner.getRunException();
+        }
       }
     }
 
     if (clientSideException != null) {
-      LOG.error("Thrift client threw an exception. Parameters:" +
-          getParametersString(), clientSideException);
+      LOG.error("Thrift Client; parameters={}", getParametersString(), clientSideException);
       throw new Exception(clientSideException);
     }
   }
 
   protected static volatile boolean tableCreated = false;
 
-  protected void talkToThriftServer() throws Exception {
-    LOG.info("Talking to port=" + this.port);
+  protected void talkToThriftServer(int port) throws Exception {
+    LOG.info("Talking to port={}", port);
     TSocket sock = new TSocket(InetAddress.getLoopbackAddress().getHostName(), port);
     TTransport transport = sock;
     if (specifyFramed || implType.isAlwaysFramed) {
@@ -347,16 +360,5 @@ public class TestThriftServerCmdLine {
       sock.close();
     }
   }
-
-  private void stopCmdLineThread() throws Exception {
-    LOG.debug("Stopping " + implType.simpleClassName() + " Thrift server");
-    thriftServer.stop();
-    cmdLineThread.join();
-    if (cmdLineException != null) {
-      LOG.error("Command-line invocation of HBase Thrift server threw an " +
-          "exception", cmdLineException);
-      throw new Exception(cmdLineException);
-    }
-  }
 }
 
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpFallbackServer.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpFallbackServer.java
index a1334a4..6f89c3c 100644
--- a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpFallbackServer.java
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftSpnegoHttpFallbackServer.java
@@ -26,6 +26,7 @@ import java.nio.file.Paths;
 import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
 import java.util.Set;
+import java.util.function.Supplier;
 
 import javax.security.auth.Subject;
 import javax.security.auth.kerberos.KerberosTicket;
@@ -62,6 +63,7 @@ import org.ietf.jgss.Oid;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -233,4 +235,20 @@ public class TestThriftSpnegoHttpFallbackServer extends TestThriftHttpServer {
         .build();
     });
   }
+
+  @Override protected Supplier<ThriftServer> getThriftServerSupplier() {
+    return () -> new ThriftServer(TEST_UTIL.getConfiguration());
+  }
+
+  /**
+   * Block call through to this method. It is a messy test that fails because of bad config
+   * and then succeeds only the first attempt adds a table which the second attempt doesn't
+   * want to be in place to succeed. Let the super impl of this test be responsible for
+   * verifying we fail if bad header size.
+   */
+  @org.junit.Ignore
+  @Test
+  @Override public void testRunThriftServerWithHeaderBufferLength() throws Exception {
+    super.testRunThriftServerWithHeaderBufferLength();
+  }
 }
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 3206b54..1eeaa10 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,14 +18,17 @@
 package org.apache.hadoop.hbase.thrift;
 
 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.*;
 
 import java.io.File;
+import java.nio.ByteBuffer;
 import java.nio.file.Paths;
 import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
+import java.util.List;
 import java.util.Set;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
 
 import javax.security.auth.Subject;
 import javax.security.auth.kerberos.KerberosTicket;
@@ -38,6 +41,7 @@ import org.apache.hadoop.hbase.security.HBaseKerberosUtils;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.thrift.generated.Hbase;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.http.HttpHeaders;
 import org.apache.http.auth.AuthSchemeProvider;
@@ -62,6 +66,7 @@ import org.ietf.jgss.Oid;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -153,6 +158,10 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
     TestThriftHttpServer.setUpBeforeClass();
   }
 
+  @Override protected Supplier<ThriftServer> getThriftServerSupplier() {
+    return () -> new ThriftServer(TEST_UTIL.getConfiguration());
+  }
+
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     TestThriftHttpServer.tearDownAfterClass();
@@ -167,6 +176,18 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
     }
   }
 
+  /**
+   * Block call through to this method. It is a messy test that fails because of bad config
+   * and then succeeds only the first attempt adds a table which the second attempt doesn't
+   * want to be in place to succeed. Let the super impl of this test be responsible for
+   * verifying we fail if bad header size.
+   */
+  @org.junit.Ignore
+  @Test
+  @Override public void testRunThriftServerWithHeaderBufferLength() throws Exception {
+    super.testRunThriftServerWithHeaderBufferLength();
+  }
+
   @Override
   protected void talkToThriftServer(String url, int customHeaderSize) throws Exception {
     // Close httpClient and THttpClient automatically on any failures
@@ -185,6 +206,15 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
 
       TProtocol prot = new TBinaryProtocol(tHttpClient);
       Hbase.Client client = new Hbase.Client(prot);
+      List<ByteBuffer> bbs = client.getTableNames();
+      LOG.info("PRE-EXISTING {}", bbs.stream().
+        map(b -> Bytes.toString(b.array())).collect(Collectors.joining(",")));
+      if (!bbs.isEmpty()) {
+        for (ByteBuffer bb: bbs) {
+          client.disableTable(bb);
+          client.deleteTable(bb);
+        }
+      }
       TestThriftServer.createTestTables(client);
       TestThriftServer.checkTableList(client);
       TestThriftServer.dropTestTables(client);
diff --git a/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
new file mode 100644
index 0000000..a006045
--- /dev/null
+++ b/hbase-thrift/src/test/java/org/apache/hadoop/hbase/thrift/ThriftServerRunner.java
@@ -0,0 +1,66 @@
+/*
+ * 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.thrift;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * Run ThriftServer with passed arguments. Access the exception thrown after we complete run -- if
+ * an exception thrown -- via {@link #getRunException()}}. Call close to shutdown this Runner
+ * and hosted {@link ThriftServer}.
+ */
+class ThriftServerRunner extends Thread implements Closeable {
+  private static final Logger LOG = LoggerFactory.getLogger(ThriftServerRunner.class);
+  Exception exception = null;
+  private final ThriftServer thriftServer;
+  private final String [] args;
+
+  ThriftServerRunner(ThriftServer thriftServer, String [] args) {
+    this.thriftServer = thriftServer;
+    this.args = args;
+    LOG.info("thriftServer={}, args={}", getThriftServer(), args);
+  }
+
+  ThriftServer getThriftServer() {
+    return this.thriftServer;
+  }
+
+  /**
+   * @return Empty unless {@link #run()} threw an exception; if it did, access it here.
+   */
+  Exception getRunException() {
+    return this.exception;
+  }
+
+  @Override public void run() {
+    try {
+      this.thriftServer.run(this.args);
+    } catch (Exception e) {
+      LOG.error("Run threw an exception", e);
+      this.exception = e;
+    }
+  }
+
+  @Override public void close() throws IOException {
+    LOG.info("Stopping {}", this);
+    this.thriftServer.stop();
+  }
+}
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
index cf084c9..d5a05ed 100644
--- 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
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -18,11 +18,13 @@
 package org.apache.hadoop.hbase.thrift2;
 
 import java.util.ArrayList;
+import java.util.function.Supplier;
 
 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.thrift.ThriftServer;
 import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
 import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
@@ -43,11 +45,8 @@ public class TestThrift2HttpServer extends TestThriftHttpServer {
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestThrift2HttpServer.class);
 
-
-
-  @Override
-  protected ThriftServer createThriftServer() {
-    return new ThriftServer(TEST_UTIL.getConfiguration());
+  @Override protected Supplier<ThriftServer> getThriftServerSupplier() {
+    return () -> new org.apache.hadoop.hbase.thrift2.ThriftServer(TEST_UTIL.getConfiguration());
   }
 
   @Override
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
index 1b8ed2d..47cc053 100644
--- 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
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -19,12 +19,14 @@ package org.apache.hadoop.hbase.thrift2;
 
 import java.net.InetAddress;
 import java.util.ArrayList;
+import java.util.function.Supplier;
 
 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.thrift.ThriftServer;
 import org.apache.hadoop.hbase.thrift2.generated.TColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
 import org.apache.hadoop.hbase.thrift2.generated.TTableDescriptor;
@@ -48,19 +50,17 @@ public class TestThrift2ServerCmdLine extends TestThriftServerCmdLine {
 
   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 Supplier<ThriftServer> getThriftServerSupplier() {
+    return () -> new org.apache.hadoop.hbase.thrift2.ThriftServer(TEST_UTIL.getConfiguration());
+  }
+
   @Override
-  protected void talkToThriftServer() throws Exception {
+  protected void talkToThriftServer(int port) throws Exception {
     TSocket sock = new TSocket(InetAddress.getLoopbackAddress().getHostName(), port);
     TTransport transport = sock;
     if (specifyFramed || implType.isAlwaysFramed()) {