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/06 04:38:12 UTC

[hbase] branch branch-2.3 updated: HBASE-24307 [Flakey Tests] krb server for secure thrift tests throws BindException (#1656)

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

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


The following commit(s) were added to refs/heads/branch-2.3 by this push:
     new 7c92e14    HBASE-24307 [Flakey Tests] krb server for secure thrift tests throws BindException (#1656)
7c92e14 is described below

commit 7c92e145b9739de50905814df99f5039924d4fb5
Author: Michael Stack <sa...@users.noreply.github.com>
AuthorDate: Tue May 5 21:08:09 2020 -0700

      HBASE-24307 [Flakey Tests] krb server for secure thrift tests throws BindException (#1656)
    
    Move the random free port generate back into hbasecommontestingutility
      from hbasetestingutility.
    
      Add a create simple kdc server utility that will start a kdc server and
      if a bindexception, create a new one on a new random port in hbase-common.
    
      Add new BoundSocketMaker helpful when trying to manufacture
      BindExceptions because of port clash.
    
      Change thrift and http kdc tests to use this new utility (removes
      code duplication around kdc server setup).
---
 hbase-common/pom.xml                               |   5 +
 .../hadoop/hbase/HBaseCommonTestingUtility.java    |  79 ++++++++++++++++
 .../apache/hadoop/hbase/net/BoundSocketMaker.java  |  85 +++++++++++++++++
 .../hadoop/hbase/util/SimpleKdcServerUtil.java     | 103 +++++++++++++++++++++
 .../hadoop/hbase/util/TestSimpleKdcServerUtil.java |  55 +++++++++++
 .../hbase/http/TestProxyUserSpnegoHttpServer.java  |  34 ++-----
 .../hadoop/hbase/http/TestSpnegoHttpServer.java    |  29 +-----
 .../apache/hadoop/hbase/HBaseTestingUtility.java   |  76 +--------------
 .../hadoop/hbase/http/TestInfoServersACL.java      |   1 -
 .../hbase/thrift/TestThriftServerCmdLine.java      |  47 +++-------
 .../thrift/TestThriftSpnegoHttpFallbackServer.java |  32 ++-----
 .../hbase/thrift/TestThriftSpnegoHttpServer.java   |  43 +++------
 12 files changed, 371 insertions(+), 218 deletions(-)

diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml
index f3f382e..9520d07 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -211,6 +211,11 @@
       <groupId>log4j</groupId>
       <artifactId>log4j</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.kerby</groupId>
+      <artifactId>kerb-simplekdc</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <profiles>
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java
index e8a2a79..eb04cca 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java
@@ -19,8 +19,12 @@ package org.apache.hadoop.hbase;
 
 import java.io.File;
 import java.io.IOException;
+import java.net.ServerSocket;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Random;
+import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ThreadLocalRandom;
 import org.apache.commons.io.FileUtils;
@@ -264,4 +268,79 @@ public class HBaseCommonTestingUtility {
       boolean failIfTimeout, Predicate<E> predicate) throws E {
     return Waiter.waitFor(this.conf, timeout, interval, failIfTimeout, predicate);
   }
+
+  // Support for Random Port Generation.
+  static Random random = new Random();
+
+  private static final PortAllocator portAllocator = new PortAllocator(random);
+
+  public static int randomFreePort() {
+    return portAllocator.randomFreePort();
+  }
+
+  static class PortAllocator {
+    private static final int MIN_RANDOM_PORT = 0xc000;
+    private static final int MAX_RANDOM_PORT = 0xfffe;
+
+    /** A set of ports that have been claimed using {@link #randomFreePort()}. */
+    private final Set<Integer> takenRandomPorts = new HashSet<>();
+
+    private final Random random;
+    private final AvailablePortChecker portChecker;
+
+    public PortAllocator(Random random) {
+      this.random = random;
+      this.portChecker = new AvailablePortChecker() {
+        @Override
+        public boolean available(int port) {
+          try {
+            ServerSocket sock = new ServerSocket(port);
+            sock.close();
+            return true;
+          } catch (IOException ex) {
+            return false;
+          }
+        }
+      };
+    }
+
+    public PortAllocator(Random random, AvailablePortChecker portChecker) {
+      this.random = random;
+      this.portChecker = portChecker;
+    }
+
+    /**
+     * Returns a random free port and marks that port as taken. Not thread-safe. Expected to be
+     * called from single-threaded test setup code/
+     */
+    public int randomFreePort() {
+      int port = 0;
+      do {
+        port = randomPort();
+        if (takenRandomPorts.contains(port)) {
+          port = 0;
+          continue;
+        }
+        takenRandomPorts.add(port);
+
+        if (!portChecker.available(port)) {
+          port = 0;
+        }
+      } while (port == 0);
+      return port;
+    }
+
+    /**
+     * Returns a random port. These ports cannot be registered with IANA and are
+     * intended for dynamic allocation (see http://bit.ly/dynports).
+     */
+    private int randomPort() {
+      return MIN_RANDOM_PORT
+        + random.nextInt(MAX_RANDOM_PORT - MIN_RANDOM_PORT);
+    }
+
+    interface AvailablePortChecker {
+      boolean available(int port);
+    }
+  }
 }
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/net/BoundSocketMaker.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/net/BoundSocketMaker.java
new file mode 100644
index 0000000..208c11a
--- /dev/null
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/net/BoundSocketMaker.java
@@ -0,0 +1,85 @@
+/*
+ * 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.net;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.util.function.Supplier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility to generate a bound socket. Useful testing for BindException.
+ * Use one of the Constructors to create an instance of this class. On creation it will have put
+ * up a ServerSocket on a random port. Get the port it is bound to using {@link #getPort()}. In
+ * your test, then try to start a Server using same port to generate a BindException. Call
+ * {@link #close()} when done to shut down the Socket.
+ */
+public final class BoundSocketMaker implements Closeable {
+  private static final Logger LOG = LoggerFactory.getLogger(BoundSocketMaker.class);
+  private final ServerSocket socket;
+
+  private BoundSocketMaker() {
+    this.socket = null;
+  }
+
+  public BoundSocketMaker(Supplier<Integer> randomPortMaker) {
+    this(InetAddress.getLoopbackAddress().getHostName(), randomPortMaker);
+  }
+
+  public BoundSocketMaker(final String hostname, Supplier<Integer> randomPortMaker) {
+    this.socket = get(hostname, randomPortMaker);
+  }
+
+  public int getPort() {
+    return this.socket.getLocalPort();
+  }
+
+  /**
+   * @return Returns a bound socket; be sure to close when done.
+   */
+  private ServerSocket get(String hostname, Supplier<Integer> randomPortMaker) {
+    ServerSocket ss = null;
+    int port = -1;
+    while (true) {
+      port = randomPortMaker.get();
+      try {
+        ss = new ServerSocket();
+        ss.bind(new InetSocketAddress(hostname, port));
+        break;
+      } catch (IOException ioe) {
+        LOG.warn("Failed bind", ioe);
+        try {
+          ss.close();
+        } catch (IOException ioe2) {
+          LOG.warn("FAILED CLOSE of failed bind socket", ioe2);
+        }
+      }
+    }
+    return ss;
+  }
+
+  @Override public void close() throws IOException {
+    if (this.socket != null) {
+      this.socket.close();
+    }
+  }
+}
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/SimpleKdcServerUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/SimpleKdcServerUtil.java
new file mode 100644
index 0000000..32ffa7f
--- /dev/null
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/SimpleKdcServerUtil.java
@@ -0,0 +1,103 @@
+/*
+ * 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.util;
+import java.io.File;
+import java.io.IOException;
+import java.net.BindException;
+import java.net.InetAddress;
+import java.util.function.Supplier;
+import org.apache.hadoop.hbase.net.BoundSocketMaker;
+import org.apache.kerby.kerberos.kerb.KrbException;
+import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+
+/**
+ * Utility for running {@link SimpleKdcServer}. Kerby KDC server is favored over Hadoop
+ * org.apache.hadoop.minikdc server which has support in the HBaseTestingUtility at
+ * #setupMiniKdc. The Kerby KDC Server came in with HBASE-5291. Its preferred. Less baggage.
+ * @see #getRunningSimpleKdcServer(File, Supplier)
+ */
+public final class SimpleKdcServerUtil {
+  protected static final Logger LOG = LoggerFactory.getLogger(SimpleKdcServerUtil.class);
+
+  private SimpleKdcServerUtil() {}
+
+  /**
+   * Returns a running kdc server. Use this method rather than start the SimpleKdcServer
+   * yourself because it takes care of BindExceptions which can happen even though port-picking
+   * is random (between the choice of port number and bind, it could have been used elsewhere).
+   * @return A SimpleKdcServer on which 'start' has been called; be sure to call stop on this
+   *   instance when done.
+   */
+  public static SimpleKdcServer getRunningSimpleKdcServer(File testDir,
+      Supplier<Integer> randomPortGenerator) throws KrbException, IOException {
+    return getRunningSimpleKdcServer(testDir, randomPortGenerator, false);
+  }
+
+  /**
+   * Internal method for testing.
+   * @param portClash True if we want to generate BindException (for testing).
+   * @return A running SimpleKdcServer on loopback/'localhost' on a random port
+   * @see #getRunningSimpleKdcServer(File, Supplier)
+   */
+  @VisibleForTesting
+  static SimpleKdcServer getRunningSimpleKdcServer(File testDir,
+      Supplier<Integer> randomPortGenerator, final boolean portClash)
+        throws KrbException, IOException {
+    File kdcDir = new File(testDir, SimpleKdcServer.class.getSimpleName());
+    Preconditions.checkArgument(kdcDir.mkdirs(), "Failed create of " + kdcDir);
+    String hostName = InetAddress.getLoopbackAddress().getHostName();
+    BoundSocketMaker bsm = portClash? new BoundSocketMaker(randomPortGenerator): null;
+    final int retries = 10;
+    for (int i = 0; i < retries; i++) {
+      SimpleKdcServer kdc = new SimpleKdcServer();
+      kdc.setWorkDir(kdcDir);
+      kdc.setKdcHost(hostName);
+      kdc.setAllowTcp(true);
+      kdc.setAllowUdp(false);
+      int kdcPort = bsm != null? bsm.getPort(): randomPortGenerator.get();
+      try {
+        kdc.setKdcTcpPort(kdcPort);
+        LOG.info("Starting KDC server at {}:{}", hostName, kdcPort);
+        kdc.init();
+        kdc.start();
+        return kdc;
+      } catch (KrbException ke) {
+        if (kdc != null) {
+          kdc.stop();
+        }
+        if (ke.getCause() != null && ke.getCause() instanceof BindException) {
+          LOG.info("Clashed using port {}; getting a new random port", kdcPort);
+          continue;
+        } else {
+          throw ke;
+        }
+      } finally {
+        if (bsm != null) {
+          bsm.close();
+          bsm = null;
+        }
+      }
+    }
+    // If we get here, we exhausted our retries. Fail.
+    throw new KrbException("Failed create of SimpleKdcServer after " + retries + " attempts");
+  }
+}
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleKdcServerUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleKdcServerUtil.java
new file mode 100644
index 0000000..1ebe82f
--- /dev/null
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/util/TestSimpleKdcServerUtil.java
@@ -0,0 +1,55 @@
+/*
+ * 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.util;
+
+import java.io.File;
+import java.io.IOException;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.kerby.kerberos.kerb.KrbException;
+import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MiscTests.class, MediumTests.class})
+public class TestSimpleKdcServerUtil {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestSimpleKdcServerUtil.class);
+  private static final HBaseCommonTestingUtility UTIL = new HBaseCommonTestingUtility();
+
+  /**
+   * Test we are able to ride over clashing port... BindException.. when starting up a
+   * kdc server.
+   */
+  @Test
+  public void testBindException() throws KrbException, IOException {
+    SimpleKdcServer kdc = null;
+    try {
+      File dir = new File(UTIL.getDataTestDir().toString());
+      kdc = SimpleKdcServerUtil.
+        getRunningSimpleKdcServer(dir, HBaseCommonTestingUtility::randomFreePort, true);
+      kdc.createPrincipal("wah");
+    } finally {
+      kdc.stop();
+    }
+  }
+}
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestProxyUserSpnegoHttpServer.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestProxyUserSpnegoHttpServer.java
index 2036b31..804c837 100644
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestProxyUserSpnegoHttpServer.java
+++ b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestProxyUserSpnegoHttpServer.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.http.TestHttpServer.EchoServlet;
 import org.apache.hadoop.hbase.http.resource.JerseyResource;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.SimpleKdcServerUtil;
 import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.http.HttpHost;
@@ -69,7 +70,7 @@ import org.slf4j.LoggerFactory;
 /**
  * Test class for SPNEGO Proxyuser authentication on the HttpServer. Uses Kerby's MiniKDC and Apache
  * HttpComponents to verify that the doas= mechanicsm works, and that the proxyuser settings are
- * observed
+ * observed.
  */
 @Category({MiscTests.class, SmallTests.class})
 public class TestProxyUserSpnegoHttpServer extends HttpServerFunctionalTest {
@@ -101,8 +102,8 @@ public class TestProxyUserSpnegoHttpServer extends HttpServerFunctionalTest {
 
     final String serverPrincipal = "HTTP/" + KDC_SERVER_HOST;
 
-    kdc = buildMiniKdc();
-    kdc.start();
+    kdc = SimpleKdcServerUtil.getRunningSimpleKdcServer(new File(htu.getDataTestDir().toString()),
+      HBaseCommonTestingUtility::randomFreePort);
     File keytabDir = new File(htu.getDataTestDir("keytabs").toString());
     if (keytabDir.exists()) {
       deleteRecursively(keytabDir);
@@ -158,32 +159,9 @@ public class TestProxyUserSpnegoHttpServer extends HttpServerFunctionalTest {
     kdc.exportPrincipal(principal, keytab);
   }
 
-  private static SimpleKdcServer buildMiniKdc() throws Exception {
-    SimpleKdcServer kdc = new SimpleKdcServer();
 
-    final File target = new File(System.getProperty("user.dir"), "target");
-    File kdcDir = new File(target, TestProxyUserSpnegoHttpServer.class.getSimpleName());
-    if (kdcDir.exists()) {
-      deleteRecursively(kdcDir);
-    }
-    kdcDir.mkdirs();
-    kdc.setWorkDir(kdcDir);
-
-    kdc.setKdcHost(KDC_SERVER_HOST);
-    int kdcPort = getFreePort();
-    kdc.setAllowTcp(true);
-    kdc.setAllowUdp(false);
-    kdc.setKdcTcpPort(kdcPort);
-
-    LOG.info("Starting KDC server at " + KDC_SERVER_HOST + ":" + kdcPort);
-
-    kdc.init();
-
-    return kdc;
-  }
-
-  protected static Configuration buildSpnegoConfiguration(Configuration conf, String serverPrincipal,
-      File serverKeytab) {
+  protected static Configuration buildSpnegoConfiguration(Configuration conf,
+      String serverPrincipal, File serverKeytab) {
     KerberosName.setRules("DEFAULT");
 
     conf.setInt(HttpServer.HTTP_MAX_THREADS, TestHttpServer.MAX_THREADS);
diff --git a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java
index 680fff1..28e4fcf 100644
--- a/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java
+++ b/hbase-http/src/test/java/org/apache/hadoop/hbase/http/TestSpnegoHttpServer.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.http.TestHttpServer.EchoServlet;
 import org.apache.hadoop.hbase.http.resource.JerseyResource;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.SimpleKdcServerUtil;
 import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.http.HttpHost;
 import org.apache.http.HttpResponse;
@@ -91,8 +92,8 @@ public class TestSpnegoHttpServer extends HttpServerFunctionalTest {
 
     final String serverPrincipal = "HTTP/" + KDC_SERVER_HOST;
 
-    kdc = buildMiniKdc();
-    kdc.start();
+    kdc = SimpleKdcServerUtil.getRunningSimpleKdcServer(new File(htu.getDataTestDir().toString()),
+      HBaseCommonTestingUtility::randomFreePort);
     File keytabDir = new File(htu.getDataTestDir("keytabs").toString());
     if (keytabDir.exists()) {
       deleteRecursively(keytabDir);
@@ -140,30 +141,6 @@ public class TestSpnegoHttpServer extends HttpServerFunctionalTest {
     kdc.exportPrincipal(principal, keytab);
   }
 
-  private static SimpleKdcServer buildMiniKdc() throws Exception {
-    SimpleKdcServer kdc = new SimpleKdcServer();
-
-    final File target = new File(System.getProperty("user.dir"), "target");
-    File kdcDir = new File(target, TestSpnegoHttpServer.class.getSimpleName());
-    if (kdcDir.exists()) {
-      deleteRecursively(kdcDir);
-    }
-    kdcDir.mkdirs();
-    kdc.setWorkDir(kdcDir);
-
-    kdc.setKdcHost(KDC_SERVER_HOST);
-    int kdcPort = getFreePort();
-    kdc.setAllowTcp(true);
-    kdc.setAllowUdp(false);
-    kdc.setKdcTcpPort(kdcPort);
-
-    LOG.info("Starting KDC server at " + KDC_SERVER_HOST + ":" + kdcPort);
-
-    kdc.init();
-
-    return kdc;
-  }
-
   private static Configuration buildSpnegoConfiguration(Configuration conf, String serverPrincipal,
       File serverKeytab) {
     KerberosName.setRules("DEFAULT");
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 4c22e18..6b36ff1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -3939,80 +3939,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     return table;
   }
 
-  private static Random random = new Random();
-
-  private static final PortAllocator portAllocator = new PortAllocator(random);
-
   public static int randomFreePort() {
-    return portAllocator.randomFreePort();
+    return HBaseCommonTestingUtility.randomFreePort();
   }
-
-  static class PortAllocator {
-    private static final int MIN_RANDOM_PORT = 0xc000;
-    private static final int MAX_RANDOM_PORT = 0xfffe;
-
-    /** A set of ports that have been claimed using {@link #randomFreePort()}. */
-    private final Set<Integer> takenRandomPorts = new HashSet<>();
-
-    private final Random random;
-    private final AvailablePortChecker portChecker;
-
-    public PortAllocator(Random random) {
-      this.random = random;
-      this.portChecker = new AvailablePortChecker() {
-        @Override
-        public boolean available(int port) {
-          try {
-            ServerSocket sock = new ServerSocket(port);
-            sock.close();
-            return true;
-          } catch (IOException ex) {
-            return false;
-          }
-        }
-      };
-    }
-
-    public PortAllocator(Random random, AvailablePortChecker portChecker) {
-      this.random = random;
-      this.portChecker = portChecker;
-    }
-
-    /**
-     * Returns a random free port and marks that port as taken. Not thread-safe. Expected to be
-     * called from single-threaded test setup code/
-     */
-    public int randomFreePort() {
-      int port = 0;
-      do {
-        port = randomPort();
-        if (takenRandomPorts.contains(port)) {
-          port = 0;
-          continue;
-        }
-        takenRandomPorts.add(port);
-
-        if (!portChecker.available(port)) {
-          port = 0;
-        }
-      } while (port == 0);
-      return port;
-    }
-
-    /**
-     * Returns a random port. These ports cannot be registered with IANA and are
-     * intended for dynamic allocation (see http://bit.ly/dynports).
-     */
-    private int randomPort() {
-      return MIN_RANDOM_PORT
-          + random.nextInt(MAX_RANDOM_PORT - MIN_RANDOM_PORT);
-    }
-
-    interface AvailablePortChecker {
-      boolean available(int port);
-    }
-  }
-
   public static String randomMultiCastAddress() {
     return "226.1.1." + random.nextInt(254);
   }
@@ -4481,6 +4410,9 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
    * Sets up {@link MiniKdc} for testing security.
    * Uses {@link HBaseKerberosUtils} to set the given keytab file as
    * {@link HBaseKerberosUtils#KRB_KEYTAB_FILE}.
+   * FYI, there is also the easier-to-use kerby KDC server and utility for using it,
+   * {@link org.apache.hadoop.hbase.util.SimpleKdcServerUtil}. The kerby KDC server is preferred;
+   * less baggage. It came in in HBASE-5291.
    */
   public MiniKdc setupMiniKdc(File keytabFile) throws Exception {
     Properties conf = MiniKdc.createConf();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
index a1e1b91..4016e7b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.http;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-
 import java.io.File;
 import java.lang.management.ManagementFactory;
 import java.net.HttpURLConnection;
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 0d02d97..248d9ad 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
@@ -27,14 +27,13 @@ import static org.junit.Assert.assertTrue;
 import java.io.IOException;
 import java.net.BindException;
 import java.net.InetAddress;
-import java.net.InetSocketAddress;
-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.net.BoundSocketMaker;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.thrift.generated.Hbase;
@@ -159,7 +158,8 @@ public class TestThriftServerCmdLine {
 
   static ThriftServerRunner createBoundServer(Supplier<ThriftServer> thriftServerSupplier,
       boolean protocolPortClash, boolean infoPortClash) throws Exception {
-    return createBoundServer(thriftServerSupplier, null, false, false, false, protocolPortClash, infoPortClash);
+    return createBoundServer(thriftServerSupplier, null, false, false,
+      false, protocolPortClash, infoPortClash);
   }
 
   static ThriftServerRunner createBoundServer(Supplier<ThriftServer> thriftServerSupplier,
@@ -169,27 +169,6 @@ public class TestThriftServerCmdLine {
       specifyBindIP, false, false);
   }
 
-  static ServerSocket getBoundSocket() {
-    ServerSocket ss = null;
-    int port = -1;
-    while (true) {
-      port = getRandomPort();
-      try {
-        ss = new ServerSocket();
-        ss.bind(new InetSocketAddress(port));
-        break;
-      } catch (IOException ioe) {
-        LOG.warn("Failed bind", ioe);
-        try {
-          ss.close();
-        } catch (IOException ioe2) {
-          LOG.warn("FAILED CLOSE of failed bind socket", ioe2);
-        }
-      }
-    }
-    return ss;
-  }
-
   /**
    * @param protocolPortClash This param is just so we can manufacture a port clash so we can test
    *   the code does the right thing when this happens during actual test runs. Ugly but works.
@@ -204,7 +183,7 @@ public class TestThriftServerCmdLine {
     boolean testClashOfFirstProtocolPort = protocolPortClash;
     boolean testClashOfFirstInfoPort = infoPortClash;
     List<String> args = new ArrayList<>();
-    ServerSocket ss = null;
+    BoundSocketMaker bsm = null;
     int port = -1;
     ThriftServerRunner tsr = null;
     for (int i = 0; i < 100; i++) {
@@ -217,8 +196,8 @@ public class TestThriftServerCmdLine {
       if (testClashOfFirstProtocolPort) {
         // Test what happens if already something bound to the socket.
         // Occupy the random port we just pulled.
-        ss = getBoundSocket();
-        port = ss.getLocalPort();
+        bsm = new BoundSocketMaker(() -> getRandomPort());
+        port = bsm.getPort();
         testClashOfFirstProtocolPort = false;
       } else {
         port = getRandomPort();
@@ -228,8 +207,8 @@ public class TestThriftServerCmdLine {
       args.add("-" + INFOPORT_OPTION);
       int infoPort;
       if (testClashOfFirstInfoPort) {
-        ss = getBoundSocket();
-        infoPort = ss.getLocalPort();
+        bsm = new BoundSocketMaker(() -> getRandomPort());
+        infoPort = bsm.getPort();
         testClashOfFirstInfoPort = false;
       } else {
         infoPort = getRandomPort();
@@ -266,9 +245,9 @@ public class TestThriftServerCmdLine {
       }
       break;
     }
-    if (ss != null) {
+    if (bsm != null) {
       try {
-        ss.close();
+        bsm.close();
       } catch (IOException ioe) {
         LOG.warn("Failed close", ioe);
       }
@@ -317,9 +296,9 @@ public class TestThriftServerCmdLine {
         thriftServerRunner.close();
         thriftServerRunner.join();
         if (thriftServerRunner.getRunException() != null) {
-           LOG.error("Command-line invocation of HBase Thrift server threw exception",
-             thriftServerRunner.getRunException());
-           throw thriftServerRunner.getRunException();
+          LOG.error("Command-line invocation of HBase Thrift server threw exception",
+            thriftServerRunner.getRunException());
+          throw thriftServerRunner.getRunException();
         }
       }
     }
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 6f89c3c..8da3c68 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
@@ -20,17 +20,15 @@ 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.assertTrue;
 import java.io.File;
 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;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -39,6 +37,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.SimpleKdcServerUtil;
 import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.http.HttpHeaders;
 import org.apache.http.auth.AuthSchemeProvider;
@@ -93,26 +92,6 @@ public class TestThriftSpnegoHttpFallbackServer extends TestThriftHttpServer {
   private static String serverPrincipal;
   private static String spnegoServerPrincipal;
 
-  private static SimpleKdcServer buildMiniKdc() throws Exception {
-    SimpleKdcServer kdc = new SimpleKdcServer();
-
-    File kdcDir = Paths.get(TEST_UTIL.getRandomDir().toString()).toAbsolutePath().toFile();
-    kdcDir.mkdirs();
-    kdc.setWorkDir(kdcDir);
-
-    kdc.setKdcHost(HConstants.LOCALHOST);
-    int kdcPort = HBaseTestingUtility.randomFreePort();
-    kdc.setAllowTcp(true);
-    kdc.setAllowUdp(false);
-    kdc.setKdcTcpPort(kdcPort);
-
-    LOG.info("Starting KDC server at " + HConstants.LOCALHOST + ":" + kdcPort);
-
-    kdc.init();
-
-    return kdc;
-  }
-
   private static void addSecurityConfigurations(Configuration conf) {
     KerberosName.setRules("DEFAULT");
 
@@ -133,11 +112,12 @@ public class TestThriftSpnegoHttpFallbackServer extends TestThriftHttpServer {
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    kdc = buildMiniKdc();
-    kdc.start();
+    kdc = SimpleKdcServerUtil.
+      getRunningSimpleKdcServer(new File(TEST_UTIL.getDataTestDir().toString()),
+        HBaseTestingUtility::randomFreePort);
 
     File keytabDir = Paths.get(TEST_UTIL.getRandomDir().toString()).toAbsolutePath().toFile();
-    keytabDir.mkdirs();
+    assertTrue(keytabDir.mkdirs());
 
     clientPrincipal = "client@" + kdc.getKdcConfig().getKdcRealm();
     clientKeytab = new File(keytabDir, clientPrincipal + ".keytab");
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 1eeaa10..9b54244 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,9 +18,10 @@
 package org.apache.hadoop.hbase.thrift;
 
 import static org.apache.hadoop.hbase.thrift.Constants.THRIFT_SUPPORT_PROXYUSER_KEY;
-import static org.junit.Assert.*;
-
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import java.io.File;
+import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.nio.file.Paths;
 import java.security.Principal;
@@ -29,19 +30,17 @@ 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;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
 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.hbase.util.SimpleKdcServerUtil;
 import org.apache.hadoop.security.authentication.util.KerberosName;
 import org.apache.http.HttpHeaders;
 import org.apache.http.auth.AuthSchemeProvider;
@@ -64,6 +63,7 @@ import org.ietf.jgss.GSSManager;
 import org.ietf.jgss.GSSName;
 import org.ietf.jgss.Oid;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Test;
@@ -93,26 +93,6 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
   private static String serverPrincipal;
   private static String spnegoServerPrincipal;
 
-  private static SimpleKdcServer buildMiniKdc() throws Exception {
-    SimpleKdcServer kdc = new SimpleKdcServer();
-
-    File kdcDir = Paths.get(TEST_UTIL.getRandomDir().toString()).toAbsolutePath().toFile();
-    kdcDir.mkdirs();
-    kdc.setWorkDir(kdcDir);
-
-    kdc.setKdcHost(HConstants.LOCALHOST);
-    int kdcPort = HBaseTestingUtility.randomFreePort();
-    kdc.setAllowTcp(true);
-    kdc.setAllowUdp(false);
-    kdc.setKdcTcpPort(kdcPort);
-
-    LOG.info("Starting KDC server at " + HConstants.LOCALHOST + ":" + kdcPort);
-
-    kdc.init();
-
-    return kdc;
-  }
-
   private static void addSecurityConfigurations(Configuration conf) {
     KerberosName.setRules("DEFAULT");
 
@@ -133,21 +113,22 @@ public class TestThriftSpnegoHttpServer extends TestThriftHttpServer {
 
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
-    kdc = buildMiniKdc();
-    kdc.start();
-
+    kdc = SimpleKdcServerUtil.
+      getRunningSimpleKdcServer(new File(TEST_UTIL.getDataTestDir().toString()),
+        HBaseTestingUtility::randomFreePort);
     File keytabDir = Paths.get(TEST_UTIL.getRandomDir().toString()).toAbsolutePath().toFile();
-    keytabDir.mkdirs();
+    Assert.assertTrue(keytabDir.mkdirs());
 
     clientPrincipal = "client@" + kdc.getKdcConfig().getKdcRealm();
     clientKeytab = new File(keytabDir, clientPrincipal + ".keytab");
     kdc.createAndExportPrincipals(clientKeytab, clientPrincipal);
 
-    serverPrincipal = "hbase/" + HConstants.LOCALHOST + "@" + kdc.getKdcConfig().getKdcRealm();
+    String hostname = InetAddress.getLoopbackAddress().getHostName();
+    serverPrincipal = "hbase/" + hostname + "@" + kdc.getKdcConfig().getKdcRealm();
     serverKeytab = new File(keytabDir, serverPrincipal.replace('/', '_') + ".keytab");
 
     // Setup separate SPNEGO keytab
-    spnegoServerPrincipal = "HTTP/" + HConstants.LOCALHOST + "@" + kdc.getKdcConfig().getKdcRealm();
+    spnegoServerPrincipal = "HTTP/" + hostname + "@" + kdc.getKdcConfig().getKdcRealm();
     spnegoServerKeytab = new File(keytabDir, spnegoServerPrincipal.replace('/', '_') + ".keytab");
     kdc.createAndExportPrincipals(spnegoServerKeytab, spnegoServerPrincipal);
     kdc.createAndExportPrincipals(serverKeytab, serverPrincipal);