You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2013/05/15 06:12:50 UTC
svn commit: r1482671 - in /hbase/branches/0.94/src:
main/java/org/apache/hadoop/hbase/master/HMaster.java
main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
test/java/org/apache/hadoop/hbase/master/TestMasterZKSessionRecovery.java
Author: stack
Date: Wed May 15 04:12:50 2013
New Revision: 1482671
URL: http://svn.apache.org/r1482671
Log:
HBASE-8539 Double(or tripple ...) ZooKeeper listeners of the same type when Master recovers from ZK SessionExpiredException
Modified:
hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestMasterZKSessionRecovery.java
Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1482671&r1=1482670&r2=1482671&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Wed May 15 04:12:50 2013
@@ -1610,6 +1610,7 @@ Server {
private boolean tryRecoveringExpiredZKSession() throws InterruptedException,
IOException, KeeperException, ExecutionException {
+ this.zooKeeper.unregisterAllListeners();
this.zooKeeper.reconnectAfterExpiration();
Callable<Boolean> callable = new Callable<Boolean> () {
Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java?rev=1482671&r1=1482670&r2=1482671&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java Wed May 15 04:12:50 2013
@@ -243,6 +243,20 @@ public class ZooKeeperWatcher implements
}
/**
+ * 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
*/
Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestMasterZKSessionRecovery.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestMasterZKSessionRecovery.java?rev=1482671&r1=1482670&r2=1482671&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestMasterZKSessionRecovery.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/master/TestMasterZKSessionRecovery.java Wed May 15 04:12:50 2013
@@ -89,6 +89,8 @@ public class TestMasterZKSessionRecovery
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"),
@@ -106,6 +108,8 @@ public class TestMasterZKSessionRecovery
// 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());
}
static class MockLoadBalancer extends DefaultLoadBalancer {