HBASE-5213 "hbase master stop" does not bring down backup masters (Gregory)
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1298859 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
5d9f98e09d
commit
35a36c2c66
|
@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
|||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
|
||||
|
||||
/**
|
||||
* Handles everything on master-side related to master election.
|
||||
|
@ -128,7 +129,8 @@ 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) {
|
||||
boolean blockUntilBecomingActiveMaster(MonitoredTask startupStatus,
|
||||
ClusterStatusTracker clusterStatusTracker) {
|
||||
startupStatus.setStatus("Trying to register in ZK as active master");
|
||||
boolean cleanSetOfActiveMaster = true;
|
||||
// Try to become the active master, watch if there is another master.
|
||||
|
@ -203,11 +205,14 @@ class ActiveMasterManager extends ZooKeeperListener {
|
|||
LOG.debug("Interrupted waiting for master to die", e);
|
||||
}
|
||||
}
|
||||
if (!clusterStatusTracker.isClusterUp()) {
|
||||
this.master.stop("Cluster went down before this master became active");
|
||||
}
|
||||
if (this.master.isStopped()) {
|
||||
return cleanSetOfActiveMaster;
|
||||
}
|
||||
// Try to become active master again now that there is no active master
|
||||
blockUntilBecomingActiveMaster(startupStatus);
|
||||
blockUntilBecomingActiveMaster(startupStatus,clusterStatusTracker);
|
||||
}
|
||||
return cleanSetOfActiveMaster;
|
||||
}
|
||||
|
|
|
@ -398,7 +398,14 @@ Server {
|
|||
this);
|
||||
this.zooKeeper.registerListener(activeMasterManager);
|
||||
stallIfBackupMaster(this.conf, this.activeMasterManager);
|
||||
return this.activeMasterManager.blockUntilBecomingActiveMaster(startupStatus);
|
||||
|
||||
// The ClusterStatusTracker is setup before the other
|
||||
// ZKBasedSystemTrackers because it's needed by the activeMasterManager
|
||||
// to check if the cluster should be shutdown.
|
||||
this.clusterStatusTracker = new ClusterStatusTracker(getZooKeeper(), this);
|
||||
this.clusterStatusTracker.start();
|
||||
return this.activeMasterManager.blockUntilBecomingActiveMaster(startupStatus,
|
||||
this.clusterStatusTracker);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -427,8 +434,6 @@ Server {
|
|||
|
||||
// Set the cluster as up. If new RSs, they'll be waiting on this before
|
||||
// going ahead with their startup.
|
||||
this.clusterStatusTracker = new ClusterStatusTracker(getZooKeeper(), this);
|
||||
this.clusterStatusTracker.start();
|
||||
boolean wasUp = this.clusterStatusTracker.isClusterUp();
|
||||
if (!wasUp) this.clusterStatusTracker.setClusterUp();
|
||||
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
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.hbase.zookeeper.ClusterStatusTracker;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
|
@ -66,30 +67,33 @@ public class TestActiveMasterManager {
|
|||
"testActiveMasterManagerFromZK", null, true);
|
||||
try {
|
||||
ZKUtil.deleteNode(zk, zk.masterAddressZNode);
|
||||
ZKUtil.deleteNode(zk, zk.clusterStateZNode);
|
||||
} catch(KeeperException.NoNodeException nne) {}
|
||||
|
||||
// Create the master node with a dummy address
|
||||
ServerName master = new ServerName("localhost", 1, System.currentTimeMillis());
|
||||
// Should not have a master yet
|
||||
DummyMaster dummyMaster = new DummyMaster();
|
||||
ActiveMasterManager activeMasterManager = new ActiveMasterManager(zk,
|
||||
master, dummyMaster);
|
||||
zk.registerListener(activeMasterManager);
|
||||
DummyMaster dummyMaster = new DummyMaster(zk,master);
|
||||
ClusterStatusTracker clusterStatusTracker =
|
||||
dummyMaster.getClusterStatusTracker();
|
||||
ActiveMasterManager activeMasterManager =
|
||||
dummyMaster.getActiveMasterManager();
|
||||
assertFalse(activeMasterManager.clusterHasActiveMaster.get());
|
||||
|
||||
// First test becoming the active master uninterrupted
|
||||
MonitoredTask status = Mockito.mock(MonitoredTask.class);
|
||||
activeMasterManager.blockUntilBecomingActiveMaster(status);
|
||||
clusterStatusTracker.setClusterUp();
|
||||
|
||||
activeMasterManager.blockUntilBecomingActiveMaster(status,clusterStatusTracker);
|
||||
assertTrue(activeMasterManager.clusterHasActiveMaster.get());
|
||||
assertMaster(zk, master);
|
||||
|
||||
// Now pretend master restart
|
||||
DummyMaster secondDummyMaster = new DummyMaster();
|
||||
ActiveMasterManager secondActiveMasterManager = new ActiveMasterManager(zk,
|
||||
master, secondDummyMaster);
|
||||
zk.registerListener(secondActiveMasterManager);
|
||||
DummyMaster secondDummyMaster = new DummyMaster(zk,master);
|
||||
ActiveMasterManager secondActiveMasterManager =
|
||||
secondDummyMaster.getActiveMasterManager();
|
||||
assertFalse(secondActiveMasterManager.clusterHasActiveMaster.get());
|
||||
activeMasterManager.blockUntilBecomingActiveMaster(status);
|
||||
activeMasterManager.blockUntilBecomingActiveMaster(status,clusterStatusTracker);
|
||||
assertTrue(activeMasterManager.clusterHasActiveMaster.get());
|
||||
assertMaster(zk, master);
|
||||
}
|
||||
|
@ -105,6 +109,7 @@ public class TestActiveMasterManager {
|
|||
"testActiveMasterManagerFromZK", null, true);
|
||||
try {
|
||||
ZKUtil.deleteNode(zk, zk.masterAddressZNode);
|
||||
ZKUtil.deleteNode(zk, zk.clusterStateZNode);
|
||||
} catch(KeeperException.NoNodeException nne) {}
|
||||
|
||||
// Create the master node with a dummy address
|
||||
|
@ -114,21 +119,22 @@ public class TestActiveMasterManager {
|
|||
new ServerName("localhost", 2, System.currentTimeMillis());
|
||||
|
||||
// Should not have a master yet
|
||||
DummyMaster ms1 = new DummyMaster();
|
||||
ActiveMasterManager activeMasterManager = new ActiveMasterManager(zk,
|
||||
firstMasterAddress, ms1);
|
||||
zk.registerListener(activeMasterManager);
|
||||
DummyMaster ms1 = new DummyMaster(zk,firstMasterAddress);
|
||||
ActiveMasterManager activeMasterManager =
|
||||
ms1.getActiveMasterManager();
|
||||
assertFalse(activeMasterManager.clusterHasActiveMaster.get());
|
||||
|
||||
// First test becoming the active master uninterrupted
|
||||
ClusterStatusTracker clusterStatusTracker =
|
||||
ms1.getClusterStatusTracker();
|
||||
clusterStatusTracker.setClusterUp();
|
||||
activeMasterManager.blockUntilBecomingActiveMaster(
|
||||
Mockito.mock(MonitoredTask.class));
|
||||
Mockito.mock(MonitoredTask.class),clusterStatusTracker);
|
||||
assertTrue(activeMasterManager.clusterHasActiveMaster.get());
|
||||
assertMaster(zk, firstMasterAddress);
|
||||
|
||||
// New manager will now try to become the active master in another thread
|
||||
WaitToBeMasterThread t = new WaitToBeMasterThread(zk, secondMasterAddress);
|
||||
zk.registerListener(t.manager);
|
||||
t.start();
|
||||
// Wait for this guy to figure out there is another active master
|
||||
// Wait for 1 second at most
|
||||
|
@ -193,18 +199,20 @@ public class TestActiveMasterManager {
|
|||
public static class WaitToBeMasterThread extends Thread {
|
||||
|
||||
ActiveMasterManager manager;
|
||||
DummyMaster dummyMaster;
|
||||
boolean isActiveMaster;
|
||||
|
||||
public WaitToBeMasterThread(ZooKeeperWatcher zk, ServerName address) {
|
||||
this.manager = new ActiveMasterManager(zk, address,
|
||||
new DummyMaster());
|
||||
this.dummyMaster = new DummyMaster(zk,address);
|
||||
this.manager = this.dummyMaster.getActiveMasterManager();
|
||||
isActiveMaster = false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
manager.blockUntilBecomingActiveMaster(
|
||||
Mockito.mock(MonitoredTask.class));
|
||||
Mockito.mock(MonitoredTask.class),
|
||||
this.dummyMaster.getClusterStatusTracker());
|
||||
LOG.info("Second master has become the active master!");
|
||||
isActiveMaster = true;
|
||||
}
|
||||
|
@ -240,6 +248,18 @@ public class TestActiveMasterManager {
|
|||
*/
|
||||
public static class DummyMaster implements Server {
|
||||
private volatile boolean stopped;
|
||||
private ClusterStatusTracker clusterStatusTracker;
|
||||
private ActiveMasterManager activeMasterManager;
|
||||
|
||||
public DummyMaster(ZooKeeperWatcher zk, ServerName master) {
|
||||
this.clusterStatusTracker =
|
||||
new ClusterStatusTracker(zk, this);
|
||||
clusterStatusTracker.start();
|
||||
|
||||
this.activeMasterManager =
|
||||
new ActiveMasterManager(zk, master, this);
|
||||
zk.registerListener(activeMasterManager);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort(final String msg, final Throwable t) {}
|
||||
|
@ -278,6 +298,14 @@ public class TestActiveMasterManager {
|
|||
public CatalogTracker getCatalogTracker() {
|
||||
return null;
|
||||
}
|
||||
|
||||
public ClusterStatusTracker getClusterStatusTracker() {
|
||||
return clusterStatusTracker;
|
||||
}
|
||||
|
||||
public ActiveMasterManager getActiveMasterManager() {
|
||||
return activeMasterManager;
|
||||
}
|
||||
}
|
||||
|
||||
@org.junit.Rule
|
||||
|
|
|
@ -0,0 +1,99 @@
|
|||
/**
|
||||
* Copyright The Apache Software Foundation
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
@Category(LargeTests.class)
|
||||
public class TestMasterShutdown {
|
||||
private static final Log LOG = LogFactory.getLog(TestMasterShutdown.class);
|
||||
|
||||
/**
|
||||
* Simple test of shutdown.
|
||||
* <p>
|
||||
* Starts with three masters. Tells the active master to shutdown the cluster.
|
||||
* Verifies that all masters are properly shutdown.
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test (timeout=240000)
|
||||
public void testMasterShutdown() throws Exception {
|
||||
|
||||
final int NUM_MASTERS = 3;
|
||||
final int NUM_RS = 3;
|
||||
|
||||
// Create config to use for this cluster
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
|
||||
// Start the cluster
|
||||
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
|
||||
TEST_UTIL.startMiniCluster(NUM_MASTERS, NUM_RS);
|
||||
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
|
||||
|
||||
// get all the master threads
|
||||
List<MasterThread> masterThreads = cluster.getMasterThreads();
|
||||
|
||||
// wait for each to come online
|
||||
for (MasterThread mt : masterThreads) {
|
||||
assertTrue(mt.isAlive());
|
||||
}
|
||||
|
||||
// find the active master
|
||||
HMaster active = null;
|
||||
for (int i = 0; i < masterThreads.size(); i++) {
|
||||
if (masterThreads.get(i).getMaster().isActiveMaster()) {
|
||||
active = masterThreads.get(i).getMaster();
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertNotNull(active);
|
||||
// make sure the other two are backup masters
|
||||
ClusterStatus status = active.getClusterStatus();
|
||||
assertEquals(2, status.getBackupMastersSize());
|
||||
assertEquals(2, status.getBackupMasters().size());
|
||||
|
||||
// tell the active master to shutdown the cluster
|
||||
active.shutdown();
|
||||
|
||||
for (int i = NUM_MASTERS - 1; i >= 0 ;--i) {
|
||||
cluster.waitOnMaster(i);
|
||||
}
|
||||
// make sure all the masters properly shutdown
|
||||
assertEquals(0,masterThreads.size());
|
||||
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
@org.junit.Rule
|
||||
public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
|
||||
new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
|
||||
}
|
||||
|
Loading…
Reference in New Issue