HBASE-8519 Backup master will never come up if primary master dies during initialization

(Jerry He via JD)


git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1482981 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jean-Daniel Cryans 2013-05-15 17:54:30 +00:00
parent 2068d57f99
commit f729fcd4ec
3 changed files with 20 additions and 11 deletions

View File

@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
@ -54,6 +53,7 @@ class ActiveMasterManager extends ZooKeeperListener {
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;
@ -76,6 +76,18 @@ class ActiveMasterManager extends ZooKeeperListener {
@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);
}
handle(path);
}
@ -132,8 +144,7 @@ class ActiveMasterManager extends ZooKeeperListener {
* 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.
@ -220,7 +231,7 @@ class ActiveMasterManager extends ZooKeeperListener {
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");
}

View File

@ -602,8 +602,7 @@ MasterServices, Server {
// 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);
}
/**

View File

@ -86,7 +86,7 @@ public class TestActiveMasterManager {
MonitoredTask status = Mockito.mock(MonitoredTask.class);
clusterStatusTracker.setClusterUp();
activeMasterManager.blockUntilBecomingActiveMaster(status,clusterStatusTracker);
activeMasterManager.blockUntilBecomingActiveMaster(status);
assertTrue(activeMasterManager.clusterHasActiveMaster.get());
assertMaster(zk, master);
@ -95,7 +95,7 @@ public class TestActiveMasterManager {
ActiveMasterManager secondActiveMasterManager =
secondDummyMaster.getActiveMasterManager();
assertFalse(secondActiveMasterManager.clusterHasActiveMaster.get());
activeMasterManager.blockUntilBecomingActiveMaster(status,clusterStatusTracker);
activeMasterManager.blockUntilBecomingActiveMaster(status);
assertTrue(activeMasterManager.clusterHasActiveMaster.get());
assertMaster(zk, master);
}
@ -131,7 +131,7 @@ public class TestActiveMasterManager {
ms1.getClusterStatusTracker();
clusterStatusTracker.setClusterUp();
activeMasterManager.blockUntilBecomingActiveMaster(
Mockito.mock(MonitoredTask.class),clusterStatusTracker);
Mockito.mock(MonitoredTask.class));
assertTrue(activeMasterManager.clusterHasActiveMaster.get());
assertMaster(zk, firstMasterAddress);
@ -213,8 +213,7 @@ public class TestActiveMasterManager {
@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;
}