You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by je...@apache.org on 2013/11/12 04:14:31 UTC

svn commit: r1540933 - in /hbase/trunk/hbase-server/src: main/java/org/apache/hadoop/hbase/master/HMaster.java test/java/org/apache/hadoop/hbase/TestZooKeeper.java

Author: jeffreyz
Date: Tue Nov 12 03:14:31 2013
New Revision: 1540933

URL: http://svn.apache.org/r1540933
Log:
HBASE-9918: MasterAddressTracker & ZKNamespaceManager ZK listeners are missed after master recovery

Modified:
    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-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=1540933&r1=1540932&r2=1540933&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 Tue Nov 12 03:14:31 2013
@@ -202,7 +202,6 @@ import org.apache.hadoop.hbase.protobuf.
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorResponse;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
-import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
@@ -573,7 +572,6 @@ MasterServices, Server {
     startupStatus.setDescription("Master startup");
     masterStartTime = System.currentTimeMillis();
     try {
-      this.registeredZKListenersBeforeRecovery = this.zooKeeper.getListeners();
       this.masterAddressManager = new MasterAddressTracker(getZooKeeperWatcher(), this);
       this.masterAddressManager.start();
 
@@ -588,6 +586,7 @@ MasterServices, Server {
         this.infoServer.start();
       }
 
+      this.registeredZKListenersBeforeRecovery = this.zooKeeper.getListeners();
       /*
        * Block on becoming the active master.
        *
@@ -913,11 +912,11 @@ MasterServices, Server {
       this.balancerChore = getAndStartBalancerChore(this);
       this.catalogJanitorChore = new CatalogJanitor(this, this);
       startCatalogJanitorChore();
-
-      status.setStatus("Starting namespace manager");
-      initNamespace();
     }
 
+    status.setStatus("Starting namespace manager");
+    initNamespace();
+
     if (this.cpHost != null) {
       try {
         this.cpHost.preMasterInitialization();

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=1540933&r1=1540932&r2=1540933&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 Tue Nov 12 03:14:31 2013
@@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.master.HM
 import org.apache.hadoop.hbase.master.LoadBalancer;
 import org.apache.hadoop.hbase.master.balancer.SimpleLoadBalancer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.zookeeper.EmptyWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
@@ -60,6 +61,7 @@ import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper.States;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
+import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
@@ -83,12 +85,12 @@ public class TestZooKeeper {
   public static void setUpBeforeClass() throws Exception {
     // Test we can first start the ZK cluster by itself
     Configuration conf = TEST_UTIL.getConfiguration();
+    TEST_UTIL.startMiniDFSCluster(2);
     TEST_UTIL.startMiniZKCluster();
     conf.setBoolean("dfs.support.append", true);
     conf.setInt(HConstants.ZK_SESSION_TIMEOUT, 1000);
     conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, MockLoadBalancer.class,
         LoadBalancer.class);
-    TEST_UTIL.startMiniCluster(2);
   }
 
   /**
@@ -104,9 +106,18 @@ public class TestZooKeeper {
    */
   @Before
   public void setUp() throws Exception {
-    TEST_UTIL.ensureSomeRegionServersAvailable(2);
+    TEST_UTIL.startMiniHBaseCluster(1, 2);
   }
 
+  @After
+  public void after() throws Exception {
+    try {
+      TEST_UTIL.shutdownMiniHBaseCluster();
+    } finally {
+      TEST_UTIL.getTestFileSystem().delete(FSUtils.getRootDir(TEST_UTIL.getConfiguration()), true);
+      ZKUtil.deleteNodeRecursively(TEST_UTIL.getZooKeeperWatcher(), "/hbase");
+    }
+  }
 
   private ZooKeeperWatcher getZooKeeperWatcher(HConnection c)
   throws NoSuchMethodException, InvocationTargetException, IllegalAccessException {
@@ -473,6 +484,7 @@ public class TestZooKeeper {
   public void testRegionAssignmentAfterMasterRecoveryDueToZKExpiry() throws Exception {
     MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
     cluster.startRegionServer();
+    cluster.waitForActiveAndReadyMaster(10000);
     HMaster m = cluster.getMaster();
     ZooKeeperWatcher zkw = m.getZooKeeperWatcher();
     int expectedNumOfListeners = zkw.getNumberOfListeners();
@@ -495,6 +507,8 @@ public class TestZooKeeper {
       // clean startup.
       assertFalse("Retain assignment should not be called", MockLoadBalancer.retainAssignCalled);
       // number of listeners should be same as the value before master aborted
+      // wait for new master is initialized
+      cluster.waitForActiveAndReadyMaster(10000);
       assertEquals(expectedNumOfListeners, zkw.getNumberOfListeners());
     } finally {
       admin.close();