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
This commit is contained in:
Michael Stack 2010-10-09 17:55:33 +00:00
parent b7f3cfbf05
commit 71d6b124dd
3 changed files with 43 additions and 7 deletions

View File

@ -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.

View File

@ -402,12 +402,18 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
* @throws InterruptedException
*/
private void initialize() throws IOException, InterruptedException {
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();
}
}
/**

View File

@ -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?
// 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;