You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2012/12/29 05:16:12 UTC
svn commit: r1426702 -
/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java
Author: larsh
Date: Sat Dec 29 04:16:12 2012
New Revision: 1426702
URL: http://svn.apache.org/viewvc?rev=1426702&view=rev
Log:
HBASE-7440 ReplicationZookeeper#addPeer is racy (Himanshu)
Modified:
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java?rev=1426702&r1=1426701&r2=1426702&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java Sat Dec 29 04:16:12 2012
@@ -411,8 +411,12 @@ public class ReplicationZookeeper implem
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 implem
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;