HBASE-24945 Remove MetaTableAccessor.getRegionCount (#2303)

Signed-off-by: Viraj Jasani <vjasani@apache.org>
This commit is contained in:
Duo Zhang 2020-08-25 21:15:22 +08:00 committed by GitHub
parent 6b0707f541
commit 227084c41f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 82 additions and 95 deletions

View File

@ -34,12 +34,10 @@ import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell.Type; import org.apache.hadoop.hbase.Cell.Type;
import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType; import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
import org.apache.hadoop.hbase.client.AsyncTable; import org.apache.hadoop.hbase.client.AsyncTable;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Consistency; import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
@ -47,7 +45,6 @@ import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.ResultScanner;
@ -743,33 +740,6 @@ public final class MetaTableAccessor {
updateTableState(conn, new TableState(tableName, actual)); updateTableState(conn, new TableState(tableName, actual));
} }
/**
* Count regions in <code>hbase:meta</code> for passed table.
* @param c Configuration object
* @param tableName table name to count regions for
* @return Count or regions in table <code>tableName</code>
*/
public static int getRegionCount(final Configuration c, final TableName tableName)
throws IOException {
try (Connection connection = ConnectionFactory.createConnection(c)) {
return getRegionCount(connection, tableName);
}
}
/**
* Count regions in <code>hbase:meta</code> for passed table.
* @param connection Connection object
* @param tableName table name to count regions for
* @return Count or regions in table <code>tableName</code>
*/
public static int getRegionCount(final Connection connection, final TableName tableName)
throws IOException {
try (RegionLocator locator = connection.getRegionLocator(tableName)) {
List<HRegionLocation> locations = locator.getAllRegionLocations();
return locations == null ? 0 : locations.size();
}
}
//////////////////////// ////////////////////////
// Editing operations // // Editing operations //
//////////////////////// ////////////////////////

View File

