HBASE-23735 [Flakey Tests] TestClusterRestartFailover & TestClusterRestartFailoverSplitWithoutZk
This commit is contained in:
parent
ee64aa044d
commit
134242720d
|
@ -1,4 +1,4 @@
|
|||
/**
|
||||
/*
|
||||
* 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
|
||||
|
@ -40,8 +40,10 @@ import org.apache.hadoop.hbase.master.assignment.ServerState;
|
|||
import org.apache.hadoop.hbase.master.assignment.ServerStateNode;
|
||||
import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MasterTests;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
|
@ -58,7 +60,7 @@ public class TestClusterRestartFailover extends AbstractTestRestartCluster {
|
|||
|
||||
private static final Logger LOG = LoggerFactory.getLogger(TestClusterRestartFailover.class);
|
||||
|
||||
private static CountDownLatch SCP_LATCH;
|
||||
private volatile static CountDownLatch SCP_LATCH;
|
||||
private static ServerName SERVER_FOR_TEST;
|
||||
|
||||
@Override
|
||||
|
@ -79,7 +81,16 @@ public class TestClusterRestartFailover extends AbstractTestRestartCluster {
|
|||
setupCluster();
|
||||
setupTable();
|
||||
|
||||
SERVER_FOR_TEST = UTIL.getHBaseCluster().getRegionServer(0).getServerName();
|
||||
// Find server that does not have hbase:namespace on it. This tests holds up SCPs. If it
|
||||
// holds up the server w/ hbase:namespace, the Master initialization will be held up
|
||||
// because this table is not online and test fails.
|
||||
for (JVMClusterUtil.RegionServerThread rst:
|
||||
UTIL.getHBaseCluster().getLiveRegionServerThreads()) {
|
||||
HRegionServer rs = rst.getRegionServer();
|
||||
if (rs.getRegions(TableName.NAMESPACE_TABLE_NAME).isEmpty()) {
|
||||
SERVER_FOR_TEST = rs.getServerName();
|
||||
}
|
||||
}
|
||||
UTIL.waitFor(60000, () -> getServerStateNode(SERVER_FOR_TEST) != null);
|
||||
ServerStateNode serverNode = getServerStateNode(SERVER_FOR_TEST);
|
||||
assertNotNull(serverNode);
|
||||
|
@ -98,8 +109,9 @@ public class TestClusterRestartFailover extends AbstractTestRestartCluster {
|
|||
LOG.info("Restarting cluster");
|
||||
UTIL.restartHBaseCluster(StartMiniClusterOption.builder().masterClass(HMasterForTest.class)
|
||||
.numMasters(1).numRegionServers(3).rsPorts(ports).build());
|
||||
LOG.info("Started cluster");
|
||||
UTIL.waitFor(60000, () -> UTIL.getHBaseCluster().getMaster().isInitialized());
|
||||
|
||||
LOG.info("Started cluster master, waiting for {}", SERVER_FOR_TEST);
|
||||
UTIL.waitFor(60000, () -> getServerStateNode(SERVER_FOR_TEST) != null);
|
||||
serverNode = getServerStateNode(SERVER_FOR_TEST);
|
||||
assertFalse("serverNode should not be ONLINE during SCP processing",
|
||||
|
@ -113,6 +125,7 @@ public class TestClusterRestartFailover extends AbstractTestRestartCluster {
|
|||
Procedure.NO_PROC_ID);
|
||||
|
||||
// Wait the SCP to finish
|
||||
LOG.info("Waiting on latch");
|
||||
SCP_LATCH.countDown();
|
||||
UTIL.waitFor(60000, () -> procedure.get().isFinished());
|
||||
|
||||
|
@ -126,13 +139,17 @@ public class TestClusterRestartFailover extends AbstractTestRestartCluster {
|
|||
}
|
||||
|
||||
private void setupCluster() throws Exception {
|
||||
LOG.info("Setup cluster");
|
||||
UTIL.startMiniCluster(
|
||||
StartMiniClusterOption.builder().masterClass(HMasterForTest.class).numMasters(1)
|
||||
.numRegionServers(3).build());
|
||||
LOG.info("Cluster is up");
|
||||
UTIL.waitFor(60000, () -> UTIL.getMiniHBaseCluster().getMaster().isInitialized());
|
||||
LOG.info("Master is up");
|
||||
// wait for all SCPs finished
|
||||
UTIL.waitFor(60000, () -> UTIL.getHBaseCluster().getMaster().getProcedures().stream()
|
||||
.noneMatch(p -> p instanceof ServerCrashProcedure));
|
||||
LOG.info("No SCPs");
|
||||
}
|
||||
|
||||
private void setupTable() throws Exception {
|
||||
|
|
Loading…
Reference in New Issue