diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java index c0c1d7ef944..eb1ce387414 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java @@ -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 hbase:meta for passed table. - * @param c Configuration object - * @param tableName table name to count regions for - * @return Count or regions in table tableName - */ - 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 hbase:meta for passed table. - * @param connection Connection object - * @param tableName table name to count regions for - * @return Count or regions in table tableName - */ - public static int getRegionCount(final Connection connection, final TableName tableName) - throws IOException { - try (RegionLocator locator = connection.getRegionLocator(tableName)) { - List locations = locator.getAllRegionLocations(); - return locations == null ? 0 : locations.size(); - } - } - //////////////////////// // Editing operations // //////////////////////// diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java index 0907fd49cd2..99f6eb4b92e 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapred/TableMapReduceUtil.java @@ -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,12 +330,11 @@ 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); + } } /** @@ -347,12 +346,11 @@ 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); + } } /** @@ -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(); + } + } } diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java index 64c0ad19142..ef954d5b7a1 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java @@ -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,12 +754,11 @@ 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); + } } /** @@ -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(); + } + } } diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHRegionPartitioner.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHRegionPartitioner.java index 2095b870fef..77245f3c360 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHRegionPartitioner.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHRegionPartitioner.java @@ -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 partitioner = new HRegionPartitioner<>(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java index 08afc9e0269..35467bc504f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java @@ -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,11 +412,13 @@ public class RegionSplitter { if (!conf.getBoolean("split.verify", true)) { // NOTE: createTable is synchronous on the table, but not on the regions int onlineRegions = 0; - while (onlineRegions < splitCount) { - onlineRegions = MetaTableAccessor.getRegionCount(connection, tableName); - LOG.debug(onlineRegions + " of " + splitCount + " regions online..."); - if (onlineRegions < splitCount) { - Thread.sleep(10 * 1000); // sleep + try (RegionLocator locator = connection.getRegionLocator(tableName)) { + while (onlineRegions < splitCount) { + onlineRegions = locator.getAllRegionLocations().size(); + LOG.debug(onlineRegions + " of " + splitCount + " regions online..."); + if (onlineRegions < splitCount) { + Thread.sleep(10 * 1000); // sleep + } } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java index ea859b45777..280c860f748 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestTableFavoredNodes.java @@ -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() { @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; + } } }); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java index ee9a160182d..173adf49db2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizerOnCluster.java @@ -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() { - @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() { - @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 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 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(); + } + } }