HBASE-24945 Remove MetaTableAccessor.getRegionCount (#2303)
Signed-off-by: Viraj Jasani <vjasani@apache.org>
This commit is contained in:
parent
6b0707f541
commit
227084c41f
|
@ -34,12 +34,10 @@ import java.util.Objects;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell.Type;
|
||||
import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
|
||||
import org.apache.hadoop.hbase.client.AsyncTable;
|
||||
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.Delete;
|
||||
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.RegionInfo;
|
||||
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.Result;
|
||||
import org.apache.hadoop.hbase.client.ResultScanner;
|
||||
|
@ -743,33 +740,6 @@ public final class MetaTableAccessor {
|
|||
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 //
|
||||
////////////////////////
|
||||
|
|
|
@ -18,14 +18,15 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.mapred;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
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.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.mapreduce.MutationSerialization;
|
||||
|
@ -283,8 +284,7 @@ public class TableMapReduceUtil {
|
|||
MutationSerialization.class.getName(), ResultSerialization.class.getName());
|
||||
if (partitioner == HRegionPartitioner.class) {
|
||||
job.setPartitionerClass(HRegionPartitioner.class);
|
||||
int regions =
|
||||
MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table));
|
||||
int regions = getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table));
|
||||
if (job.getNumReduceTasks() > regions) {
|
||||
job.setNumReduceTasks(regions);
|
||||
}
|
||||
|
@ -330,13 +330,12 @@ public class TableMapReduceUtil {
|
|||
* @throws IOException When retrieving the table details fails.
|
||||
*/
|
||||
// Used by tests.
|
||||
public static void limitNumReduceTasks(String table, JobConf job)
|
||||
throws IOException {
|
||||
int regions =
|
||||
MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table));
|
||||
if (job.getNumReduceTasks() > regions)
|
||||
public static void limitNumReduceTasks(String table, JobConf job) throws IOException {
|
||||
int regions = getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table));
|
||||
if (job.getNumReduceTasks() > regions) {
|
||||
job.setNumReduceTasks(regions);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensures that the given number of map tasks for the given job
|
||||
|
@ -347,13 +346,12 @@ public class TableMapReduceUtil {
|
|||
* @throws IOException When retrieving the table details fails.
|
||||
*/
|
||||
// Used by tests.
|
||||
public static void limitNumMapTasks(String table, JobConf job)
|
||||
throws IOException {
|
||||
int regions =
|
||||
MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table));
|
||||
if (job.getNumMapTasks() > regions)
|
||||
public static void limitNumMapTasks(String table, JobConf job) throws IOException {
|
||||
int regions = getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table));
|
||||
if (job.getNumMapTasks() > regions) {
|
||||
job.setNumMapTasks(regions);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the number of reduce tasks for the given job configuration to the
|
||||
|
@ -363,10 +361,8 @@ public class TableMapReduceUtil {
|
|||
* @param job The current job configuration to adjust.
|
||||
* @throws IOException When retrieving the table details fails.
|
||||
*/
|
||||
public static void setNumReduceTasks(String table, JobConf job)
|
||||
throws IOException {
|
||||
job.setNumReduceTasks(MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job),
|
||||
TableName.valueOf(table)));
|
||||
public static void setNumReduceTasks(String table, JobConf job) throws IOException {
|
||||
job.setNumReduceTasks(getRegionCount(HBaseConfiguration.create(job), TableName.valueOf(table)));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -377,10 +373,8 @@ public class TableMapReduceUtil {
|
|||
* @param job The current job configuration to adjust.
|
||||
* @throws IOException When retrieving the table details fails.
|
||||
*/
|
||||
public static void setNumMapTasks(String table, JobConf job)
|
||||
throws IOException {
|
||||
job.setNumMapTasks(MetaTableAccessor.getRegionCount(HBaseConfiguration.create(job),
|
||||
TableName.valueOf(table)));
|
||||
public static void setNumMapTasks(String table, JobConf job) throws IOException {
|
||||
job.setNumMapTasks(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.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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -39,7 +39,6 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
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.ConnectionFactory;
|
||||
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.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
|
@ -731,7 +731,7 @@ public class TableMapReduceUtil {
|
|||
job.setOutputValueClass(Writable.class);
|
||||
if (partitioner == 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) {
|
||||
job.setNumReduceTasks(regions);
|
||||
}
|
||||
|
@ -754,13 +754,12 @@ public class TableMapReduceUtil {
|
|||
* @param job The current job to adjust.
|
||||
* @throws IOException When retrieving the table details fails.
|
||||
*/
|
||||
public static void limitNumReduceTasks(String table, Job job)
|
||||
throws IOException {
|
||||
int regions =
|
||||
MetaTableAccessor.getRegionCount(job.getConfiguration(), TableName.valueOf(table));
|
||||
if (job.getNumReduceTasks() > regions)
|
||||
public static void limitNumReduceTasks(String table, Job job) throws IOException {
|
||||
int regions = getRegionCount(job.getConfiguration(), TableName.valueOf(table));
|
||||
if (job.getNumReduceTasks() > regions) {
|
||||
job.setNumReduceTasks(regions);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the number of reduce tasks for the given job configuration to the
|
||||
|
@ -770,10 +769,8 @@ public class TableMapReduceUtil {
|
|||
* @param job The current job to adjust.
|
||||
* @throws IOException When retrieving the table details fails.
|
||||
*/
|
||||
public static void setNumReduceTasks(String table, Job job)
|
||||
throws IOException {
|
||||
job.setNumReduceTasks(MetaTableAccessor.getRegionCount(job.getConfiguration(),
|
||||
TableName.valueOf(table)));
|
||||
public static void setNumReduceTasks(String table, Job job) throws IOException {
|
||||
job.setNumReduceTasks(getRegionCount(job.getConfiguration(), TableName.valueOf(table)));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1056,4 +1053,11 @@ public class TableMapReduceUtil {
|
|||
|
||||
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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,7 +22,6 @@ import static org.junit.Assert.assertEquals;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||
import org.apache.hadoop.hbase.testclassification.MapReduceTests;
|
||||
|
@ -81,14 +80,13 @@ public class TestHRegionPartitioner {
|
|||
|
||||
@Test
|
||||
public void testHRegionPartitionerMoreRegions() throws Exception {
|
||||
|
||||
byte[][] families = { Bytes.toBytes("familyA"), Bytes.toBytes("familyB") };
|
||||
|
||||
TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
UTIL.createTable(tableName, families, 1, Bytes.toBytes("aa"), Bytes.toBytes("cc"), 5);
|
||||
|
||||
Configuration configuration = UTIL.getConfiguration();
|
||||
int numberOfRegions = MetaTableAccessor.getRegionCount(configuration, tableName);
|
||||
int numberOfRegions = UTIL.getMiniHBaseCluster().getRegions(tableName).size();
|
||||
assertEquals(5, numberOfRegions);
|
||||
|
||||
HRegionPartitioner<Long, Long> partitioner = new HRegionPartitioner<>();
|
||||
|
|
|
@ -37,7 +37,6 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
|
@ -413,14 +412,16 @@ public class RegionSplitter {
|
|||
if (!conf.getBoolean("split.verify", true)) {
|
||||
// NOTE: createTable is synchronous on the table, but not on the regions
|
||||
int onlineRegions = 0;
|
||||
try (RegionLocator locator = connection.getRegionLocator(tableName)) {
|
||||
while (onlineRegions < splitCount) {
|
||||
onlineRegions = MetaTableAccessor.getRegionCount(connection, tableName);
|
||||
onlineRegions = locator.getAllRegionLocations().size();
|
||||
LOG.debug(onlineRegions + " of " + splitCount + " regions online...");
|
||||
if (onlineRegions < splitCount) {
|
||||
Thread.sleep(10 * 1000); // sleep
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
LOG.debug("Finished creating table with " + splitCount + " regions");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
|
|||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionLocation;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
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("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(),
|
||||
regionB.getEncodedNameAsBytes(), false).get(60, TimeUnit.SECONDS);
|
||||
|
||||
|
@ -392,7 +391,9 @@ public class TestTableFavoredNodes {
|
|||
TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
|
||||
@Override
|
||||
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;
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
|
|
@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
|
@ -29,7 +30,6 @@ import java.util.concurrent.TimeUnit;
|
|||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.MetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||
import org.apache.hadoop.hbase.RegionMetrics;
|
||||
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.Put;
|
||||
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.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
|
@ -155,11 +156,11 @@ public class TestSimpleRegionNormalizerOnCluster {
|
|||
assertEquals(
|
||||
tn1 + " should have split.",
|
||||
tn1RegionCount + 1,
|
||||
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tn1));
|
||||
getRegionCount(tn1));
|
||||
assertEquals(
|
||||
tn2 + " should not have split.",
|
||||
tn2RegionCount,
|
||||
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tn2));
|
||||
getRegionCount(tn2));
|
||||
waitForTableRegionCount(tn3, tn3RegionCount);
|
||||
} finally {
|
||||
dropIfExists(tn1);
|
||||
|
@ -195,13 +196,13 @@ public class TestSimpleRegionNormalizerOnCluster {
|
|||
assertEquals(
|
||||
tableName + " should not have split.",
|
||||
currentRegionCount,
|
||||
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tableName));
|
||||
getRegionCount(tableName));
|
||||
} else {
|
||||
waitForTableSplit(tableName, currentRegionCount + 1);
|
||||
assertEquals(
|
||||
tableName + " should have split.",
|
||||
currentRegionCount + 1,
|
||||
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tableName));
|
||||
getRegionCount(tableName));
|
||||
}
|
||||
} finally {
|
||||
dropIfExists(tableName);
|
||||
|
@ -219,7 +220,7 @@ public class TestSimpleRegionNormalizerOnCluster {
|
|||
assertEquals(
|
||||
tableName + " should have merged.",
|
||||
currentRegionCount - 1,
|
||||
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tableName));
|
||||
getRegionCount(tableName));
|
||||
} finally {
|
||||
dropIfExists(tableName);
|
||||
}
|
||||
|
@ -248,7 +249,7 @@ public class TestSimpleRegionNormalizerOnCluster {
|
|||
assertEquals(
|
||||
tn1 + " should have split.",
|
||||
tn1RegionCount + 1,
|
||||
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tn1));
|
||||
getRegionCount(tn1));
|
||||
waitForTableRegionCount(tn2, tn2RegionCount);
|
||||
} finally {
|
||||
dropIfExists(tn1);
|
||||
|
@ -277,7 +278,7 @@ public class TestSimpleRegionNormalizerOnCluster {
|
|||
assertEquals(
|
||||
tn1 + " should have split.",
|
||||
tn1RegionCount + 1,
|
||||
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tn1));
|
||||
getRegionCount(tn1));
|
||||
waitForTableRegionCount(tn2, tn2RegionCount);
|
||||
} finally {
|
||||
dropIfExists(tn1);
|
||||
|
@ -306,7 +307,7 @@ public class TestSimpleRegionNormalizerOnCluster {
|
|||
assertEquals(
|
||||
tn1 + " should have split.",
|
||||
tn1RegionCount + 1,
|
||||
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tn1));
|
||||
getRegionCount(tn1));
|
||||
waitForTableRegionCount(tn2, tn2RegionCount);
|
||||
} finally {
|
||||
dropIfExists(tn1);
|
||||
|
@ -344,10 +345,10 @@ public class TestSimpleRegionNormalizerOnCluster {
|
|||
public String explainFailure() {
|
||||
return "expected " + targetRegionCount + " number of regions for table " + tableName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean evaluate() throws IOException {
|
||||
final int currentRegionCount =
|
||||
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tableName);
|
||||
final int currentRegionCount = getRegionCount(tableName);
|
||||
return currentRegionCount == targetRegionCount;
|
||||
}
|
||||
});
|
||||
|
@ -356,12 +357,14 @@ public class TestSimpleRegionNormalizerOnCluster {
|
|||
private static void waitForTableSplit(final TableName tableName, final int targetRegionCount)
|
||||
throws 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.";
|
||||
}
|
||||
@Override public boolean evaluate() throws IOException {
|
||||
final int currentRegionCount =
|
||||
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tableName);
|
||||
|
||||
@Override
|
||||
public boolean evaluate() throws IOException {
|
||||
final int currentRegionCount = getRegionCount(tableName);
|
||||
return currentRegionCount >= targetRegionCount;
|
||||
}
|
||||
});
|
||||
|
@ -370,12 +373,14 @@ public class TestSimpleRegionNormalizerOnCluster {
|
|||
private static void waitForTableMerge(final TableName tableName, final int targetRegionCount)
|
||||
throws 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.";
|
||||
}
|
||||
@Override public boolean evaluate() throws IOException {
|
||||
final int currentRegionCount =
|
||||
MetaTableAccessor.getRegionCount(TEST_UTIL.getConnection(), tableName);
|
||||
|
||||
@Override
|
||||
public boolean evaluate() throws IOException {
|
||||
final int currentRegionCount = getRegionCount(tableName);
|
||||
return currentRegionCount <= targetRegionCount;
|
||||
}
|
||||
});
|
||||
|
@ -443,7 +448,7 @@ public class TestSimpleRegionNormalizerOnCluster {
|
|||
final boolean normalizerEnabled, final boolean isMergeEnabled)
|
||||
throws Exception {
|
||||
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();
|
||||
|
||||
final TableDescriptor td = TableDescriptorBuilder
|
||||
|
@ -481,7 +486,7 @@ public class TestSimpleRegionNormalizerOnCluster {
|
|||
private static int createTableBegsMerge(final TableName tableName) throws Exception {
|
||||
// create 5 regions with sizes to trigger merge of small regions
|
||||
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();
|
||||
|
||||
final TableDescriptor td = TableDescriptorBuilder
|
||||
|
@ -515,4 +520,10 @@ public class TestSimpleRegionNormalizerOnCluster {
|
|||
admin.deleteTable(tableName).get();
|
||||
}
|
||||
}
|
||||
|
||||
private static int getRegionCount(TableName tableName) throws IOException {
|
||||
try (RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
|
||||
return locator.getAllRegionLocations().size();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue