Index: src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java =================================================================== --- src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java (revision 24344) +++ src/test/java/org/apache/hadoop/hbase/master/TestActiveMasterManager.java (revision 24346) @@ -84,7 +84,7 @@ MonitoredTask status = Mockito.mock(MonitoredTask.class); clusterStatusTracker.setClusterUp(); - activeMasterManager.blockUntilBecomingActiveMaster(status,clusterStatusTracker); + activeMasterManager.blockUntilBecomingActiveMaster(status); assertTrue(activeMasterManager.clusterHasActiveMaster.get()); assertMaster(zk, master); @@ -93,7 +93,7 @@ ActiveMasterManager secondActiveMasterManager = secondDummyMaster.getActiveMasterManager(); assertFalse(secondActiveMasterManager.clusterHasActiveMaster.get()); - activeMasterManager.blockUntilBecomingActiveMaster(status,clusterStatusTracker); + activeMasterManager.blockUntilBecomingActiveMaster(status); assertTrue(activeMasterManager.clusterHasActiveMaster.get()); assertMaster(zk, master); } @@ -129,7 +129,7 @@ ms1.getClusterStatusTracker(); clusterStatusTracker.setClusterUp(); activeMasterManager.blockUntilBecomingActiveMaster( - Mockito.mock(MonitoredTask.class),clusterStatusTracker); + Mockito.mock(MonitoredTask.class)); assertTrue(activeMasterManager.clusterHasActiveMaster.get()); assertMaster(zk, firstMasterAddress); @@ -211,8 +211,7 @@ @Override public void run() { manager.blockUntilBecomingActiveMaster( - Mockito.mock(MonitoredTask.class), - this.dummyMaster.getClusterStatusTracker()); + Mockito.mock(MonitoredTask.class)); LOG.info("Second master has become the active master!"); isActiveMaster = true; } Index: src/main/java/org/apache/hadoop/hbase/master/HMaster.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/HMaster.java (revision 24344) +++ src/main/java/org/apache/hadoop/hbase/master/HMaster.java (revision 24346) @@ -488,8 +488,7 @@ // to check if the cluster should be shutdown. this.clusterStatusTracker = new ClusterStatusTracker(getZooKeeper(), this); this.clusterStatusTracker.start(); - return this.activeMasterManager.blockUntilBecomingActiveMaster(startupStatus, - this.clusterStatusTracker); + return this.activeMasterManager.blockUntilBecomingActiveMaster(startupStatus); } /** Index: src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java =================================================================== --- src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java (revision 24344) +++ src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java (revision 24346) @@ -49,6 +49,7 @@ private static final Log LOG = LogFactory.getLog(ActiveMasterManager.class); final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false); + final AtomicBoolean clusterShutDown = new AtomicBoolean(false); private final ServerName sn; private final Server master; @@ -73,6 +74,16 @@ @Override public void nodeDeleted(String path) { + // We need to keep track of the cluster's shutdown status while + // we wait on the current master. We consider that, if the cluster + // was already in a "shutdown" state when we started, that this master + // is part of a new cluster that was started shortly after the old cluster + // shut down, so that state is now irrelevant. This means that the shutdown + // state must be set while we wait on the active master in order + // to shutdown this master. See HBASE-8519. + if (path.equals(watcher.clusterStateZNode) && !master.isStopped()) { + clusterShutDown.set(true); + } if(path.equals(watcher.masterAddressZNode) && !master.isStopped()) { handleMasterNodeChange(); } @@ -125,8 +136,7 @@ * master was running or if some other problem (zookeeper, stop flag has been * set on this Master) */ - boolean blockUntilBecomingActiveMaster(MonitoredTask startupStatus, - ClusterStatusTracker clusterStatusTracker) { + boolean blockUntilBecomingActiveMaster(MonitoredTask startupStatus) { while (true) { startupStatus.setStatus("Trying to register in ZK as active master"); // Try to become the active master, watch if there is another master. @@ -199,7 +209,7 @@ LOG.debug("Interrupted waiting for master to die", e); } } - if (!clusterStatusTracker.isClusterUp()) { + if (clusterShutDown.get()) { this.master.stop("Cluster went down before this master became active"); } if (this.master.isStopped()) {