From 71d6b124dda724f009fd6aa3745ce200e52f2b69 Mon Sep 17 00:00:00 2001 From: Michael Stack Date: Sat, 9 Oct 2010 17:55:33 +0000 Subject: [PATCH] HBASE-3062 ZooKeeper KeeperException is a recoverable exception; we should retry a while on server startup at least. git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1006202 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 3 ++ .../hbase/regionserver/HRegionServer.java | 18 ++++++++---- .../hbase/zookeeper/ZooKeeperWatcher.java | 29 ++++++++++++++++++- 3 files changed, 43 insertions(+), 7 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index 1bb2af1bb60..2b38c73c54c 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -571,6 +571,9 @@ Release 0.21.0 - Unreleased HBASE-3008 Memstore.updateColumnValue passes wrong flag to heapSizeChange (Causes memstore size to go negative) HBASE-3089 REST tests are broken locally and up in hudson + HBASE-3062 ZooKeeper KeeperException$ConnectionLossException is a + "recoverable" exception; we should retry a while on server + startup at least. diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 36c404db48e..38a680fec46 100644 --- a/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -402,11 +402,17 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler, * @throws InterruptedException */ private void initialize() throws IOException, InterruptedException { - initializeZooKeeper(); - initializeThreads(); - int nbBlocks = conf.getInt("hbase.regionserver.nbreservationblocks", 4); - for (int i = 0; i < nbBlocks; i++) { - reservedSpace.add(new byte[HConstants.DEFAULT_SIZE_RESERVATION_BLOCK]); + try { + initializeZooKeeper(); + initializeThreads(); + int nbBlocks = conf.getInt("hbase.regionserver.nbreservationblocks", 4); + for (int i = 0; i < nbBlocks; i++) { + reservedSpace.add(new byte[HConstants.DEFAULT_SIZE_RESERVATION_BLOCK]); + } + } catch (Throwable t) { + // Call stop if error or process will stick around for ever since server + // puts up non-daemon threads. + this.server.stop(); } } @@ -2465,4 +2471,4 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler, new HRegionServerCommandLine(regionServerClass).doMain(args); } -} \ No newline at end of file +} diff --git a/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java b/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java index 26c2207ac4a..b2a2e9aa59d 100644 --- a/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java +++ b/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java @@ -104,7 +104,30 @@ public class ZooKeeperWatcher implements Watcher { try { // Create all the necessary "directories" of znodes // TODO: Move this to an init method somewhere so not everyone calls it? - ZKUtil.createAndFailSilent(this, baseZNode); + + // The first call against zk can fail with connection loss. Seems common. + // Apparently this is recoverable. Retry a while. + // See http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling + // TODO: Generalize out in ZKUtil. + long wait = conf.getLong("hbase.zookeeper.recoverable.waittime", 10000); + long finished = System.currentTimeMillis() + wait; + KeeperException ke = null; + do { + try { + ZKUtil.createAndFailSilent(this, baseZNode); + ke = null; + break; + } catch (KeeperException.ConnectionLossException e) { + if (LOG.isDebugEnabled() && (isFinishedRetryingRecoverable(finished))) { + LOG.debug("Retrying zk create for another " + + (finished - System.currentTimeMillis()) + + "ms; set 'hbase.zookeeper.recoverable.waittime' to change " + + "wait time); " + e.getMessage()); + } + ke = e; + } + } while (isFinishedRetryingRecoverable(finished)); + if (ke != null) throw ke; ZKUtil.createAndFailSilent(this, assignmentZNode); ZKUtil.createAndFailSilent(this, rsZNode); ZKUtil.createAndFailSilent(this, tableZNode); @@ -114,6 +137,10 @@ public class ZooKeeperWatcher implements Watcher { } } + private boolean isFinishedRetryingRecoverable(final long finished) { + return System.currentTimeMillis() < finished; + } + @Override public String toString() { return this.identifier;