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:18:50 UTC

svn commit: r1426704 - in /hbase/branches/0.94: CHANGES.txt src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java

Author: larsh
Date: Sat Dec 29 04:18:49 2012
New Revision: 1426704

URL: http://svn.apache.org/viewvc?rev=1426704&view=rev
Log:
HBASE-7440 ReplicationZookeeper#addPeer is racy (Himanshu)

Modified:
    hbase/branches/0.94/CHANGES.txt
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java

Modified: hbase/branches/0.94/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/CHANGES.txt?rev=1426704&r1=1426703&r2=1426704&view=diff
==============================================================================
--- hbase/branches/0.94/CHANGES.txt (original)
+++ hbase/branches/0.94/CHANGES.txt Sat Dec 29 04:18:49 2012
@@ -52,6 +52,7 @@ Bug
     [HBASE-7422] - MasterFS doesn't set configuration for internal FileSystem
     [HBASE-7432] - TestHBaseFsck prevents testsuite from finishing
     [HBASE-7435] - BuiltInGzipDecompressor is only released during full GC
+    [HBASE-7440] - ReplicationZookeeper#addPeer is racy
 
 Improvement
 

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java?rev=1426704&r1=1426703&r2=1426704&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeer.java Sat Dec 29 04:18:49 2012
@@ -79,8 +79,11 @@ public class ReplicationPeer implements 
   public void startStateTracker(ZooKeeperWatcher zookeeper, String peerStateNode)
       throws KeeperException {
     if (ZKUtil.checkExists(zookeeper, peerStateNode) == -1) {
-      ZKUtil.createAndWatch(zookeeper, peerStateNode,
-          Bytes.toBytes(PeerState.ENABLED.name())); // enabled by default
+      // 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, peerStateNode,
+        Bytes.toBytes(PeerState.ENABLED.name()));
     }
     this.peerStateTracker = new PeerStateTracker(peerStateNode, zookeeper,
         this);

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java?rev=1426704&r1=1426703&r2=1426704&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java Sat Dec 29 04:18:49 2012
@@ -389,10 +389,13 @@ public class ReplicationZookeeper {
         throw new IllegalArgumentException("Cannot add existing peer");
       }
       ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
-      ZKUtil.createAndWatch(this.zookeeper,
-          ZKUtil.joinZNode(this.peersZNode, id), Bytes.toBytes(clusterKey));
-      ZKUtil.createAndWatch(this.zookeeper, getPeerStateNode(id),
-          Bytes.toBytes(PeerState.ENABLED.name())); // enabled by default
+      ZKUtil.createAndWatch(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id),
+        Bytes.toBytes(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),
+        Bytes.toBytes(PeerState.ENABLED.name()));
     } catch (KeeperException e) {
       throw new IOException("Unable to add peer", e);
     }