Repository: hbase
Updated Branches:
  refs/heads/master 5375ff07b -> efe0787c8


HBASE-12136 Race condition between client adding tableCF replication znode and  
server triggering TableCFsTracker (Virag Kothari)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/efe0787c
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/efe0787c
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/efe0787c

Branch: refs/heads/master
Commit: efe0787c87ca03e548bec13d8ae24200f582b438
Parents: 5375ff0
Author: Ted Yu <te...@apache.org>
Authored: Fri Oct 3 21:34:45 2014 +0000
Committer: Ted Yu <te...@apache.org>
Committed: Fri Oct 3 21:34:45 2014 +0000

----------------------------------------------------------------------
 .../replication/ReplicationPeerZKImpl.java      |  9 +++++++-
 .../replication/ReplicationPeersZKImpl.java     | 23 ++++++++++++--------
 2 files changed, 22 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/efe0787c/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
index 13d9df3..848303c 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeerZKImpl.java
@@ -261,12 +261,19 @@ public class ReplicationPeerZKImpl implements 
ReplicationPeer, Abortable, Closea
         Abortable abortable) {
       super(watcher, tableCFsZNode, abortable);
     }
+    
+    @Override
+    public synchronized void nodeCreated(String path) {
+      if (path.equals(node)) {
+        super.nodeCreated(path);
+        readTableCFsZnode();
+      }
+    }
 
     @Override
     public synchronized void nodeDataChanged(String path) {
       if (path.equals(node)) {
         super.nodeDataChanged(path);
-        readTableCFsZnode();
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/efe0787c/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
index dce0903..abb0984 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/replication/ReplicationPeersZKImpl.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.replication;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -42,6 +43,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
 import org.apache.zookeeper.KeeperException;
 
 import com.google.protobuf.ByteString;
@@ -109,18 +111,21 @@ public class ReplicationPeersZKImpl extends 
ReplicationStateZKBase implements Re
             + " because that id already exists.");
       }
       ZKUtil.createWithParents(this.zookeeper, this.peersZNode);
-      ZKUtil.createAndWatch(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, 
id),
+      List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
+      ZKUtilOp op1 = 
ZKUtilOp.createAndFailSilent(ZKUtil.joinZNode(this.peersZNode, id),
         toByteArray(peerConfig));
-      // There is a race b/w PeerWatcher and ReplicationZookeeper#add method 
to create the
-      // peer-state znode. This happens while adding a peer.
+      // There is a race (if hbase.zookeeper.useMulti is false) 
+      // 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
-
+      ZKUtilOp op2 = ZKUtilOp.createAndFailSilent(getPeerStateNode(id), 
ENABLED_ZNODE_BYTES);
       String tableCFsStr = (tableCFs == null) ? "" : tableCFs;
-      ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, getTableCFsNode(id),
-                    Bytes.toBytes(tableCFsStr));
+      ZKUtilOp op3 = ZKUtilOp.createAndFailSilent(getTableCFsNode(id), 
Bytes.toBytes(tableCFsStr));
+      listOfOps.add(op1);
+      listOfOps.add(op2);
+      listOfOps.add(op3);
+      ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
+      // A peer is enabled by default
     } catch (KeeperException e) {
       throw new ReplicationException("Could not add peer with id=" + id
           + ", peerConfif=>" + peerConfig, e);

Reply via email to