You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by se...@apache.org on 2013/05/18 03:40:48 UTC

svn commit: r1484039 - in /hbase/trunk: hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ hbase-server/src/main/java/org/apache/hadoop/hbase/master/ hbase-server/src/test/java/org/apache/hadoop/hbase/

Author: sershe
Date: Sat May 18 01:40:48 2013
New Revision: 1484039

URL: http://svn.apache.org/r1484039
Log:
HBASE-8539 ADDENDUM Double(or tripple ...) ZooKeeper listeners of the same type when Master recovers from ZK SessionExpiredException (Jeffrey Zhong)

Modified:
    hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java

Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java?rev=1484039&r1=1484038&r2=1484039&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java Sat May 18 01:40:48 2013
@@ -263,6 +263,13 @@ public class ZooKeeperWatcher implements
   }
 
   /**
+   * Get a copy of current registered listeners
+   */
+  public List<ZooKeeperListener> getListeners() {
+    return new ArrayList<ZooKeeperListener>(listeners);
+  }
+
+  /**
    * @return The number of currently registered listeners
    */
   public int getNumberOfListeners() {

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1484039&r1=1484038&r2=1484039&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Sat May 18 01:40:48 2013
@@ -201,6 +201,7 @@ import org.apache.hadoop.hbase.zookeeper
 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> () {

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java?rev=1484039&r1=1484038&r2=1484039&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/TestZooKeeper.java Sat May 18 01:40:48 2013
@@ -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();
     }