HBASE-8539 ADDENDUM 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@1484039 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
sershe 2013-05-18 01:40:48 +00:00
parent 2a5c92c084
commit 6b46196c31
3 changed files with 22 additions and 0 deletions

View File

@ -262,6 +262,13 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
listeners.clear();
}
/**
* Get a copy of current registered listeners
*/
public List<ZooKeeperListener> getListeners() {
return new ArrayList<ZooKeeperListener>(listeners);
}
/**
* @return The number of currently registered listeners
*/

View File

@ -201,6 +201,7 @@ import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.metrics.util.MBeanUtil;
import org.apache.hadoop.net.DNS;
@ -358,6 +359,9 @@ MasterServices, Server {
/** flag used in test cases in order to simulate RS failures during master initialization */
private volatile boolean initializationBeforeMetaAssignment = false;
/** The following is used in master recovery scenario to re-register listeners */
private List<ZooKeeperListener> registeredZKListenersBeforeRecovery;
/**
* Initializes the HMaster. The steps are as follows:
* <p>
@ -530,6 +534,7 @@ MasterServices, Server {
startupStatus.setDescription("Master startup");
masterStartTime = System.currentTimeMillis();
try {
this.registeredZKListenersBeforeRecovery = this.zooKeeper.getListeners();
/*
* Block on becoming the active master.
*
@ -2049,6 +2054,14 @@ MasterServices, Server {
IOException, KeeperException, ExecutionException {
this.zooKeeper.unregisterAllListeners();
// add back listeners which were registered before master initialization
// because they won't be added back in below Master re-initialization code
if (this.registeredZKListenersBeforeRecovery != null) {
for (ZooKeeperListener curListener : this.registeredZKListenersBeforeRecovery) {
this.zooKeeper.registerListener(curListener);
}
}
this.zooKeeper.reconnectAfterExpiration();
Callable<Boolean> callable = new Callable<Boolean> () {

View File

@ -494,6 +494,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());
} finally {
admin.close();
}