From d0a9bb0ba8ea6029efb0c12dcf1561c87fbd13b6 Mon Sep 17 00:00:00 2001 From: Josh Elser Date: Tue, 14 May 2019 11:54:49 -0400 Subject: [PATCH] HBASE-22416 HBOSS: Wait for curator to connect to ZK before returning. We'll wait up to 30s. If we're still not connected, throw an exception. --- .../hbase/oss/sync/ZKTreeLockManager.java | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/hbase-oss/src/main/java/org/apache/hadoop/hbase/oss/sync/ZKTreeLockManager.java b/hbase-oss/src/main/java/org/apache/hadoop/hbase/oss/sync/ZKTreeLockManager.java index 3214e2e..b5009c9 100644 --- a/hbase-oss/src/main/java/org/apache/hadoop/hbase/oss/sync/ZKTreeLockManager.java +++ b/hbase-oss/src/main/java/org/apache/hadoop/hbase/oss/sync/ZKTreeLockManager.java @@ -43,10 +43,11 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static java.util.Objects.requireNonNull; + /** * Implementation based on Apache Curator and Apache ZooKeeper. This allows * HBOSS to re-use an Apache HBase cluster's ZooKeeper ensemble for file @@ -87,6 +88,7 @@ public class ZKTreeLockManager extends TreeLockManager { String zookeeperConnectionString = conf.get(Constants.ZK_CONN_STRING); curator = CuratorFrameworkFactory.newClient(zookeeperConnectionString, retryPolicy); curator.start(); + waitForCuratorToConnect(); setRoot(); try { @@ -99,6 +101,19 @@ public class ZKTreeLockManager extends TreeLockManager { zookeeperConnectionString += root; curator = CuratorFrameworkFactory.newClient(zookeeperConnectionString, retryPolicy); curator.start(); + waitForCuratorToConnect(); + } + + private void waitForCuratorToConnect() { + try { + if (!requireNonNull(curator).blockUntilConnected(30, TimeUnit.SECONDS)) { + throw new RuntimeException("Failed to connect to ZooKeeper"); + } + } catch (InterruptedException e) { + LOG.warn("Interrupted waiting to connect to ZooKeeper", e); + Thread.currentThread().interrupt(); + return; + } } @Override -- 2.18.0