@ -18,14 +18,15 @@
*/ */
package org.apache.hadoop.hbase.mapred; package org.apache.hadoop.hbase.mapred;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.MutationSerialization; import org.apache.hadoop.hbase.mapreduce.MutationSerialization;
@ -283,8 +284,7 @@ public class TableMapReduceUtil {
MutationSerialization.class.getName(), ResultSerialization.class.getName()); MutationSerialization.class.getName(), ResultSerialization.class.getName());
if (partitioner == HRegionPartitioner.class) { if (partitioner == HRegionPartitioner.class) {
job.setPartitionerClass(HRegionPartitioner.class); job.setPartitionerClass(HRegionPartitioner.class);
int regions = int regions = getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table));
MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table));
if (job.getNumReduceTasks() > regions) { if (job.getNumReduceTasks() > regions) {
job.setNumReduceTasks(regions); job.setNumReduceTasks(regions);
} }
@ -330,12 +330,11 @@ public class TableMapReduceUtil {
* @throws IOException When retrieving the table details fails. * @throws IOException When retrieving the table details fails.
*/ */
// Used by tests. // Used by tests.
public static void limitNumReduceTasks(String table, JobConf job) public static void limitNumReduceTasks(String table, JobConf job) throws IOException {
throws IOException { int regions = getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table));
int regions = if (job.getNumReduceTasks() > regions) {
MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table));
if (job.getNumReduceTasks() > regions)
job.setNumReduceTasks(regions); job.setNumReduceTasks(regions);
}
} }
/** /**
@ -347,12 +346,11 @@ public class TableMapReduceUtil {
* @throws IOException When retrieving the table details fails. * @throws IOException When retrieving the table details fails.
*/ */
// Used by tests. // Used by tests.
public static void limitNumMapTasks(String table, JobConf job) public static void limitNumMapTasks(String table, JobConf job) throws IOException {
throws IOException { int regions = getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table));
int regions = if (job.getNumMapTasks() > regions) {
MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table));
if (job.getNumMapTasks() > regions)
job.setNumMapTasks(regions); job.setNumMapTasks(regions);
}
} }
/** /**
@ -363,10 +361,8 @@ public class TableMapReduceUtil {
* @param job The current job configuration to adjust. * @param job The current job configuration to adjust.
* @throws IOException When retrieving the table details fails. * @throws IOException When retrieving the table details fails.
*/ */
public static void setNumReduceTasks(String table, JobConf job) public static void setNumReduceTasks(String table, JobConf job) throws IOException {
throws IOException { job.setNumReduceTasks(getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table)));
job.setNumReduceTasks(MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job),
TableName.valueOf(table)));
} }
/** /**
@ -377,10 +373,8 @@ public class TableMapReduceUtil {
* @param job The current job configuration to adjust. * @param job The current job configuration to adjust.
* @throws IOException When retrieving the table details fails. * @throws IOException When retrieving the table details fails.
*/ */
public static void setNumMapTasks(String table, JobConf job) public static void setNumMapTasks(String table, JobConf job) throws IOException {
throws IOException { job.setNumMapTasks(getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table)));
job.setNumMapTasks(MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job),
TableName.valueOf(table)));
} }
/** /**
@ -412,4 +406,12 @@ public class TableMapReduceUtil {
job.getClass("mapred.output.format.class", TextOutputFormat.class, OutputFormat.class), job.getClass("mapred.output.format.class", TextOutputFormat.class, OutputFormat.class),
job.getCombinerClass()); job.getCombinerClass());
} }
private static int getRegionCount(Configuration conf, TableName tableName) throws IOException {
try (Connection conn = ConnectionFactory.createConnection(conf);
RegionLocator locator = conn.getRegionLocator(tableName)) {
return locator.getAllRegionLocations().size();
}
}
} }

View File

@ -39,7 +39,6 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger; import org.slf4j.Logger;
@ -47,6 +46,7 @@ import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@ -731,7 +731,7 @@ public class TableMapReduceUtil {
job.setOutputValueClass(Writable.class); job.setOutputValueClass(Writable.class);
if (partitioner == HRegionPartitioner.class) { if (partitioner == HRegionPartitioner.class) {
job.setPartitionerClass(HRegionPartitioner.class); job.setPartitionerClass(HRegionPartitioner.class);
int regions = MetaTableAccessor.getRegionCount(conf, TableName.valueOf(table)); int regions = getRegionCount(conf, TableName.valueOf(table));
if (job.getNumReduceTasks() > regions) { if (job.getNumReduceTasks() > regions) {
job.setNumReduceTasks(regions); job.setNumReduceTasks(regions);
} }
@ -754,12 +754,11 @@ public class TableMapReduceUtil {
* @param job The current job to adjust. * @param job The current job to adjust.
* @throws IOException When retrieving the table details fails. * @throws IOException When retrieving the table details fails.
*/ */
public static void limitNumReduceTasks(String table, Job job) public static void limitNumReduceTasks(String table, Job job) throws IOException {
throws IOException { int regions = getRegionCount(job.getConfiguration(), TableName.valueOf(table));
int regions = if (job.getNumReduceTasks() > regions) {
MetaTableAccessor.getRegionCount(job.getConfiguration(), TableName.valueOf(table));
if (job.getNumReduceTasks() > regions)
job.setNumReduceTasks(regions); job.setNumReduceTasks(regions);
}
} }
/** /**
@ -770,10 +769,8 @@ public class TableMapReduceUtil {
* @param job The current job to adjust. * @param job The current job to adjust.
* @throws IOException When retrieving the table details fails. * @throws IOException When retrieving the table details fails.
*/ */
public static void setNumReduceTasks(String table, Job job) public static void setNumReduceTasks(String table, Job job) throws IOException {
throws IOException { job.setNumReduceTasks(getRegionCount(job.getConfiguration(), TableName.valueOf(table)));
job.setNumReduceTasks(MetaTableAccessor.getRegionCount(job.getConfiguration(),
TableName.valueOf(table)));
} }
/** /**
@ -1056,4 +1053,11 @@ public class TableMapReduceUtil {
return ret; return ret;
} }
private static int getRegionCount(Configuration conf, TableName tableName) throws IOException {
try (Connection conn = ConnectionFactory.createConnection(conf);
RegionLocator locator = conn.getRegionLocator(tableName)) {
return locator.getAllRegionLocations().size();
}
}
} }

View File

@ -22,7 +22,6 @@ import static org.junit.Assert.assertEquals;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.testclassification.MapReduceTests; import org.apache.hadoop.hbase.testclassification.MapReduceTests;
@ -81,14 +80,13 @@ public class TestHRegionPartitioner {
@Test @Test
public void testHRegionPartitionerMoreRegions() throws Exception { public void testHRegionPartitionerMoreRegions() throws Exception {
byte[][] families = { Bytes.toBytes("familyA"), Bytes.toBytes("familyB") }; byte[][] families = { Bytes.toBytes("familyA"), Bytes.toBytes("familyB") };
TableName tableName = TableName.valueOf(name.getMethodName()); TableName tableName = TableName.valueOf(name.getMethodName());
UTIL.createTable(tableName, families, 1, Bytes.toBytes("aa"), Bytes.toBytes("cc"), 5); UTIL.createTable(tableName, families, 1, Bytes.toBytes("aa"), Bytes.toBytes("cc"), 5);
Configuration configuration = UTIL.getConfiguration(); Configuration configuration = UTIL.getConfiguration();
int numberOfRegions = MetaTableAccessor.getRegionCount(configuration, tableName); int numberOfRegions = UTIL.getMiniHBaseCluster().getRegions(tableName).size();
assertEquals(5, numberOfRegions); assertEquals(5, numberOfRegions);
HRegionPartitioner<Long, Long> partitioner = new HRegionPartitioner<>(); HRegionPartitioner<Long, Long> partitioner = new HRegionPartitioner<>();

View File

@ -37,7 +37,6 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
@ -413,11 +412,13 @@ public class RegionSplitter {
if (!conf.getBoolean("split.verify", true)) { if (!conf.getBoolean("split.verify", true)) {
// NOTE: createTable is synchronous on the table, but not on the regions // NOTE: createTable is synchronous on the table, but not on the regions
int onlineRegions = 0; int onlineRegions = 0;
while (onlineRegions < splitCount) { try (RegionLocator locator = connection.getRegionLocator(tableName)) {
onlineRegions = MetaTableAccessor.getRegionCount(connection, tableName); while (onlineRegions < splitCount) {
LOG.debug(onlineRegions + " of " + splitCount + " regions online..."); onlineRegions = locator.getAllRegionLocations().size();
if (onlineRegions < splitCount) { LOG.debug(onlineRegions + " of " + splitCount + " regions online...");
Thread.sleep(10 * 1000); // sleep if (onlineRegions < splitCount) {
Thread.sleep(10 * 1000); // sleep
}
} }
} }
} }

View File

@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
@ -246,7 +245,7 @@ public class TestTableFavoredNodes {
LOG.info("regionA: " + regionA.getEncodedName() + " with FN: " + fnm.getFavoredNodes(regionA)); LOG.info("regionA: " + regionA.getEncodedName() + " with FN: " + fnm.getFavoredNodes(regionA));
LOG.info("regionB: " + regionA.getEncodedName() + " with FN: " + fnm.getFavoredNodes(regionB)); LOG.info("regionB: " + regionA.getEncodedName() + " with FN: " + fnm.getFavoredNodes(regionB));
int countOfRegions = MetaTableAccessor.getRegionCount(TEST_UTIL.getConfiguration(), tableName); int countOfRegions = TEST_UTIL.getMiniHBaseCluster().getRegions(tableName).size();
admin.mergeRegionsAsync(regionA.getEncodedNameAsBytes(), admin.mergeRegionsAsync(regionA.getEncodedNameAsBytes(),
regionB.getEncodedNameAsBytes(), false).get(60, TimeUnit.SECONDS); regionB.getEncodedNameAsBytes(), false).get(60, TimeUnit.SECONDS);
@ -392,7 +391,9 @@ public class TestTableFavoredNodes {
TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() { TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
@Override @Override
public boolean evaluate() throws Exception { public boolean evaluate() throws Exception {
return MetaTableAccessor.getRegionCount(TEST_UTIL.getConfiguration(), tableName) == numRegions; try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
return locator.getAllRegionLocations().size() == numRegions;
}
} }
}); });
} }

View File

@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
@ -29,7 +30,6 @@ import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.RegionMetrics; import org.apache.hadoop.hbase.RegionMetrics;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.client.AsyncAdmin;
import org.apache.hadoop.hbase.client.NormalizeTableFilterParams; import org.apache.hadoop.hbase.client.NormalizeTableFilterParams;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
@ -155,11 +156,11 @@ public class TestSimpleRegionNormalizerOnCluster {
assertEquals( assertEquals(
tn1 + " should have split.", tn1 + " should have split.",
tn1RegionCount + 1, tn1RegionCount + 1,
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tn1)); getRegionCount(tn1));
assertEquals( assertEquals(
tn2 + " should not have split.", tn2 + " should not have split.",
tn2RegionCount, tn2RegionCount,
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tn2)); getRegionCount(tn2));
waitForTableRegionCount(tn3, tn3RegionCount); waitForTableRegionCount(tn3, tn3RegionCount);
} finally { } finally {
dropIfExists(tn1); dropIfExists(tn1);
@ -195,13 +196,13 @@ public class TestSimpleRegionNormalizerOnCluster {
assertEquals( assertEquals(
tableName + " should not have split.", tableName + " should not have split.",
currentRegionCount, currentRegionCount,
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tableName)); getRegionCount(tableName));
} else { } else {
waitForTableSplit(tableName, currentRegionCount + 1); waitForTableSplit(tableName, currentRegionCount + 1);
assertEquals( assertEquals(
tableName + " should have split.", tableName + " should have split.",
currentRegionCount + 1, currentRegionCount + 1,
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tableName)); getRegionCount(tableName));
} }
} finally { } finally {
dropIfExists(tableName); dropIfExists(tableName);
@ -219,7 +220,7 @@ public class TestSimpleRegionNormalizerOnCluster {
assertEquals( assertEquals(
tableName + " should have merged.", tableName + " should have merged.",
currentRegionCount - 1, currentRegionCount - 1,
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tableName)); getRegionCount(tableName));
} finally { } finally {
dropIfExists(tableName); dropIfExists(tableName);
} }
@ -248,7 +249,7 @@ public class TestSimpleRegionNormalizerOnCluster {
assertEquals( assertEquals(
tn1 + " should have split.", tn1 + " should have split.",
tn1RegionCount + 1, tn1RegionCount + 1,
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tn1)); getRegionCount(tn1));
waitForTableRegionCount(tn2, tn2RegionCount); waitForTableRegionCount(tn2, tn2RegionCount);
} finally { } finally {
dropIfExists(tn1); dropIfExists(tn1);
@ -277,7 +278,7 @@ public class TestSimpleRegionNormalizerOnCluster {
assertEquals( assertEquals(
tn1 + " should have split.", tn1 + " should have split.",
tn1RegionCount + 1, tn1RegionCount + 1,
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tn1)); getRegionCount(tn1));
waitForTableRegionCount(tn2, tn2RegionCount); waitForTableRegionCount(tn2, tn2RegionCount);
} finally { } finally {
dropIfExists(tn1); dropIfExists(tn1);
@ -306,7 +307,7 @@ public class TestSimpleRegionNormalizerOnCluster {
assertEquals( assertEquals(
tn1 + " should have split.", tn1 + " should have split.",
tn1RegionCount + 1, tn1RegionCount + 1,
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tn1)); getRegionCount(tn1));
waitForTableRegionCount(tn2, tn2RegionCount); waitForTableRegionCount(tn2, tn2RegionCount);
} finally { } finally {
dropIfExists(tn1); dropIfExists(tn1);
@ -344,10 +345,10 @@ public class TestSimpleRegionNormalizerOnCluster {
public String explainFailure() { public String explainFailure() {
return "expected " + targetRegionCount + " number of regions for table " + tableName; return "expected " + targetRegionCount + " number of regions for table " + tableName;
} }
@Override @Override
public boolean evaluate() throws IOException { public boolean evaluate() throws IOException {
final int currentRegionCount = final int currentRegionCount = getRegionCount(tableName);
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tableName);
return currentRegionCount == targetRegionCount; return currentRegionCount == targetRegionCount;
} }
}); });
@ -356,12 +357,14 @@ public class TestSimpleRegionNormalizerOnCluster {
private static void waitForTableSplit(final TableName tableName, final int targetRegionCount) private static void waitForTableSplit(final TableName tableName, final int targetRegionCount)
throws IOException { throws IOException {
TEST_UTIL.waitFor(TimeUnit.MINUTES.toMillis(5), new ExplainingPredicate<IOException>() { TEST_UTIL.waitFor(TimeUnit.MINUTES.toMillis(5), new ExplainingPredicate<IOException>() {
@Override public String explainFailure() { @Override
public String explainFailure() {
return "expected normalizer to split region."; return "expected normalizer to split region.";
} }
@Override public boolean evaluate() throws IOException {
final int currentRegionCount = @Override
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tableName); public boolean evaluate() throws IOException {
final int currentRegionCount = getRegionCount(tableName);
return currentRegionCount >= targetRegionCount; return currentRegionCount >= targetRegionCount;
} }
}); });
@ -370,12 +373,14 @@ public class TestSimpleRegionNormalizerOnCluster {
private static void waitForTableMerge(final TableName tableName, final int targetRegionCount) private static void waitForTableMerge(final TableName tableName, final int targetRegionCount)
throws IOException { throws IOException {
TEST_UTIL.waitFor(TimeUnit.MINUTES.toMillis(5), new ExplainingPredicate<IOException>() { TEST_UTIL.waitFor(TimeUnit.MINUTES.toMillis(5), new ExplainingPredicate<IOException>() {
@Override public String explainFailure() { @Override
public String explainFailure() {
return "expected normalizer to merge regions."; return "expected normalizer to merge regions.";
} }
@Override public boolean evaluate() throws IOException {
final int currentRegionCount = @Override
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tableName); public boolean evaluate() throws IOException {
final int currentRegionCount = getRegionCount(tableName);
return currentRegionCount <= targetRegionCount; return currentRegionCount <= targetRegionCount;
} }
}); });
@ -443,7 +448,7 @@ public class TestSimpleRegionNormalizerOnCluster {
final boolean normalizerEnabled, final boolean isMergeEnabled) final boolean normalizerEnabled, final boolean isMergeEnabled)
throws Exception { throws Exception {
final List<HRegion> generatedRegions = generateTestData(tableName, 1, 1, 2, 3, 5); final List<HRegion> generatedRegions = generateTestData(tableName, 1, 1, 2, 3, 5);
assertEquals(5, MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tableName)); assertEquals(5, getRegionCount(tableName));
admin.flush(tableName).get(); admin.flush(tableName).get();
final TableDescriptor td = TableDescriptorBuilder final TableDescriptor td = TableDescriptorBuilder
@ -481,7 +486,7 @@ public class TestSimpleRegionNormalizerOnCluster {
private static int createTableBegsMerge(final TableName tableName) throws Exception { private static int createTableBegsMerge(final TableName tableName) throws Exception {
// create 5 regions with sizes to trigger merge of small regions // create 5 regions with sizes to trigger merge of small regions
final List<HRegion> generatedRegions = generateTestData(tableName, 1, 1, 3, 3, 5); final List<HRegion> generatedRegions = generateTestData(tableName, 1, 1, 3, 3, 5);
assertEquals(5, MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tableName)); assertEquals(5, getRegionCount(tableName));
admin.flush(tableName).get(); admin.flush(tableName).get();
final TableDescriptor td = TableDescriptorBuilder final TableDescriptor td = TableDescriptorBuilder
@ -515,4 +520,10 @@ public class TestSimpleRegionNormalizerOnCluster {
admin.deleteTable(tableName).get(); admin.deleteTable(tableName).get();
} }
} }
private static int getRegionCount(TableName tableName) throws IOException {
try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
return locator.getAllRegionLocations().size();
}
}
} }