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

This commit is contained in:
Ted Yu 2014-10-03 21:31:50 +00:00
parent c452942f57
commit 6b95b4a8a4
2 changed files with 22 additions and 10 deletions

View File

@ -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();
}
}
}

View File

@ -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);