HBASE-8539 Double(or tripple ...) ZooKeeper listeners of the same type when Master recovers from ZK SessionExpiredException (Jeffrey Zhong)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1482574 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
d1c5b7cbd2
commit
267e55da40
|
@ -250,6 +250,20 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
|
|||
listeners.remove(listener);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clean all existing listeners
|
||||
*/
|
||||
public void unregisterAllListeners() {
|
||||
listeners.clear();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The number of currently registered listeners
|
||||
*/
|
||||
public int getNumberOfListeners() {
|
||||
return listeners.size();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the connection to ZooKeeper.
|
||||
* @return connection reference to zookeeper
|
||||
|
|
|
@ -2017,6 +2017,7 @@ MasterServices, Server {
|
|||
private boolean tryRecoveringExpiredZKSession() throws InterruptedException,
|
||||
IOException, KeeperException, ExecutionException {
|
||||
|
||||
this.zooKeeper.unregisterAllListeners();
|
||||
this.zooKeeper.reconnectAfterExpiration();
|
||||
|
||||
Callable<Boolean> callable = new Callable<Boolean> () {
|
||||
|
|
|
@ -473,6 +473,8 @@ public class TestZooKeeper {
|
|||
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
|
||||
cluster.startRegionServer();
|
||||
HMaster m = cluster.getMaster();
|
||||
ZooKeeperWatcher zkw = m.getZooKeeperWatcher();
|
||||
int expectedNumOfListeners = zkw.getNumberOfListeners();
|
||||
// now the cluster is up. So assign some regions.
|
||||
HBaseAdmin admin = new HBaseAdmin(TEST_UTIL.getConfiguration());
|
||||
byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("a"), Bytes.toBytes("b"),
|
||||
|
@ -490,6 +492,8 @@ public class TestZooKeeper {
|
|||
// The recovered master should not call retainAssignment, as it is not a
|
||||
// clean startup.
|
||||
assertFalse("Retain assignment should not be called", MockLoadBalancer.retainAssignCalled);
|
||||
// number of listeners should be same as the value before master aborted
|
||||
assertEquals(expectedNumOfListeners, zkw.getNumberOfListeners());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue