HBASE-23808 [Flakey Test] TestMasterShutdown#testMasterShutdownBefore… (#1141)
Signed-off-by: Bharath Vissapragada <bharathv@apache.org> Signed-off-by: stack <stack@apache.org>
This commit is contained in:
parent
b7ef225609
commit
533302adde
|
@ -505,8 +505,8 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
* </ol>
|
* </ol>
|
||||||
* <p>
|
* <p>
|
||||||
* Remaining steps of initialization occur in
|
* Remaining steps of initialization occur in
|
||||||
* #finishActiveMasterInitialization(MonitoredTask) after
|
* {@link #finishActiveMasterInitialization(MonitoredTask)} after the master becomes the
|
||||||
* the master becomes the active one.
|
* active one.
|
||||||
*/
|
*/
|
||||||
public HMaster(final Configuration conf)
|
public HMaster(final Configuration conf)
|
||||||
throws IOException, KeeperException {
|
throws IOException, KeeperException {
|
||||||
|
@ -2807,6 +2807,7 @@ public class HMaster extends HRegionServer implements MasterServices {
|
||||||
if (cpHost != null) {
|
if (cpHost != null) {
|
||||||
cpHost.preShutdown();
|
cpHost.preShutdown();
|
||||||
}
|
}
|
||||||
|
|
||||||
// Tell the servermanager cluster shutdown has been called. This makes it so when Master is
|
// Tell the servermanager cluster shutdown has been called. This makes it so when Master is
|
||||||
// last running server, it'll stop itself. Next, we broadcast the cluster shutdown by setting
|
// last running server, it'll stop itself. Next, we broadcast the cluster shutdown by setting
|
||||||
// the cluster status as down. RegionServers will notice this change in state and will start
|
// the cluster status as down. RegionServers will notice this change in state and will start
|
||||||
|
|
|
@ -1,4 +1,4 @@
|
||||||
/**
|
/*
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
|
@ -18,10 +18,14 @@
|
||||||
package org.apache.hadoop.hbase.master;
|
package org.apache.hadoop.hbase.master;
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertNotEquals;
|
||||||
import static org.junit.Assert.assertNotNull;
|
import static org.junit.Assert.assertNotNull;
|
||||||
import static org.junit.Assert.assertTrue;
|
import java.io.IOException;
|
||||||
|
import java.time.Duration;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.concurrent.CompletableFuture;
|
||||||
|
import java.util.concurrent.CompletionException;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.ClusterMetrics;
|
import org.apache.hadoop.hbase.ClusterMetrics;
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
|
@ -30,26 +34,40 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||||
import org.apache.hadoop.hbase.LocalHBaseCluster;
|
import org.apache.hadoop.hbase.LocalHBaseCluster;
|
||||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||||
import org.apache.hadoop.hbase.StartMiniClusterOption;
|
import org.apache.hadoop.hbase.StartMiniClusterOption;
|
||||||
import org.apache.hadoop.hbase.client.Admin;
|
import org.apache.hadoop.hbase.Waiter;
|
||||||
import org.apache.hadoop.hbase.client.Connection;
|
import org.apache.hadoop.hbase.client.AsyncConnection;
|
||||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||||
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
|
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
|
||||||
|
import org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient;
|
||||||
|
import org.junit.Before;
|
||||||
import org.junit.ClassRule;
|
import org.junit.ClassRule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.experimental.categories.Category;
|
import org.junit.experimental.categories.Category;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
|
||||||
|
|
||||||
@Category({MasterTests.class, LargeTests.class})
|
@Category({MasterTests.class, LargeTests.class})
|
||||||
public class TestMasterShutdown {
|
public class TestMasterShutdown {
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(TestMasterShutdown.class);
|
||||||
|
|
||||||
@ClassRule
|
@ClassRule
|
||||||
public static final HBaseClassTestRule CLASS_RULE =
|
public static final HBaseClassTestRule CLASS_RULE =
|
||||||
HBaseClassTestRule.forClass(TestMasterShutdown.class);
|
HBaseClassTestRule.forClass(TestMasterShutdown.class);
|
||||||
|
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(TestMasterShutdown.class);
|
private HBaseTestingUtility htu;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void shutdownCluster() throws IOException {
|
||||||
|
if (htu != null) {
|
||||||
|
// an extra check in case the test cluster was not terminated after HBaseClassTestRule's
|
||||||
|
// Timeout interrupted the test thread.
|
||||||
|
LOG.warn("found non-null TestingUtility -- previous test did not terminate cleanly.");
|
||||||
|
htu.shutdownMiniCluster();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Simple test of shutdown.
|
* Simple test of shutdown.
|
||||||
|
@ -59,110 +77,164 @@ public class TestMasterShutdown {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testMasterShutdown() throws Exception {
|
public void testMasterShutdown() throws Exception {
|
||||||
final int NUM_MASTERS = 3;
|
|
||||||
final int NUM_RS = 3;
|
|
||||||
|
|
||||||
// Create config to use for this cluster
|
// Create config to use for this cluster
|
||||||
Configuration conf = HBaseConfiguration.create();
|
Configuration conf = HBaseConfiguration.create();
|
||||||
|
|
||||||
// Start the cluster
|
// Start the cluster
|
||||||
HBaseTestingUtility htu = new HBaseTestingUtility(conf);
|
try {
|
||||||
StartMiniClusterOption option = StartMiniClusterOption.builder()
|
htu = new HBaseTestingUtility(conf);
|
||||||
.numMasters(NUM_MASTERS).numRegionServers(NUM_RS).numDataNodes(NUM_RS).build();
|
StartMiniClusterOption option = StartMiniClusterOption.builder()
|
||||||
htu.startMiniCluster(option);
|
.numMasters(3)
|
||||||
MiniHBaseCluster cluster = htu.getHBaseCluster();
|
.numRegionServers(3)
|
||||||
|
.numDataNodes(3)
|
||||||
|
.build();
|
||||||
|
final MiniHBaseCluster cluster = htu.startMiniCluster(option);
|
||||||
|
|
||||||
// get all the master threads
|
// wait for all master thread to spawn and start their run loop.
|
||||||
List<MasterThread> masterThreads = cluster.getMasterThreads();
|
final long thirtySeconds = TimeUnit.SECONDS.toMillis(30);
|
||||||
|
final long oneSecond = TimeUnit.SECONDS.toMillis(1);
|
||||||
|
assertNotEquals(-1, htu.waitFor(thirtySeconds, oneSecond, () -> {
|
||||||
|
final List<MasterThread> masterThreads = cluster.getMasterThreads();
|
||||||
|
return masterThreads != null
|
||||||
|
&& masterThreads.size() >= 3
|
||||||
|
&& masterThreads.stream().allMatch(Thread::isAlive);
|
||||||
|
}));
|
||||||
|
|
||||||
// wait for each to come online
|
// find the active master
|
||||||
for (MasterThread mt : masterThreads) {
|
final HMaster active = cluster.getMaster();
|
||||||
assertTrue(mt.isAlive());
|
assertNotNull(active);
|
||||||
}
|
|
||||||
|
|
||||||
// find the active master
|
// make sure the other two are backup masters
|
||||||
HMaster active = null;
|
ClusterMetrics status = active.getClusterMetrics();
|
||||||
for (int i = 0; i < masterThreads.size(); i++) {
|
assertEquals(2, status.getBackupMasterNames().size());
|
||||||
if (masterThreads.get(i).getMaster().isActiveMaster()) {
|
|
||||||
active = masterThreads.get(i).getMaster();
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
assertNotNull(active);
|
|
||||||
// make sure the other two are backup masters
|
|
||||||
ClusterMetrics status = active.getClusterMetrics();
|
|
||||||
assertEquals(2, status.getBackupMasterNames().size());
|
|
||||||
|
|
||||||
// tell the active master to shutdown the cluster
|
// tell the active master to shutdown the cluster
|
||||||
active.shutdown();
|
active.shutdown();
|
||||||
|
assertNotEquals(-1, htu.waitFor(thirtySeconds, oneSecond,
|
||||||
for (int i = NUM_MASTERS - 1; i >= 0 ;--i) {
|
() -> CollectionUtils.isEmpty(cluster.getLiveMasterThreads())));
|
||||||
cluster.waitOnMaster(i);
|
assertNotEquals(-1, htu.waitFor(thirtySeconds, oneSecond,
|
||||||
}
|
() -> CollectionUtils.isEmpty(cluster.getLiveRegionServerThreads())));
|
||||||
// make sure all the masters properly shutdown
|
} finally {
|
||||||
assertEquals(0, masterThreads.size());
|
if (htu != null) {
|
||||||
|
htu.shutdownMiniCluster();
|
||||||
htu.shutdownMiniCluster();
|
htu = null;
|
||||||
}
|
|
||||||
|
|
||||||
private Connection createConnection(HBaseTestingUtility util) throws InterruptedException {
|
|
||||||
// the cluster may have not been initialized yet which means we can not get the cluster id thus
|
|
||||||
// an exception will be thrown. So here we need to retry.
|
|
||||||
for (;;) {
|
|
||||||
try {
|
|
||||||
return ConnectionFactory.createConnection(util.getConfiguration());
|
|
||||||
} catch (Exception e) {
|
|
||||||
Thread.sleep(10);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This test appears to be an intentional race between a thread that issues a shutdown RPC to the
|
||||||
|
* master, while the master is concurrently realizing it cannot initialize because there are no
|
||||||
|
* region servers available to it. The expected behavior is that master initialization is
|
||||||
|
* interruptable via said shutdown RPC.
|
||||||
|
*/
|
||||||
@Test
|
@Test
|
||||||
public void testMasterShutdownBeforeStartingAnyRegionServer() throws Exception {
|
public void testMasterShutdownBeforeStartingAnyRegionServer() throws Exception {
|
||||||
final int NUM_MASTERS = 1;
|
LocalHBaseCluster hbaseCluster = null;
|
||||||
final int NUM_RS = 0;
|
try {
|
||||||
|
htu = new HBaseTestingUtility(
|
||||||
|
createMasterShutdownBeforeStartingAnyRegionServerConfiguration());
|
||||||
|
|
||||||
// Create config to use for this cluster
|
// configure a cluster with
|
||||||
Configuration conf = HBaseConfiguration.create();
|
final StartMiniClusterOption options = StartMiniClusterOption.builder()
|
||||||
conf.setInt("hbase.ipc.client.failed.servers.expiry", 200);
|
.numDataNodes(1)
|
||||||
conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
|
.numMasters(1)
|
||||||
|
.numRegionServers(0)
|
||||||
|
.masterClass(HMaster.class)
|
||||||
|
.rsClass(MiniHBaseCluster.MiniHBaseClusterRegionServer.class)
|
||||||
|
.createRootDir(true)
|
||||||
|
.build();
|
||||||
|
|
||||||
// Start the cluster
|
// Can't simply `htu.startMiniCluster(options)` because that method waits for the master to
|
||||||
final HBaseTestingUtility util = new HBaseTestingUtility(conf);
|
// start completely. However, this test's premise is that a partially started master should
|
||||||
util.startMiniDFSCluster(3);
|
// still respond to a shutdown RPC. So instead, we manage each component lifecycle
|
||||||
util.startMiniZKCluster();
|
// independently.
|
||||||
util.createRootDir();
|
// I think it's not worth refactoring HTU's helper methods just for this class.
|
||||||
final LocalHBaseCluster cluster =
|
htu.startMiniDFSCluster(options.getNumDataNodes());
|
||||||
new LocalHBaseCluster(conf, NUM_MASTERS, NUM_RS, HMaster.class,
|
htu.startMiniZKCluster(options.getNumZkServers());
|
||||||
MiniHBaseCluster.MiniHBaseClusterRegionServer.class);
|
htu.createRootDir();
|
||||||
final int MASTER_INDEX = 0;
|
hbaseCluster = new LocalHBaseCluster(htu.getConfiguration(), options.getNumMasters(),
|
||||||
final MasterThread master = cluster.getMasters().get(MASTER_INDEX);
|
options.getNumRegionServers(), options.getMasterClass(), options.getRsClass());
|
||||||
master.start();
|
final MasterThread masterThread = hbaseCluster.getMasters().get(0);
|
||||||
LOG.info("Called master start on " + master.getName());
|
|
||||||
Thread shutdownThread = new Thread("Shutdown-Thread") {
|
final CompletableFuture<Void> shutdownFuture = CompletableFuture.runAsync(() -> {
|
||||||
@Override
|
// Switching to master registry exacerbated a race in the master bootstrap that can result
|
||||||
public void run() {
|
// in a lost shutdown command (HBASE-8422, HBASE-23836). The race is essentially because
|
||||||
LOG.info("Before call to shutdown master");
|
// the server manager in HMaster is not initialized by the time shutdown() RPC (below) is
|
||||||
try (Connection connection = createConnection(util); Admin admin = connection.getAdmin()) {
|
// made to the master. The suspected reason as to why it was uncommon before HBASE-18095
|
||||||
admin.shutdown();
|
// is because the connection creation with ZK registry is so slow that by then the server
|
||||||
} catch (Exception e) {
|
// manager is usually init'ed in time for the RPC to be made. For now, adding an explicit
|
||||||
LOG.info("Error while calling Admin.shutdown, which is expected: " + e.getMessage());
|
// wait() in the test, waiting for the server manager to become available.
|
||||||
}
|
final long timeout = TimeUnit.MINUTES.toMillis(10);
|
||||||
LOG.info("After call to shutdown master");
|
assertNotEquals("timeout waiting for server manager to become available.",
|
||||||
cluster.waitOnMaster(MASTER_INDEX);
|
-1, Waiter.waitFor(htu.getConfiguration(), timeout,
|
||||||
|
() -> masterThread.getMaster().getServerManager() != null));
|
||||||
|
|
||||||
|
// Master has come up far enough that we can terminate it without creating a zombie.
|
||||||
|
final long result = Waiter.waitFor(htu.getConfiguration(), timeout, 500, () -> {
|
||||||
|
final Configuration conf = createResponsiveZkConfig(htu.getConfiguration());
|
||||||
|
LOG.debug("Attempting to establish connection.");
|
||||||
|
final CompletableFuture<AsyncConnection> connFuture =
|
||||||
|
ConnectionFactory.createAsyncConnection(conf);
|
||||||
|
try (final AsyncConnection conn = connFuture.join()) {
|
||||||
|
LOG.debug("Sending shutdown RPC.");
|
||||||
|
try {
|
||||||
|
conn.getAdmin().shutdown().join();
|
||||||
|
LOG.debug("Shutdown RPC sent.");
|
||||||
|
return true;
|
||||||
|
} catch (CompletionException e) {
|
||||||
|
LOG.debug("Failure sending shutdown RPC.");
|
||||||
|
}
|
||||||
|
} catch (IOException|CompletionException e) {
|
||||||
|
LOG.debug("Failed to establish connection.");
|
||||||
|
} catch (Throwable e) {
|
||||||
|
LOG.info("Something unexpected happened.", e);
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
});
|
||||||
|
assertNotEquals("Failed to issue shutdown RPC after " + Duration.ofMillis(timeout),
|
||||||
|
-1, result);
|
||||||
|
});
|
||||||
|
|
||||||
|
masterThread.start();
|
||||||
|
shutdownFuture.join();
|
||||||
|
masterThread.join();
|
||||||
|
} finally {
|
||||||
|
if (hbaseCluster != null) {
|
||||||
|
hbaseCluster.shutdown();
|
||||||
}
|
}
|
||||||
};
|
if (htu != null) {
|
||||||
shutdownThread.start();
|
htu.shutdownMiniCluster();
|
||||||
LOG.info("Called master join on " + master.getName());
|
htu = null;
|
||||||
master.join();
|
}
|
||||||
shutdownThread.join();
|
}
|
||||||
|
}
|
||||||
|
|
||||||
List<MasterThread> masterThreads = cluster.getMasters();
|
/**
|
||||||
// make sure all the masters properly shutdown
|
* Create a cluster configuration suitable for
|
||||||
assertEquals(0, masterThreads.size());
|
* {@link #testMasterShutdownBeforeStartingAnyRegionServer()}.
|
||||||
|
*/
|
||||||
|
private static Configuration createMasterShutdownBeforeStartingAnyRegionServerConfiguration() {
|
||||||
|
final Configuration conf = HBaseConfiguration.create();
|
||||||
|
// make sure the master will wait forever in the absence of a RS.
|
||||||
|
conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
|
||||||
|
// don't need a long write pipeline for this test.
|
||||||
|
conf.setInt("dfs.replication", 1);
|
||||||
|
return conf;
|
||||||
|
}
|
||||||
|
|
||||||
util.shutdownMiniZKCluster();
|
/**
|
||||||
util.shutdownMiniDFSCluster();
|
* Create a new {@link Configuration} based on {@code baseConf} that has ZooKeeper connection
|
||||||
util.cleanupTestDir();
|
* settings tuned very aggressively. The resulting client is used within a retry loop, so there's
|
||||||
|
* no value in having the client itself do the retries. We want to iterate on the base
|
||||||
|
* configuration because we're waiting for the mini-cluster to start and set it's ZK client port.
|
||||||
|
*
|
||||||
|
* @return a new, configured {@link Configuration} instance.
|
||||||
|
*/
|
||||||
|
private static Configuration createResponsiveZkConfig(final Configuration baseConf) {
|
||||||
|
final Configuration conf = HBaseConfiguration.create(baseConf);
|
||||||
|
conf.setInt(ReadOnlyZKClient.RECOVERY_RETRY, 3);
|
||||||
|
conf.setInt(ReadOnlyZKClient.RECOVERY_RETRY_INTERVAL_MILLIS, 100);
|
||||||
|
return conf;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue