HBASE-7440 ReplicationZookeeper#addPeer is racy (Himanshu)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1426702 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0c56fe3198
commit
3704dc67f6
|
@ -411,8 +411,12 @@ public class ReplicationZookeeper implements Closeable{
|
|||
ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
|
||||
ZKUtil.createAndWatch(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id),
|
||||
toByteArray(clusterKey));
|
||||
// There is a race b/w PeerWatcher and ReplicationZookeeper#add method to create the
|
||||
// peer-state znode. This happens while adding a peer.
|
||||
// The peer state data is set as "ENABLED" by default.
|
||||
ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, getPeerStateNode(id),
|
||||
ENABLED_ZNODE_BYTES);
|
||||
// A peer is enabled by default
|
||||
ZKUtil.createAndWatch(this.zookeeper, getPeerStateNode(id), ENABLED_ZNODE_BYTES);
|
||||
} catch (KeeperException e) {
|
||||
throw new IOException("Unable to add peer", e);
|
||||
}
|
||||
|
@ -1067,7 +1071,10 @@ public class ReplicationZookeeper implements Closeable{
|
|||
static boolean ensurePeerEnabled(final ZooKeeperWatcher zookeeper, final String path)
|
||||
throws NodeExistsException, KeeperException {
|
||||
if (ZKUtil.checkExists(zookeeper, path) == -1) {
|
||||
ZKUtil.createAndWatch(zookeeper, path, ENABLED_ZNODE_BYTES);
|
||||
// There is a race b/w PeerWatcher and ReplicationZookeeper#add method to create the
|
||||
// peer-state znode. This happens while adding a peer.
|
||||
// The peer state data is set as "ENABLED" by default.
|
||||
ZKUtil.createNodeIfNotExistsAndWatch(zookeeper, path, ENABLED_ZNODE_BYTES);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
|
|
Loading…
Reference in New Issue