HBASE-3648 [replication] failover is sloppy with znodes

HBASE-3596  [replication] Wait a few seconds before transferring queues


git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1081957 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jean-Daniel Cryans 2011-03-15 21:47:48 +00:00
parent b408cc1217
commit 6392c15b73
3 changed files with 47 additions and 3 deletions

View File

@ -162,6 +162,7 @@ Release 0.90.2 - Unreleased
HBASE-3636 a bug about deciding whether this key is a new key for the ROWCOL
bloomfilter (Liyin Tang via Stack)
HBASE-3639 FSUtils.getRootDir should qualify path
HBASE-3648 [replication] failover is sloppy with znodes
IMPROVEMENTS
HBASE-3542 MultiGet methods in Thrift
@ -171,6 +172,7 @@ Release 0.90.2 - Unreleased
(Harsh J Chouraria via Stack)
HBASE-3625 improve/fix support excluding Tests via Maven -D property
(Alejandro Abdelnur via todd)
HBASE-3596 [replication] Wait a few seconds before transferring queues
Release 0.90.1 - Unreleased

View File

@ -516,7 +516,18 @@ public class ReplicationZookeeper {
String p = ZKUtil.joinZNode(parent, RS_LOCK_ZNODE);
ZKUtil.createAndWatch(this.zookeeper, p, Bytes.toBytes(rsServerNameZnode));
} catch (KeeperException e) {
// This exception will pop up if the znode under which we're trying to
// create the lock is already deleted by another region server, meaning
// that the transfer already occurred.
// NoNode => transfer is done and znodes are already deleted
// NodeExists => lock znode already created by another RS
if (e instanceof KeeperException.NoNodeException ||
e instanceof KeeperException.NodeExistsException) {
LOG.info("Won't transfer the queue," +
" another RS took care of it because of: " + e.getMessage());
} else {
LOG.info("Failed lock other rs", e);
}
return false;
}
return true;
@ -597,10 +608,30 @@ public class ReplicationZookeeper {
* @param znode
*/
public void deleteRsQueues(String znode) {
String fullpath = ZKUtil.joinZNode(rsZNode, znode);
try {
ZKUtil.deleteNodeRecursively(this.zookeeper,
ZKUtil.joinZNode(rsZNode, znode));
List<String> clusters =
ZKUtil.listChildrenNoWatch(this.zookeeper, fullpath);
for (String cluster : clusters) {
// We'll delete it later
if (cluster.equals(RS_LOCK_ZNODE)) {
continue;
}
String fullClusterPath = ZKUtil.joinZNode(fullpath, cluster);
ZKUtil.deleteNodeRecursively(this.zookeeper, fullClusterPath);
}
// Finish cleaning up
ZKUtil.deleteNodeRecursively(this.zookeeper, fullpath);
} catch (KeeperException e) {
if (e instanceof KeeperException.NoNodeException ||
e instanceof KeeperException.NotEmptyException) {
// Testing a special case where another region server was able to
// create a lock just after we deleted it, but then was also able to
// delete the RS znode before us or its lock znode is still there.
if (e.getPath().equals(fullpath)) {
return;
}
}
this.abortable.abort("Failed delete of " + znode, e);
}
}

View File

@ -76,6 +76,8 @@ public class ReplicationSourceManager {
private final Path logDir;
// Path to the hlog archive
private final Path oldLogDir;
// The number of ms that we wait before moving znodes, HBASE-3596
private final long sleepBeforeFailover;
/**
* Creates a replication manager and sets the watch on all the other
@ -105,6 +107,7 @@ public class ReplicationSourceManager {
this.fs = fs;
this.logDir = logDir;
this.oldLogDir = oldLogDir;
this.sleepBeforeFailover = conf.getLong("replication.sleep.before.failover", 2000);
this.zkHelper.registerRegionServerListener(
new OtherRegionServerWatcher(this.zkHelper.getZookeeperWatcher()));
List<String> otherRSs =
@ -291,6 +294,14 @@ public class ReplicationSourceManager {
* @param rsZnode
*/
public void transferQueues(String rsZnode) {
// Wait a bit before transferring the queues, we may be shutting down.
// This sleep may not be enough in some cases.
try {
Thread.sleep(this.sleepBeforeFailover);
} catch (InterruptedException e) {
LOG.warn("Interrupted while waiting before transferring a queue.");
Thread.currentThread().interrupt();
}
// We try to lock that rs' queue directory
if (this.stopper.isStopped()) {
LOG.info("Not transferring queue since we are shutting down");