HBASE-22141 Fix TestAsyncDecommissionAdminApi

This commit is contained in:
Duo Zhang 2019-04-01 18:53:19 +08:00 committed by zhangduo
parent 1c4e705592
commit a3faad76e1
3 changed files with 24 additions and 19 deletions

View File

@ -84,7 +84,7 @@ public abstract class TestAsyncAdminBase {
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 120000);
TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
TEST_UTIL.getConfiguration().setInt(START_LOG_ERRORS_AFTER_COUNT_KEY, 0);
TEST_UTIL.startMiniCluster(3);
TEST_UTIL.startMiniCluster(2);
ASYNC_CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
}
@ -116,6 +116,9 @@ public abstract class TestAsyncAdminBase {
});
}
}, ForkJoinPool.commonPool()).join();
if (!admin.isBalancerEnabled().join()) {
admin.balancerSwitch(true, true);
}
}
protected void createTableWithDefaultConf(TableName tableName) throws IOException {

View File

@ -46,6 +46,7 @@ public class TestAsyncDecommissionAdminApi extends TestAsyncAdminBase {
@Test
public void testAsyncDecommissionRegionServers() throws Exception {
admin.balancerSwitch(false, true);
List<ServerName> decommissionedRegionServers = admin.listDecommissionedRegionServers().get();
assertTrue(decommissionedRegionServers.isEmpty());
@ -55,7 +56,8 @@ public class TestAsyncDecommissionAdminApi extends TestAsyncAdminBase {
new ArrayList<>(admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).get()
.getLiveServerMetrics().keySet());
assertEquals(2, clusterRegionServers.size());
assertEquals(TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().size(),
clusterRegionServers.size());
HashMap<ServerName, List<RegionInfo>> serversToDecommssion = new HashMap<>();
// Get a server that has regions. We will decommission one of the servers,
@ -84,6 +86,11 @@ public class TestAsyncDecommissionAdminApi extends TestAsyncAdminBase {
}
}
// Maybe the TRSP is still not finished at master side, since the reportRegionTransition just
// updates the procedure store, and we still need to wake up the procedure and execute it in the
// procedure executor, which is asynchronous
TEST_UTIL.waitUntilNoRegionsInTransition(10000);
// Recommission and load regions
for (ServerName server : serversToDecommssion.keySet()) {
List<byte[]> encodedRegionNames = serversToDecommssion.get(server).stream()

View File

@ -180,23 +180,18 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase {
public void testGetOnlineRegions() throws Exception {
createTableAndGetOneRegion(tableName);
AtomicInteger regionServerCount = new AtomicInteger(0);
TEST_UTIL
.getHBaseCluster()
.getLiveRegionServerThreads()
.stream()
.map(rsThread -> rsThread.getRegionServer())
.forEach(
rs -> {
ServerName serverName = rs.getServerName();
try {
assertEquals(admin.getRegions(serverName).get().size(), rs
.getRegions().size());
} catch (Exception e) {
fail("admin.getOnlineRegions() method throws a exception: " + e.getMessage());
}
regionServerCount.incrementAndGet();
});
assertEquals(3, regionServerCount.get());
TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().stream()
.map(rsThread -> rsThread.getRegionServer()).forEach(rs -> {
ServerName serverName = rs.getServerName();
try {
assertEquals(admin.getRegions(serverName).get().size(), rs.getRegions().size());
} catch (Exception e) {
fail("admin.getOnlineRegions() method throws a exception: " + e.getMessage());
}
regionServerCount.incrementAndGet();
});
assertEquals(TEST_UTIL.getHBaseCluster().getLiveRegionServerThreads().size(),
regionServerCount.get());
}
@Test