Skip to content
Permalink
Browse files
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.

Signed-off-by: Sean Mackrory <mackrorysd@gmail.com>
  • Loading branch information
joshelser committed May 16, 2019
1 parent 074c852 commit 919403df325fe36f6a4eb2de69a97513a2b4273b
Showing 1 changed file with 16 additions and 1 deletion.
@@ -43,10 +43,11 @@
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 void initialize(FileSystem fs) throws IOException {
String zookeeperConnectionString = conf.get(Constants.ZK_CONN_STRING);
curator = CuratorFrameworkFactory.newClient(zookeeperConnectionString, retryPolicy);
curator.start();
waitForCuratorToConnect();

setRoot();
try {
@@ -99,6 +101,19 @@ public void initialize(FileSystem fs) throws IOException {
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

0 comments on commit 919403d

Please sign in to comment.