diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java index ab387a6..85405bc 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java @@ -263,6 +263,13 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { } /** + * Get a copy of current registered listeners + */ + public List getListeners() { + return new ArrayList(listeners); + } + + /** * @return The number of currently registered listeners */ public int getNumberOfListeners() { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 6d845f7..b56b6ed 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -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 registeredZKListenersBeforeRecovery; + /** * Initializes the HMaster. The steps are as follows: *

@@ -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 callable = new Callable () { diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java index ba4fdd1..4095ef7 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java @@ -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(); }