You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2014/10/03 23:29:16 UTC
git commit: HBASE-12136 Race condition between client adding tableCF
replication znode and server triggering TableCFsTracker (Virag Kothari)
Repository: hbase
Updated Branches:
refs/heads/0.98 579ce7a0d -> a9138d7f9
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/a9138d7f
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a9138d7f
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a9138d7f
Branch: refs/heads/0.98
Commit: a9138d7f96910f09e52b226248ccb169c98d6bd4
Parents: 579ce7a
Author: Ted Yu <te...@apache.org>
Authored: Fri Oct 3 21:29:02 2014 +0000
Committer: Ted Yu <te...@apache.org>
Committed: Fri Oct 3 21:29:02 2014 +0000
----------------------------------------------------------------------
.../replication/ReplicationPeersZKImpl.java | 28 +++++++++++---------
1 file changed, 16 insertions(+), 12 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/a9138d7f/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 022aa93..21d8c1c 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
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
+import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
@@ -116,21 +117,24 @@ 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),
- 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.
+ List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
+ ZKUtilOp op1 =
+ ZKUtilOp.createAndFailSilent(ZKUtil.joinZNode(this.peersZNode, id),
+ toByteArray(clusterKey));
+ // 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);
} catch (KeeperException e) {
- throw new ReplicationException("Could not add peer with id=" + id
- + ", clusterKey=" + clusterKey, e);
+ throw new ReplicationException("Could not add peer with id=" + id + ", clusterKey="
+ + clusterKey, e);
}
}