From 81adb704c8e578db82e15245ea48a3efe732965d Mon Sep 17 00:00:00 2001 From: Mike Drob Date: Wed, 10 Oct 2018 17:49:54 -0500 Subject: [PATCH] HBASE-21287 Allow configuring test master initialization wait time. --- .../hadoop/hbase/util/JVMClusterUtil.java | 69 +++++++++++-------- 1 file changed, 39 insertions(+), 30 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java index ee7ecf3fad6..8c92f6660c0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/JVMClusterUtil.java @@ -23,12 +23,13 @@ import java.io.IOException; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.CoordinatedStateManager; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -162,6 +163,8 @@ public class JVMClusterUtil { */ public static String startup(final List masters, final List regionservers) throws IOException { + // Implementation note: This method relies on timed sleeps in a loop. It's not great, and + // should probably be re-written to use actual synchronization objects, but it's ok for now Configuration configuration = null; @@ -177,21 +180,9 @@ public class JVMClusterUtil { // Wait for an active master // having an active master before starting the region threads allows // then to succeed on their connection to master - long startTime = System.currentTimeMillis(); - while (findActiveMaster(masters) == null) { - try { - Thread.sleep(100); - } catch (InterruptedException e) { - throw (InterruptedIOException)new InterruptedIOException().initCause(e); - } - int startTimeout = configuration != null ? Integer.parseInt( + final int startTimeout = configuration != null ? Integer.parseInt( configuration.get("hbase.master.start.timeout.localHBaseCluster", "30000")) : 30000; - if (System.currentTimeMillis() > startTime + startTimeout) { - String msg = "Master not active after " + startTimeout + "ms"; - Threads.printThreadInfo(System.out, "Thread dump because: " + msg); - throw new RuntimeException(msg); - } - } + waitForEvent(startTimeout, "active", () -> findActiveMaster(masters) != null); if (regionservers != null) { for (JVMClusterUtil.RegionServerThread t: regionservers) { @@ -201,32 +192,50 @@ public class JVMClusterUtil { // Wait for an active master to be initialized (implies being master) // with this, when we return the cluster is complete - startTime = System.currentTimeMillis(); - final int maxwait = 200000; + final int initTimeout = configuration != null ? Integer.parseInt( + configuration.get("hbase.master.init.timeout.localHBaseCluster", "200000")) : 200000; + waitForEvent(initTimeout, "initialized", () -> { + JVMClusterUtil.MasterThread t = findActiveMaster(masters); + // master thread should never be null at this point, but let's keep the check anyway + return t != null && t.master.isInitialized(); + } + ); + + return findActiveMaster(masters).master.getServerName().toString(); + } + + /** + * Utility method to wait some time for an event to occur, and then return control to the caller. + * @param millis How long to wait, in milliseconds. + * @param action The action that we are waiting for. Will be used in log message if the event + * does not occur. + * @param check A Supplier that will be checked periodically to produce an updated true/false + * result indicating if the expected event has happened or not. + * @throws InterruptedIOException If we are interrupted while waiting for the event. + * @throws RuntimeException If we reach the specified timeout while waiting for the event. + */ + private static void waitForEvent(long millis, String action, Supplier check) + throws InterruptedIOException { + long end = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(millis); + while (true) { - JVMClusterUtil.MasterThread t = findActiveMaster(masters); - if (t != null && t.master.isInitialized()) { - return t.master.getServerName().toString(); + if (check.get()) { + return; } - // REMOVE - if (System.currentTimeMillis() > startTime + 10000) { - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - throw (InterruptedIOException)new InterruptedIOException().initCause(e); - } - } - if (System.currentTimeMillis() > startTime + maxwait) { - String msg = "Master not initialized after " + maxwait + "ms seconds"; + + if (System.nanoTime() > end) { + String msg = "Master not " + action + " after " + millis + "ms"; Threads.printThreadInfo(System.out, "Thread dump because: " + msg); throw new RuntimeException(msg); } + try { Thread.sleep(100); } catch (InterruptedException e) { throw (InterruptedIOException)new InterruptedIOException().initCause(e); } } + } /**