From 107a8aae3ac72e323bda31c3d86528e46f29b126 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Mon, 15 Dec 2025 18:13:59 -0500 Subject: [PATCH 01/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: org.apache.hadoop.hbase --- .../hadoop/hbase/ClientMetaTableAccessor.java | 2 +- .../hadoop/hbase/CellComparatorImpl.java | 2 +- .../org/apache/hadoop/hbase/HConstants.java | 2 +- .../hbase/InnerStoreCellComparator.java | 2 +- .../apache/hadoop/hbase/MetaTableName.java | 72 +++++++++++++++++++ .../org/apache/hadoop/hbase/TableName.java | 27 +++++-- .../hadoop/hbase/TestCellComparator.java | 24 +++---- .../org/apache/hadoop/hbase/TestKeyValue.java | 24 +++---- .../hbase/TestClientClusterMetrics.java | 22 +++--- .../util/ProcessBasedLocalHBaseCluster.java | 2 +- .../hadoop/hbase/util/RestartMetaTest.java | 2 +- .../hbase/IntegrationTestMetaReplicas.java | 2 +- .../hbase/mttr/IntegrationTestMTTR.java | 2 +- .../hadoop/hbase/MetaTableAccessor.java | 8 +-- .../apache/hadoop/hbase/HBaseTestingUtil.java | 16 ++--- .../apache/hadoop/hbase/TestInfoServers.java | 2 +- .../org/apache/hadoop/hbase/HBaseCluster.java | 2 +- .../hadoop/hbase/HBaseTestingUtility.java | 20 +++--- 18 files changed, 161 insertions(+), 72 deletions(-) create mode 100644 hbase-common/src/main/java/org/apache/hadoop/hbase/MetaTableName.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClientMetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClientMetaTableAccessor.java index 42bfd757e0d1..bf8513ee959d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClientMetaTableAccessor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClientMetaTableAccessor.java @@ -196,7 +196,7 @@ private static CompletableFuture>> getTableReg final AsyncTable metaTable, final TableName tableName, final boolean excludeOfflinedSplitParents) { CompletableFuture>> future = new CompletableFuture<>(); - if (TableName.META_TABLE_NAME.equals(tableName)) { + if (MetaTableName.getInstance().equals(tableName)) { future.completeExceptionally(new IOException( "This method can't be used to locate meta regions;" + " use MetaTableLocator instead")); } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java index 0e6a53ca7c47..49eb3b9cce62 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java @@ -787,7 +787,7 @@ public static CellComparator getCellComparator(TableName tableName) { */ public static CellComparator getCellComparator(byte[] tableName) { // FYI, TableName.toBytes does not create an array; just returns existing array pointer. - return Bytes.equals(tableName, TableName.META_TABLE_NAME.toBytes()) + return Bytes.equals(tableName, MetaTableName.getInstance().toBytes()) ? MetaCellComparator.META_COMPARATOR : CellComparatorImpl.COMPARATOR; } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 9af711e7edfd..ded87f4a00b0 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1209,7 +1209,7 @@ public enum OperationStatusCode { @Deprecated public static final List HBASE_NON_USER_TABLE_DIRS = Collections.unmodifiableList(Arrays.asList( - (String[]) ArrayUtils.addAll(new String[] { TableName.META_TABLE_NAME.getNameAsString() }, + (String[]) ArrayUtils.addAll(new String[] { MetaTableName.getInstance().getNameAsString() }, HBASE_NON_TABLE_DIRS.toArray()))); /** Health script related settings. */ diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java index 7f6e87ebf911..e2fd632be19f 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java @@ -75,7 +75,7 @@ public static CellComparator getInnerStoreCellComparator(TableName tableName) { * @return CellComparator to use going off the {@code tableName} passed. */ public static CellComparator getInnerStoreCellComparator(byte[] tableName) { - return Bytes.equals(tableName, TableName.META_TABLE_NAME.toBytes()) + return Bytes.equals(tableName, MetaTableName.getInstance().toBytes()) ? MetaCellComparator.META_COMPARATOR : InnerStoreCellComparator.INNER_STORE_COMPARATOR; } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaTableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaTableName.java new file mode 100644 index 000000000000..5859e0f5d305 --- /dev/null +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaTableName.java @@ -0,0 +1,72 @@ +package org.apache.hadoop.hbase; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hbase.thirdparty.com.google.common.base.Strings; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Singleton class for managing the META_TABLE_NAME instance. + * This allows the meta table name to be overridden for testing using reflection. + */ +public class MetaTableName { + private static final Logger LOG = LoggerFactory.getLogger(MetaTableName.class); + + /** + * The singleton instance of the meta table name. + * This field can be overridden for testing using reflection. + */ + private static volatile TableName instance; + + private MetaTableName() { + // Private constructor to prevent instantiation + } + + /** + * Get the singleton instance of the meta table name. + * Initializes lazily using the default configuration if not already set. + * + * @return The meta table name instance + */ + public static TableName getInstance() { + if (instance == null) { + synchronized (MetaTableName.class) { + if (instance == null) { + instance = initializeHbaseMetaTableName(HBaseConfiguration.create()); + LOG.info("Meta table name initialized: {}", instance); + } + } + } + return instance; + } + + /** + * Initialize the meta table name from the given configuration. + * + * @param conf The configuration to use + * @return The initialized meta table name + */ + private static TableName initializeHbaseMetaTableName(Configuration conf) { + String suffix_val = conf.get(HConstants.HBASE_META_TABLE_SUFFIX, + HConstants.HBASE_META_TABLE_SUFFIX_DEFAULT_VALUE); + LOG.info("Meta table suffix value: {}", suffix_val); + if (Strings.isNullOrEmpty(suffix_val)) { + return TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); + } else { + return TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta_" + suffix_val); + } + } + + /** + * Get the instance field for reflection-based testing. + * This method is package-private to allow test classes to access the field. + * + * @return The Field object for the instance field + */ + static java.lang.reflect.Field getInstanceField() throws NoSuchFieldException { + java.lang.reflect.Field field = MetaTableName.class.getDeclaredField("instance"); + field.setAccessible(true); + return field; + } +} + diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java index b6d854c13784..6fdfc1edf001 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java @@ -23,10 +23,15 @@ import java.util.Set; import java.util.concurrent.CopyOnWriteArraySet; import org.apache.commons.lang3.ArrayUtils; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; - +import org.apache.yetus.audience.InterfaceStability; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import com.google.errorprone.annotations.RestrictedApi; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; +import org.apache.hbase.thirdparty.com.google.common.base.Strings; /** * Immutable POJO class for representing a table name. Which is of the form: <table @@ -43,8 +48,11 @@ *

*/ @InterfaceAudience.Public +@InterfaceStability.Stable public final class TableName implements Comparable { + private static final Logger LOG = LoggerFactory.getLogger(TableName.class); + /** See {@link #createTableNameIfNecessary(ByteBuffer, ByteBuffer)} */ private static final Set tableCache = new CopyOnWriteArraySet<>(); @@ -65,9 +73,18 @@ public final class TableName implements Comparable { public static final String VALID_USER_TABLE_REGEX = "(?:(?:(?:" + VALID_NAMESPACE_REGEX + "\\" + NAMESPACE_DELIM + ")?)" + "(?:" + VALID_TABLE_QUALIFIER_REGEX + "))"; - /** The hbase:meta table's name. */ - public static final TableName META_TABLE_NAME = - valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); + /** + * The name of hbase meta table could either be hbase:meta_xxx or 'hbase:meta' otherwise. Config + * hbase.meta.table.suffix will govern the decision of adding suffix to the habase:meta + * + * This field is initialized from the MetaTableName singleton and can be overridden for testing + * by modifying the singleton instance via reflection. + * + * @deprecated Use {@link MetaTableName#getInstance()} instead. This field will be removed in a + * future version. + */ + @Deprecated + public static TableName META_TABLE_NAME; /** * The Namespace table's name. @@ -87,7 +104,7 @@ public final class TableName implements Comparable { /** Returns True if tn is the hbase:meta table name. */ public static boolean isMetaTableName(final TableName tn) { - return tn.equals(TableName.META_TABLE_NAME); + return tn.equals(MetaTableName.getInstance()); } /** diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java index 553b39311369..68666e3552f0 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java @@ -197,38 +197,38 @@ public void testMetaComparisons2() { CellComparator c = MetaCellComparator.META_COMPARATOR; assertTrue(c.compare( createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,1"), now)), + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now)), createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,1"), now))) + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now))) == 0); Cell a = createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,1"), now)); + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now)); Cell b = createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,2"), now)); + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,2"), now)); assertTrue(c.compare(a, b) < 0); assertTrue(c.compare( createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,2"), now)), + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,2"), now)), createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,1"), now))) + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now))) > 0); assertTrue(c.compare( createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,1"), now)), + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now)), createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,1"), now))) + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now))) == 0); assertTrue(c.compare( createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,1"), now)), + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now)), createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,2"), now))) + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,2"), now))) < 0); assertTrue(c.compare( createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,2"), now)), + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,2"), now)), createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,1"), now))) + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now))) > 0); } diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java index 1644a6f1fce7..c06d5569e00d 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java @@ -198,31 +198,31 @@ public void testKeyValueBorderCases() { private void metacomparisons(final CellComparatorImpl c) { long now = EnvironmentEdgeManager.currentTime(); assertTrue(c.compare( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,1"), now), - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,1"), now)) + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now), + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now)) == 0); KeyValue a = - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,1"), now); + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now); KeyValue b = - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,2"), now); + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,2"), now); assertTrue(c.compare(a, b) < 0); assertTrue(c.compare( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,2"), now), - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",a,,0,1"), now)) + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,2"), now), + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now)) > 0); } private void comparisons(final CellComparatorImpl c) { long now = EnvironmentEdgeManager.currentTime(); assertTrue(c.compare( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,1"), now), - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,1"), now)) == 0); + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now), + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now)) == 0); assertTrue(c.compare( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,1"), now), - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,2"), now)) < 0); + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now), + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,2"), now)) < 0); assertTrue(c.compare( - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,2"), now), - new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + ",,1"), now)) > 0); + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,2"), now), + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now)) > 0); } @Test diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java index a459074ba27d..54ceeecfec21 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java @@ -222,16 +222,16 @@ public void testRegionStatesCount() throws Exception { ClusterMetrics metrics = ADMIN.getClusterMetrics(); Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME).getRegionsInTransition(), + metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getRegionsInTransition(), 0); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME).getOpenRegions(), 1); + metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getOpenRegions(), 1); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME).getTotalRegions(), 1); + metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getTotalRegions(), 1); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME).getClosedRegions(), 0); + metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getClosedRegions(), 0); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME).getSplitRegions(), 0); + metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getSplitRegions(), 0); Assert.assertEquals( metrics.getTableRegionStatesCount().get(TABLE_NAME).getRegionsInTransition(), 0); Assert.assertEquals(metrics.getTableRegionStatesCount().get(TABLE_NAME).getOpenRegions(), 1); @@ -253,12 +253,12 @@ public void testRegionStatesWithSplit() throws Exception { ClusterMetrics metrics = ADMIN.getClusterMetrics(); Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME).getRegionsInTransition(), + metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getRegionsInTransition(), 0); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME).getOpenRegions(), 1); + metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getOpenRegions(), 1); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME).getTotalRegions(), 1); + metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getTotalRegions(), 1); Assert.assertEquals( metrics.getTableRegionStatesCount().get(TABLE_NAME).getRegionsInTransition(), 0); Assert.assertEquals(metrics.getTableRegionStatesCount().get(TABLE_NAME).getOpenRegions(), 1); @@ -273,12 +273,12 @@ public void testRegionStatesWithSplit() throws Exception { metrics = ADMIN.getClusterMetrics(); Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME).getRegionsInTransition(), + metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getRegionsInTransition(), 0); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME).getOpenRegions(), 1); + metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getOpenRegions(), 1); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME).getTotalRegions(), 1); + metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getTotalRegions(), 1); Assert.assertEquals( metrics.getTableRegionStatesCount().get(TABLE_NAME).getRegionsInTransition(), 0); Assert.assertEquals(metrics.getTableRegionStatesCount().get(TABLE_NAME).getOpenRegions(), 2); diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java index 5e8447c2ad81..3203d98df2f4 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java @@ -165,7 +165,7 @@ public void startHBase() throws IOException { int attemptsLeft = 10; while (attemptsLeft-- > 0) { try { - testUtil.getConnection().getTable(TableName.META_TABLE_NAME); + testUtil.getConnection().getTable(MetaTableName.getInstance()); } catch (Exception e) { LOG.info("Waiting for HBase to startup. Retries left: " + attemptsLeft, e); Threads.sleep(1000); diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java index 144ea6503b06..8274993aa3ac 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java @@ -124,7 +124,7 @@ protected int doWork() throws Exception { LOG.debug("Trying to scan meta"); - Table metaTable = connection.getTable(TableName.META_TABLE_NAME); + Table metaTable = connection.getTable(MetaTableName.getInstance()); ResultScanner scanner = metaTable.getScanner(new Scan()); Result result; while ((result = scanner.next()) != null) { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java index a8c3a16d13dc..fd07d7e1dc6a 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java @@ -56,7 +56,7 @@ public static void setUp() throws Exception { 1000); // Make sure there are three servers. util.initializeCluster(3); - HBaseTestingUtil.setReplicas(util.getAdmin(), TableName.META_TABLE_NAME, 3); + HBaseTestingUtil.setReplicas(util.getAdmin(), MetaTableName.getInstance(), 3); ZKWatcher zkw = util.getZooKeeperWatcher(); Configuration conf = util.getConfiguration(); String baseZNode = diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java index 2bb87ca8f2f6..fd1f6dd5ac32 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java @@ -184,7 +184,7 @@ private static void setupActions() throws IOException { // Set up the action that will move the regions of meta. moveMetaRegionsAction = new MoveRegionsOfTableAction(sleepTime, - MonkeyConstants.DEFAULT_MOVE_REGIONS_MAX_TIME, TableName.META_TABLE_NAME); + MonkeyConstants.DEFAULT_MOVE_REGIONS_MAX_TIME, MetaTableName.getInstance()); // Set up the action that will move the regions of our table. moveRegionAction = new MoveRegionsOfTableAction(sleepTime, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java index 05b049e27dbc..24bbfb5079f7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java @@ -151,7 +151,7 @@ public static Table getMetaHTable(final Connection connection) throws IOExceptio if (connection.isClosed()) { throw new IOException("connection is closed"); } - return connection.getTable(TableName.META_TABLE_NAME); + return connection.getTable(MetaTableName.getInstance()); } /** @@ -366,7 +366,7 @@ private static Scan getMetaScan(Configuration conf, int rowUpperLimit) { public static List> getTableRegionsAndLocations( Connection connection, @Nullable final TableName tableName, final boolean excludeOfflinedSplitParents) throws IOException { - if (tableName != null && tableName.equals(TableName.META_TABLE_NAME)) { + if (tableName != null && tableName.equals(MetaTableName.getInstance())) { throw new IOException( "This method can't be used to locate meta regions; use MetaTableLocator instead"); } @@ -592,7 +592,7 @@ public static PairOfSameType getDaughterRegions(Result data) { */ @Nullable public static TableState getTableState(Connection conn, TableName tableName) throws IOException { - if (tableName.equals(TableName.META_TABLE_NAME)) { + if (tableName.equals(MetaTableName.getInstance())) { return new TableState(tableName, TableState.State.ENABLED); } Table metaHTable = getMetaHTable(conn); @@ -859,7 +859,7 @@ public static void addRegionsToMeta(Connection connection, List regi private static void updateTableState(Connection connection, TableState state) throws IOException { Put put = makePutFromTableState(state, EnvironmentEdgeManager.currentTime()); putToMetaTable(connection, put); - LOG.info("Updated {} in hbase:meta", state); + LOG.info("Updated {} in {}", state, MetaTableName.getInstance()); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java index 8ae9206694c6..78b03ee66826 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java @@ -902,7 +902,7 @@ public SingleProcessHBaseCluster startMiniHBaseCluster(StartTestingClusterOption // Populate the master address configuration from mini cluster configuration. conf.set(HConstants.MASTER_ADDRS_KEY, MasterRegistry.getMasterAddr(c)); // Don't leave here till we've done a successful scan of the hbase:meta - try (Table t = getConnection().getTable(TableName.META_TABLE_NAME); + try (Table t = getConnection().getTable(MetaTableName.getInstance()); ResultScanner s = t.getScanner(new Scan())) { for (;;) { if (s.next() == null) { @@ -1025,7 +1025,7 @@ public void restartHBaseCluster(StartTestingClusterOption option) option.getMasterClass(), option.getRsClass()); // Don't leave here till we've done a successful scan of the hbase:meta Connection conn = ConnectionFactory.createConnection(this.conf); - Table t = conn.getTable(TableName.META_TABLE_NAME); + Table t = conn.getTable(MetaTableName.getInstance()); ResultScanner s = t.getScanner(new Scan()); while (s.next() != null) { // do nothing @@ -2169,7 +2169,7 @@ public String checksumRows(final Table table) throws Exception { */ public List createMultiRegionsInMeta(final Configuration conf, final TableDescriptor htd, byte[][] startKeys) throws IOException { - try (Table meta = getConnection().getTable(TableName.META_TABLE_NAME)) { + try (Table meta = getConnection().getTable(MetaTableName.getInstance())) { Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR); List newRegions = new ArrayList<>(startKeys.length); MetaTableAccessor.updateTableState(getConnection(), htd.getTableName(), @@ -2522,7 +2522,7 @@ public void process(WatchedEvent watchedEvent) { monitor.close(); if (checkStatus) { - getConnection().getTable(TableName.META_TABLE_NAME).close(); + getConnection().getTable(MetaTableName.getInstance()).close(); } } @@ -3047,7 +3047,7 @@ public void waitUntilAllRegionsAssigned(final TableName tableName) throws IOExce * Waith until all system table's regions get assigned */ public void waitUntilAllSystemRegionsAssigned() throws IOException { - waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME); + waitUntilAllRegionsAssigned(MetaTableName.getInstance()); } /** @@ -3060,7 +3060,7 @@ public void waitUntilAllSystemRegionsAssigned() throws IOException { public void waitUntilAllRegionsAssigned(final TableName tableName, final long timeout) throws IOException { if (!TableName.isMetaTableName(tableName)) { - try (final Table meta = getConnection().getTable(TableName.META_TABLE_NAME)) { + try (final Table meta = getConnection().getTable(MetaTableName.getInstance())) { LOG.debug("Waiting until all regions of table " + tableName + " get assigned. Timeout = " + timeout + "ms"); waitFor(timeout, 200, true, new ExplainingPredicate() { @@ -3280,7 +3280,7 @@ public Table createRandomTable(TableName tableName, final Collection fam Bytes.toBytes(String.format(keyFormat, splitEndKey)), numRegions); if (hbaseCluster != null) { - getMiniHBaseCluster().flushcache(TableName.META_TABLE_NAME); + getMiniHBaseCluster().flushcache(MetaTableName.getInstance()); } BufferedMutator mutator = getConnection().getBufferedMutator(tableName); @@ -3361,7 +3361,7 @@ public static void waitForHostPort(String host, int port) throws IOException { } public static int getMetaRSPort(Connection connection) throws IOException { - try (RegionLocator locator = connection.getRegionLocator(TableName.META_TABLE_NAME)) { + try (RegionLocator locator = connection.getRegionLocator(MetaTableName.getInstance())) { return locator.getRegionLocation(Bytes.toBytes("")).getPort(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java index a894bbcd0aeb..cf0ae4f7eb80 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java @@ -91,7 +91,7 @@ public void testGetMasterInfoPort() throws Exception { @Test public void testInfoServersRedirect() throws Exception { // give the cluster time to start up - UTIL.getConnection().getTable(TableName.META_TABLE_NAME).close(); + UTIL.getConnection().getTable(MetaTableName.getInstance()).close(); int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort(); assertContainsContent(new URL("http://localhost:" + port + "/index.html"), "master.jsp"); assertContainsContent(new URL("http://localhost:" + port + "/master-status"), "master.jsp"); diff --git a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCluster.java b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCluster.java index 27f3dd4f43ab..eb40f4eee439 100644 --- a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCluster.java +++ b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCluster.java @@ -343,7 +343,7 @@ public boolean restoreClusterMetrics(ClusterMetrics desiredStatus) throws IOExce * Get the ServerName of region server serving the first hbase:meta region */ public ServerName getServerHoldingMeta() throws IOException { - return getServerHoldingRegion(TableName.META_TABLE_NAME, + return getServerHoldingRegion(MetaTableName.getInstance(), RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()); } diff --git a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 698e433bb744..21818f77a1c9 100644 --- a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -1107,7 +1107,7 @@ public MiniHBaseCluster startMiniHBaseCluster(StartMiniClusterOption option) // Populate the master address configuration from mini cluster configuration. conf.set(HConstants.MASTER_ADDRS_KEY, MasterRegistry.getMasterAddr(c)); // Don't leave here till we've done a successful scan of the hbase:meta - try (Table t = getConnection().getTable(TableName.META_TABLE_NAME); + try (Table t = getConnection().getTable(MetaTableName.getInstance()); ResultScanner s = t.getScanner(new Scan())) { for (;;) { if (s.next() == null) { @@ -1229,7 +1229,7 @@ public void restartHBaseCluster(StartMiniClusterOption option) option.getMasterClass(), option.getRsClass()); // Don't leave here till we've done a successful scan of the hbase:meta Connection conn = ConnectionFactory.createConnection(this.conf); - Table t = conn.getTable(TableName.META_TABLE_NAME); + Table t = conn.getTable(MetaTableName.getInstance()); ResultScanner s = t.getScanner(new Scan()); while (s.next() != null) { // do nothing @@ -2397,7 +2397,7 @@ public String checksumRows(final Table table) throws Exception { */ public List createMultiRegionsInMeta(final Configuration conf, final TableDescriptor htd, byte[][] startKeys) throws IOException { - Table meta = getConnection().getTable(TableName.META_TABLE_NAME); + Table meta = getConnection().getTable(MetaTableName.getInstance()); Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR); List newRegions = new ArrayList<>(startKeys.length); MetaTableAccessor.updateTableState(getConnection(), htd.getTableName(), @@ -2479,7 +2479,7 @@ public static HRegion createRegionAndWAL(final RegionInfo info, final Path rootD */ public List getMetaTableRows() throws IOException { // TODO: Redo using MetaTableAccessor class - Table t = getConnection().getTable(TableName.META_TABLE_NAME); + Table t = getConnection().getTable(MetaTableName.getInstance()); List rows = new ArrayList<>(); ResultScanner s = t.getScanner(new Scan()); for (Result result : s) { @@ -2497,7 +2497,7 @@ public List getMetaTableRows() throws IOException { */ public List getMetaTableRows(TableName tableName) throws IOException { // TODO: Redo using MetaTableAccessor. - Table t = getConnection().getTable(TableName.META_TABLE_NAME); + Table t = getConnection().getTable(MetaTableName.getInstance()); List rows = new ArrayList<>(); ResultScanner s = t.getScanner(new Scan()); for (Result result : s) { @@ -2827,7 +2827,7 @@ public void process(WatchedEvent watchedEvent) { monitor.close(); if (checkStatus) { - getConnection().getTable(TableName.META_TABLE_NAME).close(); + getConnection().getTable(MetaTableName.getInstance()).close(); } } @@ -3352,7 +3352,7 @@ public void waitUntilAllRegionsAssigned(final TableName tableName) throws IOExce * Waith until all system table's regions get assigned */ public void waitUntilAllSystemRegionsAssigned() throws IOException { - waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME); + waitUntilAllRegionsAssigned(MetaTableName.getInstance()); } /** @@ -3365,7 +3365,7 @@ public void waitUntilAllSystemRegionsAssigned() throws IOException { public void waitUntilAllRegionsAssigned(final TableName tableName, final long timeout) throws IOException { if (!TableName.isMetaTableName(tableName)) { - try (final Table meta = getConnection().getTable(TableName.META_TABLE_NAME)) { + try (final Table meta = getConnection().getTable(MetaTableName.getInstance())) { LOG.debug("Waiting until all regions of table " + tableName + " get assigned. Timeout = " + timeout + "ms"); waitFor(timeout, 200, true, new ExplainingPredicate() { @@ -3583,7 +3583,7 @@ public Table createRandomTable(TableName tableName, final Collection fam Bytes.toBytes(String.format(keyFormat, splitEndKey)), numRegions); if (hbaseCluster != null) { - getMiniHBaseCluster().flushcache(TableName.META_TABLE_NAME); + getMiniHBaseCluster().flushcache(MetaTableName.getInstance()); } BufferedMutator mutator = getConnection().getBufferedMutator(tableName); @@ -3798,7 +3798,7 @@ public static int createPreSplitLoadTestTable(Configuration conf, TableDescripto } public static int getMetaRSPort(Connection connection) throws IOException { - try (RegionLocator locator = connection.getRegionLocator(TableName.META_TABLE_NAME)) { + try (RegionLocator locator = connection.getRegionLocator(MetaTableName.getInstance())) { return locator.getRegionLocation(Bytes.toBytes("")).getPort(); } } From 7a39e71aa3fe3ec2ed3e9352b757d0612e7a85ee Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Mon, 15 Dec 2025 18:43:59 -0500 Subject: [PATCH 02/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: org.apache.hadoop.hbase.client --- .../client/AsyncNonMetaRegionLocator.java | 8 +++---- .../hbase/client/AsyncRegionLocator.java | 4 ++-- .../client/AsyncTableRegionLocatorImpl.java | 3 ++- .../hbase/client/MutableRegionInfo.java | 5 +++-- .../hbase/client/RawAsyncHBaseAdmin.java | 8 +++---- .../hadoop/hbase/client/RegionInfo.java | 3 ++- .../hbase/client/RegionInfoBuilder.java | 3 ++- .../hbase/client/RegionInfoDisplay.java | 3 ++- .../hbase/client/TableDescriptorBuilder.java | 3 ++- .../client/TestAsyncAdminRpcPriority.java | 3 ++- .../client/TestAsyncRegionLocatorTracing.java | 15 ++++++------- .../client/TestAsyncTableRpcPriority.java | 21 ++++++++++--------- .../hbase/client/TestRegionInfoBuilder.java | 3 ++- .../client/TestTableDescriptorBuilder.java | 5 +++-- .../apache/hadoop/hbase/MetaTableName.java | 2 ++ .../util/ProcessBasedLocalHBaseCluster.java | 1 + .../hadoop/hbase/util/RestartMetaTest.java | 1 + .../hbase/mttr/IntegrationTestMTTR.java | 1 + .../client/AbstractTestRegionLocator.java | 19 +++++++++-------- .../client/MetaWithReplicasTestBase.java | 3 ++- .../hbase/client/RegionReplicaTestHelper.java | 3 ++- .../apache/hadoop/hbase/client/TestAdmin.java | 3 ++- .../hadoop/hbase/client/TestAdmin2.java | 9 ++++---- .../TestAsyncAdminWithRegionReplicas.java | 9 ++++---- .../client/TestAsyncMetaRegionLocator.java | 5 +++-- .../client/TestAsyncNonMetaRegionLocator.java | 5 +++-- .../client/TestAsyncRegionAdminApi2.java | 16 +++++++------- .../hbase/client/TestAsyncTableAdminApi.java | 8 +++---- .../hbase/client/TestAsyncTableAdminApi2.java | 3 ++- .../hbase/client/TestAsyncTableAdminApi3.java | 20 +++++++++--------- .../client/TestAsyncTableUseMetaReplicas.java | 5 +++-- ...talogReplicaLoadBalanceSimpleSelector.java | 18 ++++++++-------- .../hbase/client/TestCleanupMetaReplica.java | 3 ++- .../client/TestClientScannerTimeouts.java | 3 ++- .../client/TestClientSideRegionScanner.java | 9 ++++---- .../hadoop/hbase/client/TestEnableTable.java | 3 ++- .../hbase/client/TestFromClientSide5.java | 3 ++- .../TestIncreaseMetaReplicaThroughConfig.java | 3 ++- .../hbase/client/TestMasterRegistry.java | 3 ++- .../hadoop/hbase/client/TestMetaCache.java | 3 ++- .../client/TestMetaRegionLocationCache.java | 3 ++- .../client/TestMetaWithReplicasBasic.java | 3 ++- .../TestMetaWithReplicasShutdownHandling.java | 5 +++-- .../TestMultiActionMetricsFromClient.java | 3 ++- .../hbase/client/TestReplicaWithCluster.java | 3 ++- .../client/TestRpcConnectionRegistry.java | 3 ++- .../client/TestSeparateClientZKCluster.java | 9 ++++---- .../TestShutdownOfMetaReplicaHolder.java | 3 ++- .../hbase/client/TestSnapshotFromClient.java | 3 ++- .../client/TestZKConnectionRegistry.java | 5 +++-- 50 files changed, 166 insertions(+), 122 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java index e26fb837b89d..3afe3ae63bed 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java @@ -22,7 +22,6 @@ import static org.apache.hadoop.hbase.HConstants.NINES; import static org.apache.hadoop.hbase.HConstants.USE_META_REPLICAS; import static org.apache.hadoop.hbase.HConstants.ZEROES; -import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.createRegionLocations; import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.isGood; import static org.apache.hadoop.hbase.client.ConnectionConfiguration.HBASE_CLIENT_META_CACHE_INVALIDATE_INTERVAL; @@ -54,6 +53,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Scan.ReadType; @@ -238,14 +238,14 @@ private boolean tryComplete(LocateRequest req, CompletableFuture { + .createSelector(replicaSelectorClass, MetaTableName.getInstance(), conn, () -> { int numOfReplicas = CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS; try { RegionLocations metaLocations = conn.registry.getMetaRegionLocations() .get(conn.connConf.getMetaReadRpcTimeoutNs(), TimeUnit.NANOSECONDS); numOfReplicas = metaLocations.size(); } catch (Exception e) { - LOG.error("Failed to get table {}'s region replication, ", META_TABLE_NAME, e); + LOG.error("Failed to get table {}'s region replication, ", MetaTableName.getInstance(), e); } return numOfReplicas; }); @@ -427,7 +427,7 @@ private void locateInMeta(TableName tableName, LocateRequest req) { // do nothing } - conn.getTable(META_TABLE_NAME).scan(scan, new AdvancedScanResultConsumer() { + conn.getTable(MetaTableName.getInstance()).scan(scan, new AdvancedScanResultConsumer() { private boolean completeNormally = false; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java index 0e872a5b21da..f77a8fcdd492 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.REGION_NAMES_KEY; import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.SERVER_NAME_KEY; import static org.apache.hadoop.hbase.util.FutureUtils.addListener; @@ -38,6 +37,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.trace.ConnectionSpanBuilder; import org.apache.hadoop.hbase.client.trace.TableSpanBuilder; @@ -217,7 +217,7 @@ void clearCache(TableName tableName) { new TableSpanBuilder(conn).setName("AsyncRegionLocator.clearCache").setTableName(tableName); TraceUtil.trace(() -> { LOG.debug("Clear meta cache for {}", tableName); - if (tableName.equals(META_TABLE_NAME)) { + if (tableName.equals(MetaTableName.getInstance())) { metaRegionLocator.clearCache(); } else { nonMetaRegionLocator.clearCache(tableName); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java index b7ec7fcd8725..32349a64651d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java @@ -25,6 +25,7 @@ import java.util.concurrent.CompletableFuture; import org.apache.hadoop.hbase.ClientMetaTableAccessor; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.yetus.audience.InterfaceAudience; @@ -63,7 +64,7 @@ public CompletableFuture> getAllRegionLocations() { .thenApply(locs -> Arrays.asList(locs.getRegionLocations())); } CompletableFuture> future = ClientMetaTableAccessor - .getTableHRegionLocations(conn.getTable(TableName.META_TABLE_NAME), tableName); + .getTableHRegionLocations(conn.getTable(MetaTableName.getInstance()), tableName); addListener(future, (locs, error) -> locs.forEach(loc -> { // the cache assumes that all locations have a serverName. only add if that's true if (loc.getServerName() != null) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java index d6d8e00f7822..5e629839782f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java @@ -21,6 +21,7 @@ import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; @@ -120,7 +121,7 @@ private static int checkReplicaId(int regionId) { this.replicaId = checkReplicaId(replicaId); this.offLine = offLine; this.regionName = RegionInfo.createRegionName(this.tableName, this.startKey, this.regionId, - this.replicaId, !this.tableName.equals(TableName.META_TABLE_NAME)); + this.replicaId, !this.tableName.equals(MetaTableName.getInstance())); this.encodedName = RegionInfo.encodeRegionName(this.regionName); this.hashCode = generateHashCode(this.tableName, this.startKey, this.endKey, this.regionId, this.replicaId, this.offLine, this.regionName); @@ -232,7 +233,7 @@ public boolean containsRow(byte[] row) { /** Returns true if this region is a meta region */ @Override public boolean isMetaRegion() { - return tableName.equals(TableName.META_TABLE_NAME); + return tableName.equals(MetaTableName.getInstance()); } /** Returns True if has been split and has daughters. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index f4a474957a2f..c649efe4ea54 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.HConstants.HIGH_QOS; -import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; import static org.apache.hadoop.hbase.util.FutureUtils.addListener; import static org.apache.hadoop.hbase.util.FutureUtils.unwrapCompletionException; @@ -63,6 +62,7 @@ import org.apache.hadoop.hbase.RegionMetricsBuilder; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableExistsException; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotEnabledException; @@ -405,7 +405,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin { AsyncAdminBuilderBase builder) { this.connection = connection; this.retryTimer = retryTimer; - this.metaTable = connection.getTable(META_TABLE_NAME); + this.metaTable = connection.getTable(MetaTableName.getInstance()); this.rpcTimeoutNs = builder.rpcTimeoutNs; this.operationTimeoutNs = builder.operationTimeoutNs; this.pauseNs = builder.pauseNs; @@ -1012,7 +1012,7 @@ List> adminCall(controller, stub, @Override public CompletableFuture> getRegions(TableName tableName) { - if (tableName.equals(META_TABLE_NAME)) { + if (tableName.equals(MetaTableName.getInstance())) { return connection.registry.getMetaRegionLocations() .thenApply(locs -> Stream.of(locs.getRegionLocations()).map(HRegionLocation::getRegion) .collect(Collectors.toList())); @@ -1303,7 +1303,7 @@ private CompletableFuture compactRegion(byte[] regionName, byte[] columnFa * List all region locations for the specific table. */ private CompletableFuture> getTableHRegionLocations(TableName tableName) { - if (TableName.META_TABLE_NAME.equals(tableName)) { + if (MetaTableName.getInstance().equals(tableName)) { CompletableFuture> future = new CompletableFuture<>(); addListener(connection.registry.getMetaRegionLocations(), (metaRegions, err) -> { if (err != null) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java index 10c554e26f79..bc3b48a54a28 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.stream.Collectors; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.util.ByteArrayHashKey; @@ -431,7 +432,7 @@ static byte[] toByteArray(RegionInfo ri) { */ static String prettyPrint(final String encodedRegionName) { if (encodedRegionName.equals("1028785192")) { - return encodedRegionName + "/hbase:meta"; + return encodedRegionName + "/" + MetaTableName.getInstance(); } return encodedRegionName; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java index ef927fd3a55b..1c2aab455b55 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.client; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.yetus.audience.InterfaceAudience; @@ -43,7 +44,7 @@ public class RegionInfoBuilder { // TODO: How come Meta regions still do not have encoded region names? Fix. // hbase:meta,,1.1588230740 should be the hbase:meta first region name. public static final RegionInfo FIRST_META_REGIONINFO = - new MutableRegionInfo(1L, TableName.META_TABLE_NAME, RegionInfo.DEFAULT_REPLICA_ID); + new MutableRegionInfo(1L, MetaTableName.getInstance(), RegionInfo.DEFAULT_REPLICA_ID); private final TableName tableName; private byte[] startKey = HConstants.EMPTY_START_ROW; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoDisplay.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoDisplay.java index 3f353b5799d4..3b4a4ab022af 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoDisplay.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoDisplay.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.util.Bytes; @@ -82,7 +83,7 @@ public static String getRegionNameAsStringForDisplay(RegionInfo ri, Configuratio */ public static byte[] getRegionNameForDisplay(RegionInfo ri, Configuration conf) { boolean displayKey = conf.getBoolean(DISPLAY_KEYS_KEY, true); - if (displayKey || ri.getTable().equals(TableName.META_TABLE_NAME)) { + if (displayKey || ri.getTable().equals(MetaTableName.getInstance())) { return ri.getRegionName(); } else { // create a modified regionname with the startkey replaced but preserving diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java index eed1a40a2c2f..a1b766696250 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java @@ -38,6 +38,7 @@ import java.util.stream.Collectors; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.HBaseException; @@ -616,7 +617,7 @@ private ModifyableTableDescriptor(final TableName name, families.forEach(c -> this.families.put(c.getName(), ColumnFamilyDescriptorBuilder.copy(c))); this.values.putAll(values); this.values.put(IS_META_KEY, - new Bytes(Bytes.toBytes(Boolean.toString(name.equals(TableName.META_TABLE_NAME))))); + new Bytes(Bytes.toBytes(Boolean.toString(name.equals(MetaTableName.getInstance()))))); } /** diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java index f65c7ccb6e75..2206a800767e 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.HBaseRpcController; @@ -195,7 +196,7 @@ public void testCreateSystemTable() { // that we pass the correct priority @Test public void testCreateMetaTable() { - conn.getAdmin().createTable(TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME) + conn.getAdmin().createTable(TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()) .setColumnFamily(ColumnFamilyDescriptorBuilder.of("cf")).build()).join(); verify(masterStub, times(1)).createTable(assertPriority(SYSTEMTABLE_QOS), any(CreateTableRequest.class), any()); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java index a7df92999d08..a710e53bad95 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MatcherPredicate; import org.apache.hadoop.hbase.RegionLocations; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; @@ -85,7 +86,7 @@ public class TestAsyncRegionLocatorTracing { @Before public void setUp() throws IOException { - RegionInfo metaRegionInfo = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build(); + RegionInfo metaRegionInfo = RegionInfoBuilder.newBuilder(MetaTableName.getInstance()).build(); locs = new RegionLocations( new HRegionLocation(metaRegionInfo, ServerName.valueOf("127.0.0.1", 12345, EnvironmentEdgeManager.currentTime())), @@ -147,30 +148,30 @@ public void testClearCacheServerName() { @Test public void testClearCacheTableName() { - conn.getLocator().clearCache(TableName.META_TABLE_NAME); + conn.getLocator().clearCache(MetaTableName.getInstance()); SpanData span = waitSpan("AsyncRegionLocator.clearCache"); assertThat(span, allOf(hasStatusWithCode(StatusCode.OK), hasKind(SpanKind.INTERNAL), buildConnectionAttributesMatcher(conn), - buildTableAttributesMatcher(TableName.META_TABLE_NAME))); + buildTableAttributesMatcher(MetaTableName.getInstance()))); } @Test public void testGetRegionLocation() { - conn.getLocator().getRegionLocation(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW, + conn.getLocator().getRegionLocation(MetaTableName.getInstance(), HConstants.EMPTY_START_ROW, RegionLocateType.CURRENT, TimeUnit.SECONDS.toNanos(1)).join(); SpanData span = waitSpan("AsyncRegionLocator.getRegionLocation"); assertThat(span, allOf(hasStatusWithCode(StatusCode.OK), hasKind(SpanKind.INTERNAL), buildConnectionAttributesMatcher(conn), - buildTableAttributesMatcher(TableName.META_TABLE_NAME), + buildTableAttributesMatcher(MetaTableName.getInstance()), hasAttributes(containsEntryWithStringValuesOf("db.hbase.regions", locs.getDefaultRegionLocation().getRegion().getRegionNameAsString())))); } @Test public void testGetRegionLocations() { - conn.getLocator().getRegionLocations(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW, + conn.getLocator().getRegionLocations(MetaTableName.getInstance(), HConstants.EMPTY_START_ROW, RegionLocateType.CURRENT, false, TimeUnit.SECONDS.toNanos(1)).join(); SpanData span = waitSpan("AsyncRegionLocator.getRegionLocations"); String[] expectedRegions = @@ -178,7 +179,7 @@ public void testGetRegionLocations() { .map(RegionInfo::getRegionNameAsString).toArray(String[]::new); assertThat(span, allOf(hasStatusWithCode(StatusCode.OK), hasKind(SpanKind.INTERNAL), buildConnectionAttributesMatcher(conn), - buildTableAttributesMatcher(TableName.META_TABLE_NAME), hasAttributes( + buildTableAttributesMatcher(MetaTableName.getInstance()), hasAttributes( containsEntryWithStringValuesOf("db.hbase.regions", containsInAnyOrder(expectedRegions))))); } } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java index cb5431c35d3e..34e9ba201838 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java @@ -50,6 +50,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.HBaseRpcController; @@ -237,7 +238,7 @@ public void testGetSystemTable() { @Test public void testGetMetaTable() { - conn.getTable(TableName.META_TABLE_NAME).get(new Get(Bytes.toBytes(0))).join(); + conn.getTable(MetaTableName.getInstance()).get(new Get(Bytes.toBytes(0))).join(); verify(stub, times(1)).get(assertPriority(SYSTEMTABLE_QOS), any(GetRequest.class), any()); } @@ -268,7 +269,7 @@ public void testPutSystemTable() { @Test public void testPutMetaTable() { - conn.getTable(TableName.META_TABLE_NAME).put(new Put(Bytes.toBytes(0)) + conn.getTable(MetaTableName.getInstance()).put(new Put(Bytes.toBytes(0)) .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v"))).join(); verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any()); } @@ -296,7 +297,7 @@ public void testDeleteSystemTable() { @Test public void testDeleteMetaTable() { - conn.getTable(TableName.META_TABLE_NAME).delete(new Delete(Bytes.toBytes(0))).join(); + conn.getTable(MetaTableName.getInstance()).delete(new Delete(Bytes.toBytes(0))).join(); verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any()); } @@ -327,7 +328,7 @@ public void testAppendSystemTable() { @Test public void testAppendMetaTable() { - conn.getTable(TableName.META_TABLE_NAME).append(new Append(Bytes.toBytes(0)) + conn.getTable(MetaTableName.getInstance()).append(new Append(Bytes.toBytes(0)) .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v"))).join(); verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any()); } @@ -355,7 +356,7 @@ public void testIncrementSystemTable() { @Test public void testIncrementMetaTable() { - conn.getTable(TableName.META_TABLE_NAME) + conn.getTable(MetaTableName.getInstance()) .incrementColumnValue(Bytes.toBytes(0), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1).join(); verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any()); } @@ -393,7 +394,7 @@ public void testCheckAndPutSystemTable() { @Test public void testCheckAndPutMetaTable() { - conn.getTable(TableName.META_TABLE_NAME).checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")) + conn.getTable(MetaTableName.getInstance()).checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")) .qualifier(Bytes.toBytes("cq")).ifNotExists().thenPut(new Put(Bytes.toBytes(0)) .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v"))) .join(); @@ -426,7 +427,7 @@ public void testCheckAndDeleteSystemTable() { @Test public void testCheckAndDeleteMetaTable() { - conn.getTable(TableName.META_TABLE_NAME).checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")) + conn.getTable(MetaTableName.getInstance()).checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")) .qualifier(Bytes.toBytes("cq")).ifNotExists().thenPut(new Put(Bytes.toBytes(0)) .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v"))) .join(); @@ -467,7 +468,7 @@ public void testCheckAndMutateSystemTable() throws IOException { @Test public void testCheckAndMutateMetaTable() throws IOException { - conn.getTable(TableName.META_TABLE_NAME).checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")) + conn.getTable(MetaTableName.getInstance()).checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")) .qualifier(Bytes.toBytes("cq")).ifEquals(Bytes.toBytes("v")) .thenMutate(new RowMutations(Bytes.toBytes(0)).add((Mutation) new Delete(Bytes.toBytes(0)))) .join(); @@ -555,7 +556,7 @@ public void testScanSystemTable() throws Exception { @Test public void testScanMetaTable() throws Exception { CompletableFuture renewFuture = mockScanReturnRenewFuture(SYSTEMTABLE_QOS); - testForTable(TableName.META_TABLE_NAME, renewFuture, Optional.empty()); + testForTable(MetaTableName.getInstance(), renewFuture, Optional.empty()); } private void testForTable(TableName tableName, CompletableFuture renewFuture, @@ -598,7 +599,7 @@ public void testBatchSystemTable() { @Test public void testBatchMetaTable() { - conn.getTable(TableName.META_TABLE_NAME).batchAll(Arrays.asList(new Delete(Bytes.toBytes(0)))) + conn.getTable(MetaTableName.getInstance()).batchAll(Arrays.asList(new Delete(Bytes.toBytes(0)))) .join(); verify(stub, times(1)).multi(assertPriority(SYSTEMTABLE_QOS), any(ClientProtos.MultiRequest.class), any()); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java index f74b79a0672e..e01b3b741dcc 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java @@ -27,6 +27,7 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.exceptions.DeserializationException; @@ -141,7 +142,7 @@ public void testContainsRange() { @Test public void testContainsRangeForMetaTable() { TableDescriptor tableDesc = - TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).build(); + TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).build(); RegionInfo hri = RegionInfoBuilder.newBuilder(tableDesc.getTableName()).build(); byte[] startRow = HConstants.EMPTY_START_ROW; byte[] row1 = Bytes.toBytes("a,a,0"); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java index 53f33845ef7d..d09f7a225a6e 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java @@ -26,6 +26,7 @@ import java.io.IOException; import java.util.regex.Pattern; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.HBaseException; @@ -59,7 +60,7 @@ public class TestTableDescriptorBuilder { @Test(expected = IOException.class) public void testAddCoprocessorTwice() throws IOException { String cpName = "a.b.c.d"; - TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).setCoprocessor(cpName) + TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).setCoprocessor(cpName) .setCoprocessor(cpName).build(); } @@ -67,7 +68,7 @@ public void testAddCoprocessorTwice() throws IOException { public void testPb() throws DeserializationException, IOException { final int v = 123; TableDescriptor htd = - TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).setMaxFileSize(v) + TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).setMaxFileSize(v) .setDurability(Durability.ASYNC_WAL).setReadOnly(true).setRegionReplication(2).build(); byte[] bytes = TableDescriptorBuilder.toByteArray(htd); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaTableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaTableName.java index 5859e0f5d305..f4afd8bbe1c7 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaTableName.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaTableName.java @@ -2,6 +2,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hbase.thirdparty.com.google.common.base.Strings; +import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -9,6 +10,7 @@ * Singleton class for managing the META_TABLE_NAME instance. * This allows the meta table name to be overridden for testing using reflection. */ +@InterfaceAudience.Public public class MetaTableName { private static final Logger LOG = LoggerFactory.getLogger(MetaTableName.class); diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java index 3203d98df2f4..6407ff13bbdb 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java @@ -42,6 +42,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.zookeeper.ZKUtil; diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java index 8274993aa3ac..dc7d025796bd 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java index fd1f6dd5ac32..fbc98f006393 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.InvalidFamilyOperationException; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.NamespaceExistException; import org.apache.hadoop.hbase.NamespaceNotFoundException; import org.apache.hadoop.hbase.TableExistsException; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java index 031dff736c84..bfed5d2452cf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; @@ -49,7 +50,7 @@ public abstract class AbstractTestRegionLocator { protected static void startClusterAndCreateTable() throws Exception { UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), TableName.META_TABLE_NAME, REGION_REPLICATION); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), MetaTableName.getInstance(), REGION_REPLICATION); TableDescriptor td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRegionReplication(REGION_REPLICATION) .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build(); @@ -69,7 +70,7 @@ protected static void startClusterAndCreateTable() throws Exception { @After public void tearDownAfterTest() throws IOException { clearCache(TABLE_NAME); - clearCache(TableName.META_TABLE_NAME); + clearCache(MetaTableName.getInstance()); } private byte[] getStartKey(int index) { @@ -171,7 +172,7 @@ private void assertMetaRegionLocation(HRegionLocation loc, int replicaId) { assertArrayEquals(HConstants.EMPTY_END_ROW, region.getEndKey()); assertEquals(replicaId, region.getReplicaId()); ServerName expected = - findRegionLocation(TableName.META_TABLE_NAME, region.getStartKey(), replicaId); + findRegionLocation(MetaTableName.getInstance(), region.getStartKey(), replicaId); assertEquals(expected, loc.getServerName()); } @@ -184,19 +185,19 @@ private void assertMetaRegionLocations(List locs) { @Test public void testMeta() throws IOException { - assertMetaStartOrEndKeys(getStartKeys(TableName.META_TABLE_NAME)); - assertMetaStartOrEndKeys(getEndKeys(TableName.META_TABLE_NAME)); - Pair startEndKeys = getStartEndKeys(TableName.META_TABLE_NAME); + assertMetaStartOrEndKeys(getStartKeys(MetaTableName.getInstance())); + assertMetaStartOrEndKeys(getEndKeys(MetaTableName.getInstance())); + Pair startEndKeys = getStartEndKeys(MetaTableName.getInstance()); assertMetaStartOrEndKeys(startEndKeys.getFirst()); assertMetaStartOrEndKeys(startEndKeys.getSecond()); for (int replicaId = 0; replicaId < REGION_REPLICATION; replicaId++) { assertMetaRegionLocation( - getRegionLocation(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW, replicaId), + getRegionLocation(MetaTableName.getInstance(), HConstants.EMPTY_START_ROW, replicaId), replicaId); } assertMetaRegionLocations( - getRegionLocations(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW)); - assertMetaRegionLocations(getAllRegionLocations(TableName.META_TABLE_NAME)); + getRegionLocations(MetaTableName.getInstance(), HConstants.EMPTY_START_ROW)); + assertMetaRegionLocations(getAllRegionLocations(MetaTableName.getInstance())); } protected abstract byte[][] getStartKeys(TableName tableName) throws IOException; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java index 2b6b3d017fcb..0fb896111a42 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.master.assignment.AssignmentManager; import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil; @@ -58,7 +59,7 @@ protected static void startCluster() throws Exception { .numAlwaysStandByMasters(1).numMasters(1).numRegionServers(REGIONSERVERS_COUNT).build(); TEST_UTIL.startMiniCluster(option); Admin admin = TEST_UTIL.getAdmin(); - HBaseTestingUtil.setReplicas(admin, TableName.META_TABLE_NAME, 3); + HBaseTestingUtil.setReplicas(admin, MetaTableName.getInstance(), 3); AssignmentManager am = TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager(); Set sns = new HashSet(); ServerName hbaseMetaServerName = am.getRegionStates() diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java index a98ae217e3c2..38b4ee29a434 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.regionserver.Region; @@ -47,7 +48,7 @@ static void waitUntilAllMetaReplicasAreReady(HBaseTestingUtil util, ConnectionRe throws IOException { Configuration conf = util.getConfiguration(); int regionReplicaCount = - util.getAdmin().getDescriptor(TableName.META_TABLE_NAME).getRegionReplication(); + util.getAdmin().getDescriptor(MetaTableName.getInstance()).getRegionReplication(); Waiter.waitFor(conf, conf.getLong("hbase.client.sync.wait.timeout.msec", 60000), 200, true, new ExplainingPredicate() { @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java index 68a841b7d671..61a390b6ba51 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.assignment.AssignmentManager; @@ -64,7 +65,7 @@ public class TestAdmin extends TestAdminBase { @Test public void testListTableDescriptors() throws IOException { TableDescriptor metaTableDescriptor = - TEST_UTIL.getAdmin().getDescriptor(TableName.META_TABLE_NAME); + TEST_UTIL.getAdmin().getDescriptor(MetaTableName.getInstance()); List tableDescriptors = TEST_UTIL.getAdmin().listTableDescriptors(true); assertTrue(tableDescriptors.contains(metaTableDescriptor)); tableDescriptors = TEST_UTIL.getAdmin().listTableDescriptors(false); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index 2cf088fa6a82..1e4f2c19cc3d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; @@ -84,13 +85,13 @@ public class TestAdmin2 extends TestAdminBase { public void testCreateBadTables() throws IOException { String msg = null; try { - ADMIN.createTable(TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).build()); + ADMIN.createTable(TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).build()); } catch (TableExistsException e) { msg = e.toString(); } assertTrue("Unexcepted exception message " + msg, msg != null && msg.startsWith(TableExistsException.class.getName()) - && msg.contains(TableName.META_TABLE_NAME.getNameAsString())); + && msg.contains(MetaTableName.getInstance().getNameAsString())); // Now try and do concurrent creation with a bunch of threads. TableDescriptor tableDescriptor = @@ -456,7 +457,7 @@ private void setUpforLogRolling() { private HRegionServer startAndWriteData(TableName tableName, byte[] value) throws IOException, InterruptedException { // When the hbase:meta table can be opened, the region servers are running - TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME).close(); + TEST_UTIL.getConnection().getTable(MetaTableName.getInstance()).close(); // Create the test table and open it TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName) @@ -486,7 +487,7 @@ private HRegionServer startAndWriteData(TableName tableName, byte[] value) @Test public void testDisableCatalogTable() throws Exception { try { - ADMIN.disableTable(TableName.META_TABLE_NAME); + ADMIN.disableTable(MetaTableName.getInstance()); fail("Expected to throw ConstraintException"); } catch (ConstraintException e) { } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java index bb0eb31d2549..08f72317ef56 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -54,7 +55,7 @@ public class TestAsyncAdminWithRegionReplicas extends TestAsyncAdminBase { @BeforeClass public static void setUpBeforeClass() throws Exception { TestAsyncAdminBase.setUpBeforeClass(); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TableName.META_TABLE_NAME, 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), MetaTableName.getInstance(), 3); try (ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent())) { RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, registry); @@ -80,7 +81,7 @@ public void testMoveNonDefaultReplica() throws InterruptedException, ExecutionException, IOException { createTableWithDefaultConf(tableName, 3); testMoveNonDefaultReplica(tableName); - testMoveNonDefaultReplica(TableName.META_TABLE_NAME); + testMoveNonDefaultReplica(MetaTableName.getInstance()); } @Test @@ -138,11 +139,11 @@ public void testCloneTableSchema() throws IOException, InterruptedException, Exe @Test public void testGetTableRegions() throws InterruptedException, ExecutionException, IOException { - List metaRegions = admin.getRegions(TableName.META_TABLE_NAME).get(); + List metaRegions = admin.getRegions(MetaTableName.getInstance()).get(); assertEquals(3, metaRegions.size()); for (int i = 0; i < 3; i++) { RegionInfo metaRegion = metaRegions.get(i); - assertEquals(TableName.META_TABLE_NAME, metaRegion.getTable()); + assertEquals(MetaTableName.getInstance(), metaRegion.getTable()); assertEquals(i, metaRegion.getReplicaId()); } createTableWithDefaultConf(tableName, 3); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java index e14cd32a3889..7dc75ee9d935 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.RegionReplicaTestHelper.Locator; import org.apache.hadoop.hbase.client.trace.StringTraceRenderer; @@ -105,7 +106,7 @@ private void assertInitialized() { protected void before() throws Throwable { final AsyncAdmin admin = connectionRule.getAsyncConnection().getAdmin(); testUtil = miniClusterRule.getTestingUtility(); - HBaseTestingUtil.setReplicas(admin, TableName.META_TABLE_NAME, 3); + HBaseTestingUtil.setReplicas(admin, MetaTableName.getInstance(), 3); testUtil.waitUntilNoRegionsInTransition(); registry = ConnectionRegistryFactory.create(testUtil.getConfiguration(), User.getCurrent()); RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(testUtil, registry); @@ -163,7 +164,7 @@ public void test() throws Exception { TraceUtil.trace(() -> { try { - testLocator(miniClusterRule.getTestingUtility(), TableName.META_TABLE_NAME, + testLocator(miniClusterRule.getTestingUtility(), MetaTableName.getInstance(), new Locator() { @Override public void updateCachedLocationOnError(HRegionLocation loc, Throwable error) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java index 0bfbd18eb32f..4d4c307f4593 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.client.RegionReplicaTestHelper.Locator; @@ -106,9 +107,9 @@ public static void setUp() throws Exception { admin.balancerSwitch(false, true); // Enable hbase:meta replication. - HBaseTestingUtil.setReplicas(admin, TableName.META_TABLE_NAME, NUM_OF_META_REPLICA); + HBaseTestingUtil.setReplicas(admin, MetaTableName.getInstance(), NUM_OF_META_REPLICA); TEST_UTIL.waitFor(30000, - () -> TEST_UTIL.getMiniHBaseCluster().getRegions(TableName.META_TABLE_NAME).size() + () -> TEST_UTIL.getMiniHBaseCluster().getRegions(MetaTableName.getInstance()).size() >= NUM_OF_META_REPLICA); SPLIT_KEYS = new byte[8][]; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java index 61dd87007c11..ff869d5c9c0e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; @@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil; @@ -86,7 +86,7 @@ public void testSplitSwitch() throws Exception { final int rows = 10000; TestAsyncRegionAdminApi.loadData(tableName, families, rows); - AsyncTable metaTable = ASYNC_CONN.getTable(META_TABLE_NAME); + AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); int originalCount = regionLocations.size(); @@ -117,7 +117,7 @@ public void testMergeSwitch() throws Exception { byte[][] families = { FAMILY }; TestAsyncRegionAdminApi.loadData(tableName, families, 1000); - AsyncTable metaTable = ASYNC_CONN.getTable(META_TABLE_NAME); + AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); int originalCount = regionLocations.size(); @@ -162,7 +162,7 @@ public void testMergeRegions() throws Exception { byte[][] splitRows = new byte[][] { Bytes.toBytes("3"), Bytes.toBytes("6") }; createTableWithDefaultConf(tableName, splitRows); - AsyncTable metaTable = ASYNC_CONN.getTable(META_TABLE_NAME); + AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); RegionInfo regionA; @@ -242,7 +242,7 @@ private void splitTest(TableName tableName, int rowCount, boolean isSplitRegion, // create table createTableWithDefaultConf(tableName); - AsyncTable metaTable = ASYNC_CONN.getTable(META_TABLE_NAME); + AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); assertEquals(1, regionLocations.size()); @@ -299,7 +299,7 @@ public void testTruncateRegion() throws Exception { final byte[][] bFamilies = new byte[][] { Bytes.toBytes(family1), Bytes.toBytes(family2) }; createTableWithDefaultConf(tableName, splitKeys, bFamilies); - AsyncTable metaTable = ASYNC_CONN.getTable(META_TABLE_NAME); + AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); RegionInfo regionToBeTruncated = regionLocations.get(0).getRegion(); @@ -333,7 +333,7 @@ public void testTruncateReplicaRegionNotAllowed() throws Exception { final byte[][] bFamilies = new byte[][] { Bytes.toBytes(family1), Bytes.toBytes(family2) }; createTableWithDefaultConf(tableName, 2, splitKeys, bFamilies); - AsyncTable metaTable = ASYNC_CONN.getTable(META_TABLE_NAME); + AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); RegionInfo primaryRegion = regionLocations.get(0).getRegion(); @@ -354,7 +354,7 @@ public void testTruncateReplicaRegionNotAllowed() throws Exception { @Test public void testTruncateRegionsMetaTableRegionsNotAllowed() throws Exception { - AsyncTableRegionLocator locator = ASYNC_CONN.getRegionLocator(META_TABLE_NAME); + AsyncTableRegionLocator locator = ASYNC_CONN.getRegionLocator(MetaTableName.getInstance()); List regionLocations = locator.getAllRegionLocations().get(); HRegionLocation regionToBeTruncated = regionLocations.get(0); // 1 diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java index 33fbc906f19f..44d01cecb92a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -33,6 +32,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.TableExistsException; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; @@ -73,14 +73,14 @@ public void testCreateTable() throws Exception { static TableState.State getStateFromMeta(TableName table) throws Exception { Optional state = ClientMetaTableAccessor - .getTableState(ASYNC_CONN.getTable(TableName.META_TABLE_NAME), table).get(); + .getTableState(ASYNC_CONN.getTable(MetaTableName.getInstance()), table).get(); assertTrue(state.isPresent()); return state.get().getState(); } @Test public void testCreateTableNumberOfRegions() throws Exception { - AsyncTable metaTable = ASYNC_CONN.getTable(META_TABLE_NAME); + AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); createTableWithDefaultConf(tableName); List regionLocations = @@ -128,7 +128,7 @@ public void testCreateTableWithRegions() throws Exception { boolean tableAvailable = admin.isTableAvailable(tableName).get(); assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable); - AsyncTable metaTable = ASYNC_CONN.getTable(META_TABLE_NAME); + AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); List regions = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); Iterator hris = regions.iterator(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java index 9db82a3bcd82..f56412c5d589 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -57,7 +58,7 @@ public class TestAsyncTableAdminApi2 extends TestAsyncAdminBase { @Test public void testDisableCatalogTable() throws Exception { try { - this.admin.disableTable(TableName.META_TABLE_NAME).join(); + this.admin.disableTable(MetaTableName.getInstance()).join(); fail("Expected to throw ConstraintException"); } catch (Exception e) { } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java index d9007f748308..89708d40bf1e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; import static org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory.TRACKER_IMPL; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; @@ -35,6 +34,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -66,10 +66,10 @@ public void testTableExist() throws Exception { TEST_UTIL.createTable(tableName, FAMILY); exist = admin.tableExists(tableName).get(); assertTrue(exist); - exist = admin.tableExists(TableName.META_TABLE_NAME).get(); + exist = admin.tableExists(MetaTableName.getInstance()).get(); assertTrue(exist); // meta table already exists - exist = admin.tableExists(TableName.META_TABLE_NAME).get(); + exist = admin.tableExists(MetaTableName.getInstance()).get(); assertTrue(exist); } @@ -118,7 +118,7 @@ public void testListTables() throws Exception { assertEquals(0, size); Collections.addAll(tableNames, tables); - tableNames.add(TableName.META_TABLE_NAME); + tableNames.add(MetaTableName.getInstance()); tableDescs = admin.listTableDescriptors(tableNames).get(); size = tableDescs.size(); assertEquals(tables.length + 1, size); @@ -126,7 +126,7 @@ public void testListTables() throws Exception { assertTrue("tableName should be equal in order", tableDescs.get(j).getTableName().equals(tables[i])); } - assertTrue(tableDescs.get(size - 1).getTableName().equals(TableName.META_TABLE_NAME)); + assertTrue(tableDescs.get(size - 1).getTableName().equals(MetaTableName.getInstance())); for (int i = 0; i < tables.length; i++) { admin.disableTable(tables[i]).join(); @@ -205,7 +205,7 @@ public void testDisableAndEnableTable() throws Exception { assertTrue(ok); // meta table can not be disabled. try { - admin.disableTable(TableName.META_TABLE_NAME).get(); + admin.disableTable(MetaTableName.getInstance()).get(); fail("meta table can not be disabled"); } catch (ExecutionException e) { Throwable cause = e.getCause(); @@ -285,7 +285,7 @@ public void testEnableTableRetainAssignment() throws Exception { int expectedRegions = splitKeys.length + 1; createTableWithDefaultConf(tableName, splitKeys); - AsyncTable metaTable = ASYNC_CONN.getTable(META_TABLE_NAME); + AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); List regions = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); assertEquals( @@ -314,8 +314,8 @@ public void testIsTableEnabledAndDisabled() throws Exception { assertTrue(admin.isTableDisabled(tableName).get()); // meta table is always enabled - assertTrue(admin.isTableEnabled(TableName.META_TABLE_NAME).get()); - assertFalse(admin.isTableDisabled(TableName.META_TABLE_NAME).get()); + assertTrue(admin.isTableEnabled(MetaTableName.getInstance()).get()); + assertFalse(admin.isTableDisabled(MetaTableName.getInstance()).get()); } @Test @@ -323,6 +323,6 @@ public void testIsTableAvailable() throws Exception { createTableWithDefaultConf(tableName); TEST_UTIL.waitTableAvailable(tableName); assertTrue(admin.isTableAvailable(tableName).get()); - assertTrue(admin.isTableAvailable(TableName.META_TABLE_NAME).get()); + assertTrue(admin.isTableAvailable(MetaTableName.getInstance()).get()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java index 18c53a49de7b..96bb7b2bd1ba 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; @@ -94,14 +95,14 @@ public static void setUp() throws Exception { conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, FailPrimaryMetaScanCp.class.getName()); UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), TableName.META_TABLE_NAME, 3); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), MetaTableName.getInstance(), 3); try (ConnectionRegistry registry = ConnectionRegistryFactory.create(conf, User.getCurrent())) { RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(UTIL, registry); } try (Table table = UTIL.createTable(TABLE_NAME, FAMILY)) { table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)); } - UTIL.flush(TableName.META_TABLE_NAME); + UTIL.flush(MetaTableName.getInstance()); // wait for the store file refresh so we can read the region location from secondary meta // replicas Thread.sleep(2000); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java index 184b4ba0d3cc..e180d33df541 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.client; import static org.apache.hadoop.hbase.HConstants.EMPTY_START_ROW; -import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; @@ -29,6 +28,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.RegionLocations; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -72,9 +72,9 @@ public static void setUp() throws Exception { admin.balancerSwitch(false, true); // Enable hbase:meta replication. - HBaseTestingUtil.setReplicas(admin, TableName.META_TABLE_NAME, numOfMetaReplica); + HBaseTestingUtil.setReplicas(admin, MetaTableName.getInstance(), numOfMetaReplica); TEST_UTIL.waitFor(30000, - () -> TEST_UTIL.getMiniHBaseCluster().getRegions(TableName.META_TABLE_NAME).size() + () -> TEST_UTIL.getMiniHBaseCluster().getRegions(MetaTableName.getInstance()).size() >= numOfMetaReplica); registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); @@ -95,14 +95,14 @@ public void testMetaChangeFromReplicaNoReplica() throws IOException, Interrupted CatalogReplicaLoadBalanceSimpleSelector.class.getName()); CatalogReplicaLoadBalanceSelector metaSelector = CatalogReplicaLoadBalanceSelectorFactory - .createSelector(replicaSelectorClass, META_TABLE_NAME, CONN, () -> { + .createSelector(replicaSelectorClass, MetaTableName.getInstance(), CONN, () -> { int numOfReplicas = CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS; try { RegionLocations metaLocations = CONN.registry.getMetaRegionLocations() .get(CONN.connConf.getMetaReadRpcTimeoutNs(), TimeUnit.NANOSECONDS); numOfReplicas = metaLocations.size(); } catch (Exception e) { - LOG.error("Failed to get table {}'s region replication, ", META_TABLE_NAME, e); + LOG.error("Failed to get table {}'s region replication, ", MetaTableName.getInstance(), e); } return numOfReplicas; }); @@ -116,12 +116,12 @@ public void testMetaChangeFromReplicaNoReplica() throws IOException, Interrupted IntStream.range(0, numOfMetaReplica).forEach(i -> assertNotEquals(replicaIdCount[i], 0)); // Change to No meta replica - HBaseTestingUtil.setReplicas(admin, TableName.META_TABLE_NAME, 1); + HBaseTestingUtil.setReplicas(admin, MetaTableName.getInstance(), 1); TEST_UTIL.waitFor(30000, - () -> TEST_UTIL.getMiniHBaseCluster().getRegions(TableName.META_TABLE_NAME).size() == 1); + () -> TEST_UTIL.getMiniHBaseCluster().getRegions(MetaTableName.getInstance()).size() == 1); CatalogReplicaLoadBalanceSelector metaSelectorWithNoReplica = - CatalogReplicaLoadBalanceSelectorFactory.createSelector(replicaSelectorClass, META_TABLE_NAME, + CatalogReplicaLoadBalanceSelectorFactory.createSelector(replicaSelectorClass, MetaTableName.getInstance(), CONN, () -> { int numOfReplicas = CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS; try { @@ -129,7 +129,7 @@ public void testMetaChangeFromReplicaNoReplica() throws IOException, Interrupted .get(CONN.connConf.getMetaReadRpcTimeoutNs(), TimeUnit.NANOSECONDS); numOfReplicas = metaLocations.size(); } catch (Exception e) { - LOG.error("Failed to get table {}'s region replication, ", META_TABLE_NAME, e); + LOG.error("Failed to get table {}'s region replication, ", MetaTableName.getInstance(), e); } return numOfReplicas; }); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java index 912ded0a27bb..3c314b5aa968 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; @@ -48,7 +49,7 @@ public void testReplicaCleanup() throws Exception { ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher(); List metaReplicaZnodes = zkw.getMetaReplicaNodes(); assertEquals(3, metaReplicaZnodes.size()); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TableName.META_TABLE_NAME, 1); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), MetaTableName.getInstance(), 1); metaReplicaZnodes = zkw.getMetaReplicaNodes(); assertEquals(1, metaReplicaZnodes.size()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java index 583dc02763d0..91beea20c73c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.SingleProcessHBaseCluster.MiniHBaseClusterRegionServer; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException; import org.apache.hadoop.hbase.ipc.CallTimeoutException; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -453,7 +454,7 @@ public ScanResponse scan(final RpcController controller, final ScanRequest reque } else { ScanResponse scanRes = super.scan(controller, request); String regionName = Bytes.toString(request.getRegion().getValue().toByteArray()); - if (!regionName.contains(TableName.META_TABLE_NAME.getNameAsString())) { + if (!regionName.contains(MetaTableName.getInstance().getNameAsString())) { tableScannerId = scanRes.getScannerId(); if (sleepOnOpen) { try { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java index 253e61f995cf..b9294f8f991b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.client.metrics.ScanMetricsRegionInfo; import org.apache.hadoop.hbase.filter.FilterBase; @@ -89,8 +90,8 @@ public void setup() throws IOException { conf = TEST_UTIL.getConfiguration(); rootDir = TEST_UTIL.getDefaultRootDirPath(); fs = TEST_UTIL.getTestFileSystem(); - htd = TEST_UTIL.getAdmin().getDescriptor(TableName.META_TABLE_NAME); - hri = TEST_UTIL.getAdmin().getRegions(TableName.META_TABLE_NAME).get(0); + htd = TEST_UTIL.getAdmin().getDescriptor(MetaTableName.getInstance()); + hri = TEST_UTIL.getAdmin().getRegions(MetaTableName.getInstance()).get(0); scan = new Scan(); } @@ -200,7 +201,7 @@ private void testScanMetricsWithScanMetricsByRegionDisabled(ScanMetrics scanMetr Configuration copyConf = new Configuration(conf); Scan scan = new Scan(); scan.setScanMetricsEnabled(true); - TEST_UTIL.getAdmin().flush(TableName.META_TABLE_NAME); + TEST_UTIL.getAdmin().flush(MetaTableName.getInstance()); try (ClientSideRegionScanner clientSideRegionScanner = new ClientSideRegionScanner(copyConf, fs, rootDir, htd, hri, scan, scanMetrics)) { clientSideRegionScanner.next(); @@ -229,7 +230,7 @@ private void testScanMetricByRegion(ScanMetrics scanMetrics) throws IOException Configuration copyConf = new Configuration(conf); Scan scan = new Scan(); scan.setEnableScanMetricsByRegion(true); - TEST_UTIL.getAdmin().flush(TableName.META_TABLE_NAME); + TEST_UTIL.getAdmin().flush(MetaTableName.getInstance()); try (ClientSideRegionScanner clientSideRegionScanner = new ClientSideRegionScanner(copyConf, fs, rootDir, htd, hri, scan, scanMetrics)) { clientSideRegionScanner.next(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java index 2384e02955da..b5b3e652ea89 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; @@ -92,7 +93,7 @@ public void testDeleteForSureClearsAllTableRowsFromMeta() } // Now I have a nice table, mangle it by removing the HConstants.REGIONINFO_QUALIFIER_STR // content from a few of the rows. - try (Table metaTable = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) { + try (Table metaTable = TEST_UTIL.getConnection().getTable(MetaTableName.getInstance())) { try (ResultScanner scanner = metaTable.getScanner( MetaTableAccessor.getScanForTableName(TEST_UTIL.getConfiguration(), tableName))) { for (Result result : scanner) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java index 8f820158e460..8a46e0f79a18 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java @@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Scan.ReadType; @@ -2532,7 +2533,7 @@ public void testFilterAllRecords() throws IOException { scan.setCaching(1); // Filter out any records scan.setFilter(new FilterList(new FirstKeyOnlyFilter(), new InclusiveStopFilter(new byte[0]))); - try (Table table = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) { + try (Table table = TEST_UTIL.getConnection().getTable(MetaTableName.getInstance())) { try (ResultScanner s = table.getScanner(scan)) { assertNull(s.next()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java index f93fc9d5bf5d..044be8b5aa70 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -50,7 +51,7 @@ public static void setUp() throws Exception { public void testUpgradeAndIncreaseReplicaCount() throws Exception { HMaster oldMaster = TEST_UTIL.getMiniHBaseCluster().getMaster(); TableDescriptors oldTds = oldMaster.getTableDescriptors(); - TableDescriptor oldMetaTd = oldTds.get(TableName.META_TABLE_NAME); + TableDescriptor oldMetaTd = oldTds.get(MetaTableName.getInstance()); assertEquals(3, oldMetaTd.getRegionReplication()); // force update the replica count to 1 and then kill the master, to simulate that hen upgrading, // we have no region replication in meta table descriptor but we actually have meta region diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java index d79603cea3cc..e14ccfa949e8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.security.User; @@ -61,7 +62,7 @@ public static void setUp() throws Exception { StartTestingClusterOption.Builder builder = StartTestingClusterOption.builder(); builder.numMasters(3).numRegionServers(3); TEST_UTIL.startMiniCluster(builder.build()); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TableName.META_TABLE_NAME, 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), MetaTableName.getInstance(), 3); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java index ce52918bfe42..5c4d6642f311 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.RegionTooBusyException; import org.apache.hadoop.hbase.RetryImmediatelyException; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil; import org.apache.hadoop.hbase.exceptions.RegionOpeningException; import org.apache.hadoop.hbase.quotas.RpcThrottlingException; @@ -87,7 +88,7 @@ public static void setUpBeforeClass() throws Exception { conf.setStrings(HConstants.REGION_SERVER_IMPL, RegionServerWithFakeRpcServices.class.getName()); TEST_UTIL.startMiniCluster(1); TEST_UTIL.getHBaseCluster().waitForActiveAndReadyMaster(); - TEST_UTIL.waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME); + TEST_UTIL.waitUntilAllRegionsAssigned(MetaTableName.getInstance()); badRS = TEST_UTIL.getHBaseCluster().getRegionServer(0); assertTrue(badRS.getRSRpcServices() instanceof FakeRSRpcServices); TableDescriptor desc = TableDescriptorBuilder.newBuilder(TABLE_NAME) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java index 29223dea5dbe..977a3571f595 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.MultithreadedTestUtil; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.security.User; @@ -63,7 +64,7 @@ public class TestMetaRegionLocationCache { @BeforeClass public static void setUp() throws Exception { TEST_UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TableName.META_TABLE_NAME, 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), MetaTableName.getInstance(), 3); REGISTRY = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, REGISTRY); TEST_UTIL.getAdmin().balancerSwitch(false, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java index eae7078639d1..799c90c57c3f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -54,7 +55,7 @@ public static void setUp() throws Exception { @Test public void testMetaHTDReplicaCount() throws Exception { assertEquals(3, - TEST_UTIL.getAdmin().getDescriptor(TableName.META_TABLE_NAME).getRegionReplication()); + TEST_UTIL.getAdmin().getDescriptor(MetaTableName.getInstance()).getRegionReplication()); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java index 267d618d03d1..aab092dda927 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -97,7 +98,7 @@ public static void shutdownMetaAndDoValidations(HBaseTestingUtil util) throws Ex ServerName master = null; try (Connection c = ConnectionFactory.createConnection(util.getConfiguration())) { try (Table htable = util.createTable(TABLE, FAMILIES)) { - util.getAdmin().flush(TableName.META_TABLE_NAME); + util.getAdmin().flush(MetaTableName.getInstance()); Thread.sleep( conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 30000) * 6); List regions = MetaTableAccessor.getTableRegions(c, TABLE); @@ -114,7 +115,7 @@ public static void shutdownMetaAndDoValidations(HBaseTestingUtil util) throws Ex Thread.sleep(10); hrl = MetaTableAccessor.getRegionLocation(c, regions.get(0)); } while (primary.equals(hrl.getServerName())); - util.getAdmin().flush(TableName.META_TABLE_NAME); + util.getAdmin().flush(MetaTableName.getInstance()); Thread.sleep( conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 30000) * 3); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java index 55646c35e435..4e6f9965c6d9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -48,7 +49,7 @@ public class TestMultiActionMetricsFromClient { public static void setUpBeforeClass() throws Exception { TEST_UTIL.startMiniCluster(1); TEST_UTIL.getHBaseCluster().waitForActiveAndReadyMaster(); - TEST_UTIL.waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME); + TEST_UTIL.waitUntilAllRegionsAssigned(MetaTableName.getInstance()); TEST_UTIL.createTable(TABLE_NAME, FAMILY); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java index c38be19a238e..caa7c825601e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor; import org.apache.hadoop.hbase.coprocessor.ObserverContext; @@ -261,7 +262,7 @@ public static void beforeClass() throws Exception { HTU.startMiniCluster(NB_SERVERS); // Enable meta replica at server side - HBaseTestingUtil.setReplicas(HTU.getAdmin(), TableName.META_TABLE_NAME, 2); + HBaseTestingUtil.setReplicas(HTU.getAdmin(), MetaTableName.getInstance(), 2); HTU.getHBaseCluster().startMaster(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java index d33cc943355c..14c750398a3a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.regionserver.BootstrapNodeManager; import org.apache.hadoop.hbase.regionserver.RSRpcServices; @@ -72,7 +73,7 @@ public static void setUpBeforeClass() throws Exception { UTIL.getConfiguration().setLong(RpcConnectionRegistry.MIN_SECS_BETWEEN_REFRESHES, 0); UTIL.getConfiguration().setLong(BootstrapNodeManager.REQUEST_MASTER_MIN_INTERVAL_SECS, 1); UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), TableName.META_TABLE_NAME, 3); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), MetaTableName.getInstance(), 3); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java index 52ccd5d8b7da..84430264474d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil; @@ -280,13 +281,13 @@ public void testAsyncTable() throws Exception { public void testChangeMetaReplicaCount() throws Exception { Admin admin = TEST_UTIL.getAdmin(); try (RegionLocator locator = - TEST_UTIL.getConnection().getRegionLocator(TableName.META_TABLE_NAME)) { + TEST_UTIL.getConnection().getRegionLocator(MetaTableName.getInstance())) { assertEquals(1, locator.getAllRegionLocations().size()); - HBaseTestingUtil.setReplicas(admin, TableName.META_TABLE_NAME, 3); + HBaseTestingUtil.setReplicas(admin, MetaTableName.getInstance(), 3); TEST_UTIL.waitFor(30000, () -> locator.getAllRegionLocations().size() == 3); - HBaseTestingUtil.setReplicas(admin, TableName.META_TABLE_NAME, 2); + HBaseTestingUtil.setReplicas(admin, MetaTableName.getInstance(), 2); TEST_UTIL.waitFor(30000, () -> locator.getAllRegionLocations().size() == 2); - HBaseTestingUtil.setReplicas(admin, TableName.META_TABLE_NAME, 1); + HBaseTestingUtil.setReplicas(admin, MetaTableName.getInstance(), 1); TEST_UTIL.waitFor(30000, () -> locator.getAllRegionLocations().size() == 1); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java index c39fc076ef2b..32c9df8ad3bc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java @@ -22,6 +22,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.junit.BeforeClass; @@ -50,7 +51,7 @@ public void testShutdownOfReplicaHolder() throws Exception { // checks that the when the server holding meta replica is shut down, the meta replica // can be recovered try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); - RegionLocator locator = conn.getRegionLocator(TableName.META_TABLE_NAME)) { + RegionLocator locator = conn.getRegionLocator(MetaTableName.getInstance())) { HRegionLocation hrl = locator.getRegionLocations(HConstants.EMPTY_START_ROW, true).get(1); ServerName oldServer = hrl.getServerName(); TEST_UTIL.getHBaseClusterInterface().killRegionServer(oldServer); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java index 8abb4d754a7a..c0b6a337da26 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; @@ -163,7 +164,7 @@ public static void cleanupTest() throws Exception { */ @Test(expected = IllegalArgumentException.class) public void testMetaTablesSnapshot() throws Exception { - UTIL.getAdmin().snapshot("metaSnapshot", TableName.META_TABLE_NAME); + UTIL.getAdmin().snapshot("metaSnapshot", MetaTableName.getInstance()); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java index 6d585245e959..395cddf1cd97 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; @@ -64,7 +65,7 @@ public class TestZKConnectionRegistry { @BeforeClass public static void setUp() throws Exception { TEST_UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TableName.META_TABLE_NAME, 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), MetaTableName.getInstance(), 3); REGISTRY = new ZKConnectionRegistry(TEST_UTIL.getConfiguration(), null); } @@ -89,7 +90,7 @@ public void test() throws InterruptedException, ExecutionException, IOException IntStream.range(0, 3).forEach(i -> { HRegionLocation loc = locs.getRegionLocation(i); assertNotNull("Replica " + i + " doesn't have location", loc); - assertEquals(TableName.META_TABLE_NAME, loc.getRegion().getTable()); + assertEquals(MetaTableName.getInstance(), loc.getRegion().getTable()); assertEquals(i, loc.getRegion().getReplicaId()); }); } From c0cec14e96a0779ca5591b5c242420d99b41ffe0 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Mon, 15 Dec 2025 18:55:02 -0500 Subject: [PATCH 03/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: org.apache.hadoop.hbase.master --- .../apache/hadoop/hbase/master/HMaster.java | 19 ++++++++++-------- ...sterAnnotationReadingPriorityFunction.java | 3 ++- .../master/RegionPlacementMaintainer.java | 12 ++++++----- .../hbase/master/TableNamespaceManager.java | 9 +++++---- .../hbase/master/TableStateManager.java | 5 +++-- .../master/assignment/AssignmentManager.java | 7 ++++--- .../MergeTableRegionsProcedure.java | 7 +++++-- .../master/assignment/RegionStateStore.java | 20 ++++++++++--------- .../assignment/SplitTableRegionProcedure.java | 7 +++++-- .../cleaner/ReplicationBarrierCleaner.java | 3 ++- .../hadoop/hbase/master/http/MetaBrowser.java | 3 ++- .../hbase/master/janitor/CatalogJanitor.java | 7 ++++--- .../hbase/master/janitor/MetaFixer.java | 13 ++++++------ .../master/janitor/ReportMakingVisitor.java | 6 ++++-- .../procedure/DeleteTableProcedure.java | 5 +++-- .../procedure/DisableTableProcedure.java | 5 +++-- .../procedure/HBCKServerCrashProcedure.java | 13 ++++++------ .../master/procedure/InitMetaProcedure.java | 5 +++-- .../procedure/MasterProcedureScheduler.java | 5 +++-- .../hbase/master/procedure/MetaQueue.java | 3 ++- .../MigrateNamespaceTableProcedure.java | 5 +++-- .../procedure/ModifyTableProcedure.java | 3 ++- .../hbase/master/procedure/SchemaLocking.java | 5 +++-- .../procedure/TruncateRegionProcedure.java | 3 ++- .../hadoop/hbase/master/TestMaster.java | 3 ++- ...MasterFileSystemWithStoreFileTracking.java | 3 ++- ...TestMasterOperationsForRegionReplicas.java | 3 ++- .../hbase/master/TestMasterRepairMode.java | 5 +++-- .../hbase/master/TestMasterTransitions.java | 3 ++- .../TestMetaAssignmentWithStopMaster.java | 3 ++- .../TestMigrateAndMirrorMetaLocations.java | 9 +++++---- .../master/TestMigrateNamespaceTable.java | 5 +++-- .../hbase/master/TestRecreateCluster.java | 5 +++-- .../TestRestartWithEmptyWALDirectory.java | 3 ++- ...ServerCrashProcedureCarryingMetaStuck.java | 5 +++-- .../TestCloseRegionWhileRSCrash.java | 7 ++++--- .../TestOpenRegionProcedureBackoff.java | 3 ++- .../assignment/TestRegionStateStore.java | 3 ++- .../master/assignment/TestRollbackSCP.java | 3 ++- .../BalancerConditionalsTestUtil.java | 3 ++- .../TestFavoredStochasticLoadBalancer.java | 3 ++- ...MetaTableIsolationBalancerConditional.java | 5 +++-- .../TestReplicationBarrierCleaner.java | 13 ++++++------ .../TestCatalogJanitorInMemoryStates.java | 3 ++- .../janitor/TestMetaFixerNoCluster.java | 17 ++++++++-------- .../TestSimpleRegionNormalizer.java | 3 ++- .../hbase/master/procedure/TestHBCKSCP.java | 5 +++-- .../procedure/TestProcedurePriority.java | 3 ++- ...TestTableProcedureWaitingQueueCleanup.java | 3 ++- 49 files changed, 176 insertions(+), 118 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index c997f1c6e822..d3498bf3d897 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -93,6 +93,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerTask; import org.apache.hadoop.hbase.ServerTaskBuilder; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; @@ -1092,7 +1093,7 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE startupTaskGroup.addTask("Initializing meta table if this is a new deploy"); InitMetaProcedure initMetaProc = null; // Print out state of hbase:meta on startup; helps debugging. - if (!this.assignmentManager.getRegionStates().hasTableRegionStates(TableName.META_TABLE_NAME)) { + if (!this.assignmentManager.getRegionStates().hasTableRegionStates(MetaTableName.getInstance())) { Optional optProc = procedureExecutor.getProcedures().stream() .filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny(); initMetaProc = optProc.orElseGet(() -> { @@ -1156,7 +1157,7 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE return; } - TableDescriptor metaDescriptor = tableDescriptors.get(TableName.META_TABLE_NAME); + TableDescriptor metaDescriptor = tableDescriptors.get(MetaTableName.getInstance()); final ColumnFamilyDescriptor tableFamilyDesc = metaDescriptor.getColumnFamily(HConstants.TABLE_FAMILY); final ColumnFamilyDescriptor replBarrierFamilyDesc = @@ -1174,16 +1175,17 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE if (conf.get(HConstants.META_REPLICAS_NUM) != null) { int replicasNumInConf = conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM); - TableDescriptor metaDesc = tableDescriptors.get(TableName.META_TABLE_NAME); + TableDescriptor metaDesc = tableDescriptors.get(MetaTableName.getInstance()); if (metaDesc.getRegionReplication() != replicasNumInConf) { // it is possible that we already have some replicas before upgrading, so we must set the // region replication number in meta TableDescriptor directly first, without creating a // ModifyTableProcedure, otherwise it may cause a double assign for the meta replicas. int existingReplicasCount = - assignmentManager.getRegionStates().getRegionsOfTable(TableName.META_TABLE_NAME).size(); + assignmentManager.getRegionStates().getRegionsOfTable(MetaTableName.getInstance()).size(); if (existingReplicasCount > metaDesc.getRegionReplication()) { - LOG.info("Update replica count of hbase:meta from {}(in TableDescriptor)" - + " to {}(existing ZNodes)", metaDesc.getRegionReplication(), existingReplicasCount); + LOG.info( + "Update replica count of {} from {}(in TableDescriptor)" + " to {}(existing ZNodes)", + MetaTableName.getInstance(), metaDesc.getRegionReplication(), existingReplicasCount); metaDesc = TableDescriptorBuilder.newBuilder(metaDesc) .setRegionReplication(existingReplicasCount).build(); tableDescriptors.update(metaDesc); @@ -1193,7 +1195,8 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE LOG.info( "The {} config is {} while the replica count in TableDescriptor is {}" + " for hbase:meta, altering...", - HConstants.META_REPLICAS_NUM, replicasNumInConf, metaDesc.getRegionReplication()); + HConstants.META_REPLICAS_NUM, replicasNumInConf, metaDesc.getRegionReplication(), + MetaTableName.getInstance()); procedureExecutor.submitProcedure(new ModifyTableProcedure( procedureExecutor.getEnvironment(), TableDescriptorBuilder.newBuilder(metaDesc) .setRegionReplication(replicasNumInConf).build(), @@ -2601,7 +2604,7 @@ private void startActiveMasterManager(int infoPort) throws KeeperException { } private static boolean isCatalogTable(final TableName tableName) { - return tableName.equals(TableName.META_TABLE_NAME); + return tableName.equals(MetaTableName.getInstance()); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterAnnotationReadingPriorityFunction.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterAnnotationReadingPriorityFunction.java index 464dfaca7035..c77d2c0c1a85 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterAnnotationReadingPriorityFunction.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterAnnotationReadingPriorityFunction.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.master; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.AnnotationReadingPriorityFunction; import org.apache.yetus.audience.InterfaceAudience; @@ -84,7 +85,7 @@ protected int getBasePriority(RequestHeader header, Message param) { if (rst.getRegionInfoList() != null) { for (HBaseProtos.RegionInfo info : rst.getRegionInfoList()) { TableName tn = ProtobufUtil.toTableName(info.getTableName()); - if (TableName.META_TABLE_NAME.equals(tn)) { + if (MetaTableName.getInstance().equals(tn)) { return META_TRANSITION_QOS; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java index 854c21da2bc7..c1195e0a90a1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AsyncClusterConnection; import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin; @@ -605,7 +606,7 @@ public static void printAssignmentPlan(FavoredNodesPlan plan) { */ public void updateAssignmentPlanToMeta(FavoredNodesPlan plan) throws IOException { try { - LOG.info("Start to update the hbase:meta with the new assignment plan"); + LOG.info("Started updating {} with the new assignment plan", MetaTableName.getInstance()); Map> assignmentMap = plan.getAssignmentMap(); Map> planToUpdate = new HashMap<>(assignmentMap.size()); Map regionToRegionInfoMap = @@ -619,6 +620,7 @@ public void updateAssignmentPlanToMeta(FavoredNodesPlan plan) throws IOException } catch (Exception e) { LOG.error( "Failed to update hbase:meta with the new assignment" + "plan because " + e.getMessage()); + LOG.info("Updated {} with the new assignment plan", MetaTableName.getInstance()); } } @@ -690,14 +692,14 @@ private void updateAssignmentPlanToRegionServers(FavoredNodesPlan plan) throws I } public void updateAssignmentPlan(FavoredNodesPlan plan) throws IOException { - LOG.info("Start to update the new assignment plan for the hbase:meta table and" - + " the region servers"); + LOG.info("Started updating the new assignment plan for {} and the region servers", + MetaTableName.getInstance()); // Update the new assignment plan to META updateAssignmentPlanToMeta(plan); // Update the new assignment plan to Region Servers updateAssignmentPlanToRegionServers(plan); - LOG.info("Finish to update the new assignment plan for the hbase:meta table and" - + " the region servers"); + LOG.info("Finished updating the new assignment plan for {} and the region servers", + MetaTableName.getInstance()); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java index 4d18b2ad8f4e..70aa46960e00 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.NamespaceDescriptor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Delete; @@ -79,7 +80,7 @@ private void tryMigrateNamespaceTable() throws IOException, InterruptedException if (!opt.isPresent()) { // the procedure is not present, check whether have the ns family in meta table TableDescriptor metaTableDesc = - masterServices.getTableDescriptors().get(TableName.META_TABLE_NAME); + masterServices.getTableDescriptors().get(MetaTableName.getInstance()); if (metaTableDesc.hasColumnFamily(HConstants.NAMESPACE_FAMILY)) { // normal case, upgrading is done or the cluster is created with 3.x code migrationDone = true; @@ -106,7 +107,7 @@ private void addToCache(Result result, byte[] family, byte[] qualifier) throws I } private void loadFromMeta() throws IOException { - try (Table table = masterServices.getConnection().getTable(TableName.META_TABLE_NAME); + try (Table table = masterServices.getConnection().getTable(MetaTableName.getInstance()); ResultScanner scanner = table.getScanner(HConstants.NAMESPACE_FAMILY)) { for (Result result;;) { result = scanner.next(); @@ -204,7 +205,7 @@ public static void insertNamespaceToMeta(Connection conn, NamespaceDescriptor ns Put put = new Put(row, true).addColumn(HConstants.NAMESPACE_FAMILY, HConstants.NAMESPACE_COL_DESC_QUALIFIER, ProtobufUtil.toProtoNamespaceDescriptor(ns).toByteArray()); - try (Table table = conn.getTable(TableName.META_TABLE_NAME)) { + try (Table table = conn.getTable(MetaTableName.getInstance())) { table.put(put); } } @@ -212,7 +213,7 @@ public static void insertNamespaceToMeta(Connection conn, NamespaceDescriptor ns public void deleteNamespace(String namespaceName) throws IOException { checkMigrationDone(); Delete d = new Delete(Bytes.toBytes(namespaceName)); - try (Table table = masterServices.getConnection().getTable(TableName.META_TABLE_NAME)) { + try (Table table = masterServices.getConnection().getTable(MetaTableName.getInstance())) { table.delete(d); } cache.remove(namespaceName); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java index 6ad32623be1a..cdd54d616bee 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.CatalogFamilyFormat; import org.apache.hadoop.hbase.ClientMetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Result; @@ -86,7 +87,7 @@ public boolean isTableState(TableName tableName, TableState.State... states) { } public void setDeletedTable(TableName tableName) throws IOException { - if (tableName.equals(TableName.META_TABLE_NAME)) { + if (tableName.equals(MetaTableName.getInstance())) { // Can't delete the hbase:meta table. return; } @@ -147,7 +148,7 @@ public TableState getTableState(TableName tableName) throws IOException { } private void updateMetaState(TableName tableName, TableState.State newState) throws IOException { - if (tableName.equals(TableName.META_TABLE_NAME)) { + if (tableName.equals(MetaTableName.getInstance())) { if ( TableState.State.DISABLING.equals(newState) || TableState.State.DISABLED.equals(newState) ) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java index 7bfa79e21484..7d7f2dacda89 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java @@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.PleaseHoldException; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.client.DoNotRetryRegionException; @@ -354,7 +355,7 @@ public void start() throws IOException, KeeperException { if (RegionReplicaUtil.isDefaultReplica(regionInfo.getReplicaId())) { setMetaAssigned(regionInfo, state == State.OPEN); } - LOG.debug("Loaded hbase:meta {}", regionNode); + LOG.debug("Loaded {} {}", MetaTableName.getInstance(), regionNode); }, result); } } @@ -1962,8 +1963,8 @@ private void checkMetaLoaded(RegionInfo hri, long procId) throws PleaseHoldExcep boolean meta = isMetaRegion(hri); boolean metaLoaded = isMetaLoaded(); if (!meta && !metaLoaded) { - throw new PleaseHoldException( - "Master not fully online; hbase:meta=" + meta + ", metaLoaded=" + metaLoaded); + throw new PleaseHoldException("Master not fully online; " + MetaTableName.getInstance() + "=" + + meta + ", metaLoaded=" + metaLoaded); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java index c370fed9d9c0..11ae106a0b8c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java @@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.MetaMutationAnnotation; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -718,8 +719,10 @@ private void preMergeRegionsCommit(final MasterProcedureEnv env) throws IOExcept RegionInfo.parseRegionName(p.getRow()); } } catch (IOException e) { - LOG.error("Row key of mutation from coprocessor is not parsable as region name. " - + "Mutations from coprocessor should only be for hbase:meta table.", e); + LOG.error( + "Row key of mutation from coprocessor is not parsable as region name. " + + "Mutations from coprocessor should only be for {} table.", + MetaTableName.getInstance(), e); throw e; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java index 5987fc7537b4..8bcf3b5a93fb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AsyncTable; import org.apache.hadoop.hbase.client.Delete; @@ -169,9 +170,10 @@ public static void visitMetaEntry(final RegionStateVisitor visitor, final Result final long openSeqNum = hrl.getSeqNum(); LOG.debug( - "Load hbase:meta entry region={}, regionState={}, lastHost={}, " + "Load {} entry region={}, regionState={}, lastHost={}, " + "regionLocation={}, openSeqNum={}", - regionInfo.getEncodedName(), state, lastHost, regionLocation, openSeqNum); + MetaTableName.getInstance(), regionInfo.getEncodedName(), state, lastHost, regionLocation, + openSeqNum); visitor.visitRegionState(result, regionInfo, state, regionLocation, lastHost, openSeqNum); } } @@ -190,8 +192,8 @@ private Put generateUpdateRegionLocationPut(RegionStateNode regionStateNode) thr final Put put = new Put(CatalogFamilyFormat.getMetaKeyForRegion(regionInfo), time); MetaTableAccessor.addRegionInfo(put, regionInfo); final StringBuilder info = - new StringBuilder("pid=").append(pid).append(" updating hbase:meta row=") - .append(regionInfo.getEncodedName()).append(", regionState=").append(state); + new StringBuilder("pid=").append(pid).append(" updating ").append(MetaTableName.getInstance()) + .append(" row=").append(regionInfo.getEncodedName()).append(", regionState=").append(state); if (openSeqNum >= 0) { Preconditions.checkArgument(state == State.OPEN && regionLocation != null, "Open region should be on a server"); @@ -283,7 +285,7 @@ private CompletableFuture updateRegionLocation(RegionInfo regionInfo, Stat future = FutureUtils.failedFuture(e); } } else { - AsyncTable table = master.getAsyncConnection().getTable(TableName.META_TABLE_NAME); + AsyncTable table = master.getAsyncConnection().getTable(MetaTableName.getInstance()); future = table.put(put); } FutureUtils.addListener(future, (r, e) -> { @@ -330,7 +332,7 @@ private void multiMutate(RegionInfo ri, List mutations) throws IOExcep } MutateRowsRequest request = builder.build(); AsyncTable table = - master.getConnection().toAsyncConnection().getTable(TableName.META_TABLE_NAME); + master.getConnection().toAsyncConnection().getTable(MetaTableName.getInstance()); CompletableFuture future = table. coprocessorService(MultiRowMutationService::newStub, (stub, controller, done) -> stub.mutateRows(controller, request, done), row); @@ -338,7 +340,7 @@ MutateRowsResponse> coprocessorService(MultiRowMutationService::newStub, } private Table getMetaTable() throws IOException { - return master.getConnection().getTable(TableName.META_TABLE_NAME); + return master.getConnection().getTable(MetaTableName.getInstance()); } private Result getRegionCatalogResult(RegionInfo region) throws IOException { @@ -504,7 +506,7 @@ public void deleteMergeQualifiers(RegionInfo mergeRegion) throws IOException { + " in meta table, they are cleaned up already, Skip."); return; } - try (Table table = master.getConnection().getTable(TableName.META_TABLE_NAME)) { + try (Table table = master.getConnection().getTable(MetaTableName.getInstance())) { table.delete(delete); } LOG.info( @@ -694,7 +696,7 @@ public static State getRegionState(final Result r, RegionInfo regionInfo) { return State.valueOf(state); } catch (IllegalArgumentException e) { LOG.warn( - "BAD value {} in hbase:meta info:state column for region {} , " + "BAD value {} in " + MetaTableName.getInstance() + " info:state column for region {} , " + "Consider using HBCK2 setRegionState ENCODED_REGION_NAME STATE", state, regionInfo.getEncodedName()); return null; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java index 3d3d3d18de23..1e405a001a9e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -903,8 +904,10 @@ private void preSplitRegionBeforeMETA(final MasterProcedureEnv env) RegionInfo.parseRegionName(p.getRow()); } } catch (IOException e) { - LOG.error("pid=" + getProcId() + " row key of mutation from coprocessor not parsable as " - + "region name." + "Mutations from coprocessor should only for hbase:meta table."); + LOG.error( + "pid={} row key of mutation from coprocessor not parsable as region name. " + + "Mutations from coprocessor should only be for {} table.", + getProcId(), MetaTableName.getInstance()); throw e; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java index 77b1082d0f03..1f141a8de06e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.Stoppable; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Delete; @@ -80,7 +81,7 @@ public synchronized void chore() { long deletedLastPushedSeqIds = 0; TableName tableName = null; List peerIds = null; - try (Table metaTable = conn.getTable(TableName.META_TABLE_NAME); + try (Table metaTable = conn.getTable(MetaTableName.getInstance()); ResultScanner scanner = metaTable.getScanner( new Scan().addFamily(HConstants.REPLICATION_BARRIER_FAMILY).readAllVersions())) { for (;;) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/http/MetaBrowser.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/http/MetaBrowser.java index 9f5ff857d4d8..f9fa67da83ae 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/http/MetaBrowser.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/http/MetaBrowser.java @@ -33,6 +33,7 @@ import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer; import org.apache.hadoop.hbase.client.AsyncConnection; @@ -156,7 +157,7 @@ public TableName getScanTable() { public Results getResults() { final AsyncTable asyncTable = - connection.getTable(TableName.META_TABLE_NAME); + connection.getTable(MetaTableName.getInstance()); return new Results(asyncTable.getScanner(buildScan())); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java index 0d3ddb43abd4..4e7925e708af 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.ScheduledChore; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -105,7 +106,7 @@ protected boolean initialChore() { scan(); } } catch (IOException e) { - LOG.warn("Failed initial janitorial scan of hbase:meta table", e); + LOG.warn("Failed initial janitorial scan of {} table", MetaTableName.getInstance(), e); return false; } return true; @@ -145,7 +146,7 @@ protected void chore() { + this.services.getServerManager().isClusterShutdown()); } } catch (IOException e) { - LOG.warn("Failed janitorial scan of hbase:meta table", e); + LOG.warn("Failed janitorial scan of {} table", MetaTableName.getInstance(), e); } } @@ -484,7 +485,7 @@ public static void main(String[] args) throws IOException { */ Get g = new Get(Bytes.toBytes("t2,40,1564119846424.1db8c57d64e0733e0f027aaeae7a0bf0.")); g.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); - try (Table t = connection.getTable(TableName.META_TABLE_NAME)) { + try (Table t = connection.getTable(MetaTableName.getInstance())) { Result r = t.get(g); byte[] row = g.getRow(); row[row.length - 2] <<= row[row.length - 2]; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java index 1244d5bf3525..9b30d5198510 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java @@ -32,6 +32,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -203,19 +204,19 @@ private static List createMetaEntries(final MasterServices masterSer .flatMap(List::stream).collect(Collectors.toList()); final List createMetaEntriesFailures = addMetaEntriesResults.stream() .filter(Either::hasRight).map(Either::getRight).collect(Collectors.toList()); - LOG.debug("Added {}/{} entries to hbase:meta", createMetaEntriesSuccesses.size(), - newRegionInfos.size()); + LOG.debug("Added {}/{} entries to {}", createMetaEntriesSuccesses.size(), newRegionInfos.size(), + MetaTableName.getInstance()); if (!createMetaEntriesFailures.isEmpty()) { LOG.warn( - "Failed to create entries in hbase:meta for {}/{} RegionInfo descriptors. First" + "Failed to create entries in {}} for {}/{} RegionInfo descriptors. First" + " failure message included; full list of failures with accompanying stack traces is" + " available at log level DEBUG. message={}", - createMetaEntriesFailures.size(), addMetaEntriesResults.size(), + MetaTableName.getInstance(), createMetaEntriesFailures.size(), addMetaEntriesResults.size(), createMetaEntriesFailures.get(0).getMessage()); if (LOG.isDebugEnabled()) { - createMetaEntriesFailures - .forEach(ioe -> LOG.debug("Attempt to fix region hole in hbase:meta failed.", ioe)); + createMetaEntriesFailures.forEach(ioe -> LOG + .debug("Attempt to fix region hole in {} failed.", MetaTableName.getInstance(), ioe)); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java index c712f1cba672..f370fdc2ffe8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java @@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Result; @@ -137,8 +138,9 @@ private RegionInfo metaTableConsistencyCheck(Result metaTableRow) { if (!Bytes.equals(metaTableRow.getRow(), ri.getRegionName())) { LOG.warn( "INCONSISTENCY: Row name is not equal to serialized info:regioninfo content; " - + "row={} {}; See if RegionInfo is referenced in another hbase:meta row? Delete?", - Bytes.toStringBinary(metaTableRow.getRow()), ri.getRegionNameAsString()); + + "row={} {}; See if RegionInfo is referenced in another {} row? Delete?", + Bytes.toStringBinary(metaTableRow.getRow()), ri.getRegionNameAsString(), + MetaTableName.getInstance()); return null; } // Skip split parent region diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java index e199f6d5971d..960fa3defe6d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java @@ -24,6 +24,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; @@ -394,7 +395,7 @@ private static void cleanRegionsInMeta(final MasterProcedureEnv env, final Table long now = EnvironmentEdgeManager.currentTime(); List deletes = new ArrayList<>(); try ( - Table metaTable = env.getMasterServices().getConnection().getTable(TableName.META_TABLE_NAME); + Table metaTable = env.getMasterServices().getConnection().getTable(MetaTableName.getInstance()); ResultScanner scanner = metaTable.getScanner(tableScan)) { for (;;) { Result result = scanner.next(); @@ -405,7 +406,7 @@ private static void cleanRegionsInMeta(final MasterProcedureEnv env, final Table } if (!deletes.isEmpty()) { LOG.warn("Deleting some vestigial " + deletes.size() + " rows of " + tableName + " from " - + TableName.META_TABLE_NAME); + + MetaTableName.getInstance()); metaTable.delete(deletes); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java index e8999b886afd..8ce33c1574ca 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; @@ -111,7 +112,7 @@ protected Flow executeFromState(final MasterProcedureEnv env, final DisableTable ) { MasterFileSystem fs = env.getMasterFileSystem(); try (BufferedMutator mutator = env.getMasterServices().getConnection() - .getBufferedMutator(TableName.META_TABLE_NAME)) { + .getBufferedMutator(MetaTableName.getInstance())) { for (RegionInfo region : env.getAssignmentManager().getRegionStates() .getRegionsOfTable(tableName)) { long maxSequenceId = WALSplitUtil.getMaxRegionSequenceId( @@ -230,7 +231,7 @@ public TableOperationType getTableOperationType() { */ private boolean prepareDisable(final MasterProcedureEnv env) throws IOException { boolean canTableBeDisabled = true; - if (tableName.equals(TableName.META_TABLE_NAME)) { + if (tableName.equals(MetaTableName.getInstance())) { setFailure("master-disable-table", new ConstraintException("Cannot disable " + this.tableName)); canTableBeDisabled = false; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java index 43d69361c2d2..635dcc0a91b2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.Result; @@ -102,14 +103,14 @@ List getRegionsOnCrashedServer(MasterProcedureEnv env) { MetaTableAccessor.scanMetaForTableRegions(env.getMasterServices().getConnection(), visitor, null); } catch (IOException ioe) { - LOG.warn("Failed scan of hbase:meta for 'Unknown Servers'", ioe); + LOG.warn("Failed scan of {} for 'Unknown Servers'", MetaTableName.getInstance(), ioe); return ris; } // create the server state node too env.getAssignmentManager().getRegionStates().createServer(getServerName()); - LOG.info("Found {} mentions of {} in hbase:meta of OPEN/OPENING Regions: {}", - visitor.getReassigns().size(), getServerName(), visitor.getReassigns().stream() - .map(RegionInfo::getEncodedName).collect(Collectors.joining(","))); + LOG.info("Found {} mentions of {} in {} of OPEN/OPENING Regions: {}", + visitor.getReassigns().size(), getServerName(), MetaTableName.getInstance(), visitor + .getReassigns().stream().map(RegionInfo::getEncodedName).collect(Collectors.joining(","))); return visitor.getReassigns(); } @@ -150,8 +151,8 @@ public boolean visit(Result result) throws IOException { RegionState rs = new RegionState(hrl.getRegion(), state, hrl.getServerName()); if (rs.isClosing()) { // Move region to CLOSED in hbase:meta. - LOG.info("Moving {} from CLOSING to CLOSED in hbase:meta", - hrl.getRegion().getRegionNameAsString()); + LOG.info("Moving {} from CLOSING to CLOSED in {}", + hrl.getRegion().getRegionNameAsString(), MetaTableName.getInstance()); try { MetaTableAccessor.updateRegionState(this.connection, hrl.getRegion(), RegionState.State.CLOSED); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java index 2d54eaf6c58c..e2ab040cd850 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java @@ -30,6 +30,7 @@ import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; @@ -67,7 +68,7 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure proc) { // Meta Queue Lookup Helpers // ============================================================================ private MetaQueue getMetaQueue() { - MetaQueue node = AvlTree.get(metaMap, TableName.META_TABLE_NAME, META_QUEUE_KEY_COMPARATOR); + MetaQueue node = AvlTree.get(metaMap, MetaTableName.getInstance(), META_QUEUE_KEY_COMPARATOR); if (node != null) { return node; } @@ -1079,7 +1080,7 @@ public boolean waitMetaExclusiveLock(Procedure procedure) { return false; } waitProcedure(lock, procedure); - logLockedResource(LockedResourceType.META, TableName.META_TABLE_NAME.getNameAsString()); + logLockedResource(LockedResourceType.META, MetaTableName.getInstance().getNameAsString()); return true; } finally { schedUnlock(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MetaQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MetaQueue.java index 3d313c9ac3ab..5915971bd4c1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MetaQueue.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MetaQueue.java @@ -19,6 +19,7 @@ import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.procedure2.LockStatus; import org.apache.hadoop.hbase.procedure2.Procedure; @@ -33,7 +34,7 @@ class MetaQueue extends Queue { protected MetaQueue(LockStatus lockStatus) { - super(TableName.META_TABLE_NAME, 1, lockStatus); + super(MetaTableName.getInstance(), 1, lockStatus); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java index dc9eac4c879d..30a120143ade 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.BufferedMutator; import org.apache.hadoop.hbase.client.Connection; @@ -64,7 +65,7 @@ private void migrate(MasterProcedureEnv env) throws IOException { try (Table nsTable = conn.getTable(TableName.NAMESPACE_TABLE_NAME); ResultScanner scanner = nsTable.getScanner( new Scan().addFamily(TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES).readAllVersions()); - BufferedMutator mutator = conn.getBufferedMutator(TableName.META_TABLE_NAME)) { + BufferedMutator mutator = conn.getBufferedMutator(MetaTableName.getInstance())) { for (Result result;;) { result = scanner.next(); if (result == null) { @@ -88,7 +89,7 @@ protected Flow executeFromState(MasterProcedureEnv env, MigrateNamespaceTablePro switch (state) { case MIGRATE_NAMESPACE_TABLE_ADD_FAMILY: TableDescriptor metaTableDesc = - env.getMasterServices().getTableDescriptors().get(TableName.META_TABLE_NAME); + env.getMasterServices().getTableDescriptors().get(MetaTableName.getInstance()); if (!metaTableDesc.hasColumnFamily(HConstants.NAMESPACE_FAMILY)) { TableDescriptor newMetaTableDesc = TableDescriptorBuilder.newBuilder(metaTableDesc) .setColumnFamily( diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java index 3450f3059106..6c7dc29e4ddf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.CoprocessorDescriptor; @@ -110,7 +111,7 @@ protected void preflightChecks(MasterProcedureEnv env, Boolean enabled) throws H for (byte[] family : UNDELETABLE_META_COLUMNFAMILIES) { if (!cfs.contains(family)) { throw new HBaseIOException( - "Delete of hbase:meta column family " + Bytes.toString(family)); + "Delete of " + MetaTableName.getInstance() + " column family " + Bytes.toString(family)); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java index 642df36d535f..2afaf1c20b6f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java @@ -26,6 +26,7 @@ import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.locking.LockProcedure; import org.apache.hadoop.hbase.procedure2.LockAndQueue; @@ -174,7 +175,7 @@ List getLocks() { addToLockedResources(lockedResources, regionLocks, Function.identity(), LockedResourceType.REGION); addToLockedResources(lockedResources, peerLocks, Function.identity(), LockedResourceType.PEER); - addToLockedResources(lockedResources, ImmutableMap.of(TableName.META_TABLE_NAME, metaLock), + addToLockedResources(lockedResources, ImmutableMap.of(MetaTableName.getInstance(), metaLock), tn -> tn.getNameAsString(), LockedResourceType.META); addToLockedResources(lockedResources, globalLocks, Function.identity(), LockedResourceType.GLOBAL); @@ -236,7 +237,7 @@ public String toString() { .append("tableLocks", filterUnlocked(tableLocks)) .append("regionLocks", filterUnlocked(regionLocks)) .append("peerLocks", filterUnlocked(peerLocks)) - .append("metaLocks", filterUnlocked(ImmutableMap.of(TableName.META_TABLE_NAME, metaLock))) + .append("metaLocks", filterUnlocked(ImmutableMap.of(MetaTableName.getInstance(), metaLock))) .append("globalLocks", filterUnlocked(globalLocks)).build(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java index ef11e68217a5..57f4e9e4359c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseIOException; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; @@ -231,7 +232,7 @@ public void toStringClassDetails(StringBuilder sb) { } private boolean prepareTruncate() throws IOException { - if (getTableName().equals(TableName.META_TABLE_NAME)) { + if (getTableName().equals(MetaTableName.getInstance())) { throw new IOException("Can't truncate region in catalog tables"); } return true; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java index 41848a58b784..16c3829a1a7a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -243,7 +244,7 @@ public void testFlushedSequenceIdPersistLoad() throws Exception { .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf"))).build(); Table table = TEST_UTIL.createTable(tableDescriptor, null); // flush META region - TEST_UTIL.flush(TableName.META_TABLE_NAME); + TEST_UTIL.flush(MetaTableName.getInstance()); // wait for regionserver report Threads.sleep(msgInterval * 2); // record flush seqid before cluster shutdown diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java index b3fadc7ed27a..4fda3f2b5677 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java @@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -64,7 +65,7 @@ public static void teardownTest() throws Exception { @Test public void tesMetaDescriptorHasSFTConfig() throws Exception { - TableDescriptor descriptor = UTIL.getAdmin().getDescriptor(TableName.META_TABLE_NAME); + TableDescriptor descriptor = UTIL.getAdmin().getDescriptor(MetaTableName.getInstance()); assertEquals(FILE.name(), descriptor.getValue(TRACKER_IMPL)); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java index f640c3084cb8..df1e86fa4916 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; @@ -307,7 +308,7 @@ public void testIncompleteMetaTableReplicaInformation() throws Exception { ADMIN.disableTable(tableName); // now delete one replica info from all the rows // this is to make the meta appear to be only partially updated - Table metaTable = ADMIN.getConnection().getTable(TableName.META_TABLE_NAME); + Table metaTable = ADMIN.getConnection().getTable(MetaTableName.getInstance()); for (byte[] row : tableRows) { Delete deleteOneReplicaLocation = new Delete(row); deleteOneReplicaLocation.addColumns(HConstants.CATALOG_FAMILY, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java index 910692d93c30..14a2f83c21a3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.AsyncTable; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Put; @@ -93,7 +94,7 @@ public void testNewCluster() throws Exception { Connection conn = TEST_UTIL.getConnection(); assertTrue(conn.getAdmin().isMasterInMaintenanceMode()); - try (Table table = conn.getTable(TableName.META_TABLE_NAME); + try (Table table = conn.getTable(MetaTableName.getInstance()); ResultScanner scanner = table.getScanner(new Scan())) { assertNotNull("Could not read meta.", scanner.next()); } @@ -120,7 +121,7 @@ public void testExistingCluster() throws Exception { Connection conn = TEST_UTIL.getConnection(); assertTrue(conn.getAdmin().isMasterInMaintenanceMode()); - try (Table table = conn.getTable(TableName.META_TABLE_NAME); + try (Table table = conn.getTable(MetaTableName.getInstance()); ResultScanner scanner = table.getScanner(HConstants.TABLE_FAMILY); Stream results = StreamSupport.stream(scanner.spliterator(), false)) { assertTrue("Did not find user table records while reading hbase:meta", diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java index e59ef4919126..29b55a9d46a4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; @@ -301,7 +302,7 @@ public void testKillRSWithOpeningRegion2482() throws Exception { */ private static int addToEachStartKey(final int expected) throws IOException { Table t = TEST_UTIL.getConnection().getTable(TABLENAME); - Table meta = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); + Table meta = TEST_UTIL.getConnection().getTable(MetaTableName.getInstance()); int rows = 0; Scan scan = new Scan(); scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java index b6bce31eed9c..12121dbd064b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.RegionLocator; @@ -66,7 +67,7 @@ public static void tearDownAfterClass() throws Exception { @Test public void testStopActiveMaster() throws Exception { try (Connection conn = ConnectionFactory.createConnection(UTIL.getConfiguration()); - RegionLocator locator = conn.getRegionLocator(TableName.META_TABLE_NAME)) { + RegionLocator locator = conn.getRegionLocator(MetaTableName.getInstance())) { ServerName oldMetaServer = locator.getAllRegionLocations().get(0).getServerName(); ServerName oldMaster = UTIL.getMiniHBaseCluster().getMaster().getServerName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java index cdb243b06cdb..91b9325c2831 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; @@ -68,7 +69,7 @@ public class TestMigrateAndMirrorMetaLocations { @BeforeClass public static void setUp() throws Exception { UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), TableName.META_TABLE_NAME, 2); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), MetaTableName.getInstance(), 2); } @AfterClass @@ -143,20 +144,20 @@ public void test() throws Exception { } // wait until all meta regions have been assigned UTIL.waitFor(30000, - () -> UTIL.getMiniHBaseCluster().getRegions(TableName.META_TABLE_NAME).size() == 2); + () -> UTIL.getMiniHBaseCluster().getRegions(MetaTableName.getInstance()).size() == 2); // make sure all the SCPs are finished waitUntilNoSCP(); checkMirrorLocation(2); // increase replica count to 3 - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), TableName.META_TABLE_NAME, 3); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), MetaTableName.getInstance(), 3); checkMirrorLocation(3); byte[] replica2Data = ZKUtil.getData(UTIL.getZooKeeperWatcher(), UTIL.getZooKeeperWatcher().getZNodePaths().getZNodeForReplica(2)); // decrease replica count to 1 - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), TableName.META_TABLE_NAME, 1); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), MetaTableName.getInstance(), 1); checkMirrorLocation(1); // restart the whole cluster, put an extra replica znode on zookeeper, to see if we will remove diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java index 30dd308c28f3..4fa5761a29e9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; @@ -77,7 +78,7 @@ public static final class SuspendProcedure extends Procedure @Override public TableName getTableName() { - return TableName.META_TABLE_NAME; + return MetaTableName.getInstance(); } @Override @@ -154,7 +155,7 @@ public static void tearDown() throws Exception { private void removeNamespaceFamily() throws IOException { FileSystem fs = UTIL.getTestFileSystem(); Path rootDir = CommonFSUtils.getRootDir(UTIL.getConfiguration()); - Path tableDir = CommonFSUtils.getTableDir(rootDir, TableName.META_TABLE_NAME); + Path tableDir = CommonFSUtils.getTableDir(rootDir, MetaTableName.getInstance()); TableDescriptor metaTableDesc = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir); TableDescriptor noNsMetaTableDesc = TableDescriptorBuilder.newBuilder(metaTableDesc) .removeColumnFamily(HConstants.NAMESPACE_FAMILY).build(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java index 42f54e5c8758..7d2c74d7ef3c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -126,7 +127,7 @@ private void validateRecreateClusterWithUserTableEnabled(boolean cleanupWALs, private void restartHBaseCluster(boolean cleanUpWALs, boolean cleanUpZnodes) throws Exception { // flush cache so that everything is on disk - TEST_UTIL.getMiniHBaseCluster().flushcache(TableName.META_TABLE_NAME); + TEST_UTIL.getMiniHBaseCluster().flushcache(MetaTableName.getInstance()); TEST_UTIL.getMiniHBaseCluster().flushcache(); List oldServers = @@ -177,7 +178,7 @@ private void prepareDataBeforeRecreate(HBaseTestingUtil testUtil, TableName tabl put.addColumn(Bytes.toBytes("f"), Bytes.toBytes("c"), Bytes.toBytes("v")); table.put(put); - ensureTableNotColocatedWithSystemTable(tableName, TableName.META_TABLE_NAME); + ensureTableNotColocatedWithSystemTable(tableName, MetaTableName.getInstance()); } private void ensureTableNotColocatedWithSystemTable(TableName userTable, TableName systemTable) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java index 866f74b73191..7f8b4d9ed1a0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; @@ -80,7 +81,7 @@ public void testRestart() throws IOException, InterruptedException { table.put(new Put(row).addColumn(FAMILY, QUALIFIER, row)); } // flush all in memory data - UTIL.flush(TableName.META_TABLE_NAME); + UTIL.flush(MetaTableName.getInstance()); UTIL.flush(NAME); // stop master first, so when stopping region server, we will not schedule a SCP. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java index 8263298a8e4f..a679a36f6bb0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java @@ -22,6 +22,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.AsyncAdmin; import org.apache.hadoop.hbase.client.AsyncConnection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -64,13 +65,13 @@ public static void tearDown() throws Exception { public void test() throws Exception { RegionServerThread rsThread = null; for (RegionServerThread t : UTIL.getMiniHBaseCluster().getRegionServerThreads()) { - if (!t.getRegionServer().getRegions(TableName.META_TABLE_NAME).isEmpty()) { + if (!t.getRegionServer().getRegions(MetaTableName.getInstance()).isEmpty()) { rsThread = t; break; } } HRegionServer rs = rsThread.getRegionServer(); - RegionInfo hri = rs.getRegions(TableName.META_TABLE_NAME).get(0).getRegionInfo(); + RegionInfo hri = rs.getRegions(MetaTableName.getInstance()).get(0).getRegionInfo(); HMaster master = UTIL.getMiniHBaseCluster().getMaster(); ProcedureExecutor executor = master.getMasterProcedureExecutor(); DummyRegionProcedure proc = new DummyRegionProcedure(executor.getEnvironment(), hri); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java index b86493287e52..e2e5a7fbe6c5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.ProcedureTestUtil; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; @@ -150,11 +151,11 @@ public static void setUp() throws Exception { UTIL.createTable(TABLE_NAME, CF); UTIL.getAdmin().balancerSwitch(false, true); HRegionServer srcRs = UTIL.getRSForFirstRegionInTable(TABLE_NAME); - if (!srcRs.getRegions(TableName.META_TABLE_NAME).isEmpty()) { - RegionInfo metaRegion = srcRs.getRegions(TableName.META_TABLE_NAME).get(0).getRegionInfo(); + if (!srcRs.getRegions(MetaTableName.getInstance()).isEmpty()) { + RegionInfo metaRegion = srcRs.getRegions(MetaTableName.getInstance()).get(0).getRegionInfo(); HRegionServer dstRs = UTIL.getOtherRegionServer(srcRs); UTIL.getAdmin().move(metaRegion.getEncodedNameAsBytes(), dstRs.getServerName()); - UTIL.waitFor(30000, () -> !dstRs.getRegions(TableName.META_TABLE_NAME).isEmpty()); + UTIL.waitFor(30000, () -> !dstRs.getRegions(MetaTableName.getInstance()).isEmpty()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java index 2f88f6087dd4..6794ebbbc24d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ProcedureTestUtil; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AsyncAdmin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -94,7 +95,7 @@ public static void setUp() throws Exception { Configuration conf = UTIL.getConfiguration(); conf.setClass(HConstants.MASTER_IMPL, HMasterForTest.class, HMaster.class); UTIL.startMiniCluster(1); - UTIL.waitTableAvailable(TableName.META_TABLE_NAME); + UTIL.waitTableAvailable(MetaTableName.getInstance()); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java index 0e00006251ac..6458e233977e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.client.Admin; @@ -121,7 +122,7 @@ public void testVisitMetaForBadRegionState() throws Exception { put.addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER, Bytes.toBytes("BAD_STATE")); - try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) { + try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { table.put(put); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java index cd73e09af6db..5e187a998f2a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.BalanceRequest; import org.apache.hadoop.hbase.master.HMaster; @@ -167,7 +168,7 @@ public void describeTo(Description description) { @Test public void testFailAndRollback() throws Exception { - HRegionServer rsWithMeta = UTIL.getRSForFirstRegionInTable(TableName.META_TABLE_NAME); + HRegionServer rsWithMeta = UTIL.getRSForFirstRegionInTable(MetaTableName.getInstance()); UTIL.getMiniHBaseCluster().killRegionServer(rsWithMeta.getServerName()); UTIL.waitFor(15000, () -> getSCPForServer(rsWithMeta.getServerName()) != null); ServerCrashProcedure scp = getSCPForServer(rsWithMeta.getServerName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java index 8a7169b09309..0d5c4a7888ac 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; @@ -146,7 +147,7 @@ static void validateReplicaDistribution(Connection connection, TableName tableNa static void validateRegionLocations(Map> tableToServers, TableName productTableName, boolean shouldBeBalanced) { ServerName metaServer = - tableToServers.get(TableName.META_TABLE_NAME).stream().findFirst().orElseThrow(); + tableToServers.get(MetaTableName.getInstance()).stream().findFirst().orElseThrow(); ServerName quotaServer = tableToServers.get(QuotaUtil.QUOTA_TABLE_NAME).stream().findFirst().orElseThrow(); Set productServers = tableToServers.get(productTableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java index dcaebbd84356..f6f8fd1a7eb5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Admin; @@ -161,7 +162,7 @@ public void testRoundRobinAssignment() throws Exception { LoadBalancer balancer = master.getLoadBalancer(); List regions = admin.getRegions(tableName); - regions.addAll(admin.getRegions(TableName.META_TABLE_NAME)); + regions.addAll(admin.getRegions(MetaTableName.getInstance())); List servers = Lists.newArrayList( admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet()); Map> map = balancer.roundRobinAssignment(regions, servers); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java index 80f9728651e3..768854c32778 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -93,7 +94,7 @@ public void testTableIsolation() throws Exception { BalancerConditionalsTestUtil.generateSplits(2 * NUM_SERVERS)); Set tablesToBeSeparated = ImmutableSet. builder() - .add(TableName.META_TABLE_NAME).add(QuotaUtil.QUOTA_TABLE_NAME).add(productTableName).build(); + .add(MetaTableName.getInstance()).add(QuotaUtil.QUOTA_TABLE_NAME).add(productTableName).build(); // Pause the balancer admin.balancerSwitch(false, true); @@ -147,7 +148,7 @@ private static void validateRegionLocations(Map> tabl TableName productTableName, boolean shouldBeBalanced) { // Validate that the region assignments ServerName metaServer = - tableToServers.get(TableName.META_TABLE_NAME).stream().findFirst().orElseThrow(); + tableToServers.get(MetaTableName.getInstance()).stream().findFirst().orElseThrow(); ServerName quotaServer = tableToServers.get(QuotaUtil.QUOTA_TABLE_NAME).stream().findFirst().orElseThrow(); Set productServers = tableToServers.get(productTableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java index 88d1a298aa48..006a63625fc5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Stoppable; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; @@ -94,7 +95,7 @@ public static void tearDownAfterClass() throws Exception { @After public void tearDown() throws IOException { - try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME); + try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance()); ResultScanner scanner = table.getScanner(new Scan().addFamily(HConstants.CATALOG_FAMILY) .addFamily(HConstants.REPLICATION_BARRIER_FAMILY).setFilter(new FirstKeyOnlyFilter()))) { for (;;) { @@ -148,20 +149,20 @@ private void addBarrier(RegionInfo region, long... barriers) throws IOException put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER, put.getTimestamp() - barriers.length + i, Bytes.toBytes(barriers[i])); } - try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) { + try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { table.put(put); } } private void fillCatalogFamily(RegionInfo region) throws IOException { - try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) { + try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { table.put(new Put(region.getRegionName()).addColumn(HConstants.CATALOG_FAMILY, Bytes.toBytes("whatever"), Bytes.toBytes("whatever"))); } } private void clearCatalogFamily(RegionInfo region) throws IOException { - try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) { + try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { table.delete(new Delete(region.getRegionName()).addFamily(HConstants.CATALOG_FAMILY)); } } @@ -281,7 +282,7 @@ public void testDeleteRowForDeletedRegion() throws IOException, ReplicationExcep // No catalog family, then we should remove the whole row clearCatalogFamily(region); cleaner.chore(); - try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) { + try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { assertFalse(table .exists(new Get(region.getRegionName()).addFamily(HConstants.REPLICATION_BARRIER_FAMILY))); } @@ -303,7 +304,7 @@ public void testDeleteRowForDeletedRegionNoPeers() throws IOException { // There are no peers, and no catalog family for this region either, so we should remove the // barriers. And since there is no catalog family, after we delete the barrier family, the whole // row is deleted. - try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) { + try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { assertFalse(table.exists(new Get(region.getRegionName()))); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java index cf118260b401..b610c2750041 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaMockingUtil; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.Waiter; @@ -182,7 +183,7 @@ private PairOfSameType waitOnDaughters(final RegionInfo r) throws IO long start = EnvironmentEdgeManager.currentTime(); PairOfSameType pair = null; try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); - Table metaTable = conn.getTable(TableName.META_TABLE_NAME)) { + Table metaTable = conn.getTable(MetaTableName.getInstance())) { Result result = null; RegionInfo region = null; while ((EnvironmentEdgeManager.currentTime() - start) < 60000) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java index 614385ec04d6..75940a4fd42a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.SortedSet; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -50,21 +51,21 @@ public class TestMetaFixerNoCluster { private static byte[] D = Bytes.toBytes("d"); private static RegionInfo ALL = RegionInfoBuilder.FIRST_META_REGIONINFO; private static RegionInfo _ARI = - RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setEndKey(A).build(); + RegionInfoBuilder.newBuilder(MetaTableName.getInstance()).setEndKey(A).build(); private static RegionInfo _BRI = - RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setEndKey(B).build(); + RegionInfoBuilder.newBuilder(MetaTableName.getInstance()).setEndKey(B).build(); private static RegionInfo ABRI = - RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setStartKey(A).setEndKey(B).build(); + RegionInfoBuilder.newBuilder(MetaTableName.getInstance()).setStartKey(A).setEndKey(B).build(); private static RegionInfo ACRI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(TableName.META_TABLE_NAME).setStartKey(A).setEndKey(C).build(); + .newBuilder(MetaTableName.getInstance()).setStartKey(A).setEndKey(C).build(); private static RegionInfo CDRI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(TableName.META_TABLE_NAME).setStartKey(C).setEndKey(D).build(); + .newBuilder(MetaTableName.getInstance()).setStartKey(C).setEndKey(D).build(); private static RegionInfo ADRI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(TableName.META_TABLE_NAME).setStartKey(A).setEndKey(D).build(); + .newBuilder(MetaTableName.getInstance()).setStartKey(A).setEndKey(D).build(); private static RegionInfo D_RI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(TableName.META_TABLE_NAME).setStartKey(D).build(); + .newBuilder(MetaTableName.getInstance()).setStartKey(D).build(); private static RegionInfo C_RI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(TableName.META_TABLE_NAME).setStartKey(C).build(); + .newBuilder(MetaTableName.getInstance()).setStartKey(C).build(); @Test public void testGetRegionInfoWithLargestEndKey() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java index e931716e77ed..67537a96def7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java @@ -59,6 +59,7 @@ import org.apache.hadoop.hbase.RegionMetrics; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.Size; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.client.RegionInfo; @@ -104,7 +105,7 @@ public void before() { @Test public void testNoNormalizationForMetaTable() { - TableName testTable = TableName.META_TABLE_NAME; + TableName testTable = MetaTableName.getInstance(); TableDescriptor testMetaTd = TableDescriptorBuilder.newBuilder(testTable).build(); List RegionInfo = new ArrayList<>(); Map regionSizes = new HashMap<>(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java index a878af785783..39c34794bb2f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.client.RegionInfo; @@ -262,7 +263,7 @@ public String toString() { private static class PrimaryNotMetaRegionSelector extends RegionSelector { @Override boolean regionFilter(final RegionInfo info) { - return !Objects.equals(TableName.META_TABLE_NAME, info.getTable()) + return !Objects.equals(MetaTableName.getInstance(), info.getTable()) && Objects.equals(RegionInfo.DEFAULT_REPLICA_ID, info.getReplicaId()); } @@ -278,7 +279,7 @@ Exception regionFilterFailure() { private static class ReplicaNonMetaRegionSelector extends RegionSelector { @Override boolean regionFilter(RegionInfo info) { - return !Objects.equals(TableName.META_TABLE_NAME, info.getTable()) + return !Objects.equals(MetaTableName.getInstance(), info.getTable()) && !Objects.equals(RegionInfo.DEFAULT_REPLICA_ID, info.getReplicaId()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java index d2f04c674c97..c5e99471a7f6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.client.AsyncAdmin; @@ -147,7 +148,7 @@ public static void tearDown() throws Exception { @Test public void test() throws Exception { RegionServerThread rsWithMetaThread = UTIL.getMiniHBaseCluster().getRegionServerThreads() - .stream().filter(t -> !t.getRegionServer().getRegions(TableName.META_TABLE_NAME).isEmpty()) + .stream().filter(t -> !t.getRegionServer().getRegions(MetaTableName.getInstance()).isEmpty()) .findAny().get(); HRegionServer rsNoMeta = UTIL.getOtherRegionServer(rsWithMetaThread.getRegionServer()); FAIL = true; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java index 386356124f5b..cf58f01b7843 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java @@ -22,6 +22,7 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; @@ -122,7 +123,7 @@ public static class MetaTableProcedure extends Procedure @Override public TableName getTableName() { - return TableName.META_TABLE_NAME; + return MetaTableName.getInstance(); } @Override From 91f3fb75a5995da68fc4ceca83b4e0282a123a97 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Mon, 15 Dec 2025 18:59:21 -0500 Subject: [PATCH 04/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: org.apache.hadoop.hbase.regionserver --- .../hbase/regionserver/RSRpcServices.java | 3 +- .../TestCompactionInDeadRegionServer.java | 3 +- .../regionserver/TestDefaultMemStore.java | 3 +- .../TestEndToEndSplitTransaction.java | 3 +- .../TestGetClosestAtOrBefore.java | 3 +- .../TestReadAndWriteRegionInfoFile.java | 5 +-- .../hbase/regionserver/TestRegionInfo.java | 35 ++++++++++--------- .../regionserver/TestRegionReplicas.java | 3 +- .../TestRegionServerCrashDisableWAL.java | 3 +- .../TestRegionServerNoMaster.java | 5 +-- .../TestRegionServerRejectDuringAbort.java | 3 +- .../TestShutdownWhileWALBroken.java | 3 +- .../wal/AbstractTestLogRolling.java | 3 +- .../regionserver/wal/TestLogRollAbort.java | 3 +- .../regionserver/wal/TestLogRolling.java | 3 +- .../wal/TestLogRollingNoCluster.java | 7 ++-- 16 files changed, 52 insertions(+), 36 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index ba838e2f16ca..46b81988a9e7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -62,6 +62,7 @@ import org.apache.hadoop.hbase.RegionTooBusyException; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.client.Append; @@ -1928,7 +1929,7 @@ public OpenRegionResponse openRegion(final RpcController controller, tableName = ProtobufUtil.toTableName(ri.getTableName()); } } - if (!TableName.META_TABLE_NAME.equals(tableName)) { + if (!MetaTableName.getInstance().equals(tableName)) { throw new ServiceException(ie); } // We are assigning meta, wait a little for regionserver to finish initialization. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java index 64454ab268fa..58507a063d2d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.YouAreDeadException; import org.apache.hadoop.hbase.client.Get; @@ -131,7 +132,7 @@ public void test() throws Exception { HRegionServer regionSvr = UTIL.getRSForFirstRegionInTable(TABLE_NAME); HRegion region = regionSvr.getRegions(TABLE_NAME).get(0); String regName = region.getRegionInfo().getEncodedName(); - List metaRegs = regionSvr.getRegions(TableName.META_TABLE_NAME); + List metaRegs = regionSvr.getRegions(MetaTableName.getInstance()); if (metaRegs != null && !metaRegs.isEmpty()) { LOG.info("meta is on the same server: " + regionSvr); // when region is on same server as hbase:meta, reassigning meta would abort the server diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java index af3902c9aa1e..63dbfda6f2d1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java @@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; @@ -1002,7 +1003,7 @@ public void testShouldFlushMeta() throws Exception { TableDescriptors tds = new FSTableDescriptors(conf); FSTableDescriptors.tryUpdateMetaTableDescriptor(conf); HRegion meta = HRegion.createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, testDir, conf, - tds.get(TableName.META_TABLE_NAME), wFactory.getWAL(RegionInfoBuilder.FIRST_META_REGIONINFO)); + tds.get(MetaTableName.getInstance()), wFactory.getWAL(RegionInfoBuilder.FIRST_META_REGIONINFO)); // parameterized tests add [#] suffix get rid of [ and ]. TableDescriptor desc = TableDescriptorBuilder .newBuilder(TableName.valueOf(name.getMethodName().replaceAll("[\\[\\]]", "_"))) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java index 897152f8b6dd..7916101c425e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -429,7 +430,7 @@ public static void blockUntilRegionSplit(Configuration conf, long timeout, log("blocking until region is split:" + Bytes.toStringBinary(regionName)); RegionInfo daughterA = null, daughterB = null; try (Connection conn = ConnectionFactory.createConnection(conf); - Table metaTable = conn.getTable(TableName.META_TABLE_NAME)) { + Table metaTable = conn.getTable(MetaTableName.getInstance())) { Result result = null; RegionInfo region = null; while ((EnvironmentEdgeManager.currentTime() - start) < timeout) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java index a435b9d9b239..8acd75c5e675 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; @@ -92,7 +93,7 @@ public void testUsingMetaAndBinary() throws IOException { // Up flush size else we bind up when we use default catalog flush of 16k. TableDescriptors tds = new FSTableDescriptors(UTIL.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(UTIL.getConfiguration()); - TableDescriptor td = tds.get(TableName.META_TABLE_NAME); + TableDescriptor td = tds.get(MetaTableName.getInstance()); td = TableDescriptorBuilder.newBuilder(td).setMemStoreFlushSize(64 * 1024 * 1024).build(); HRegion mr = HBaseTestingUtil.createRegionAndWAL(RegionInfoBuilder.FIRST_META_REGIONINFO, rootdir, conf, td); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java index 2869be090f42..f3f824ee5135 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HBaseCommonTestingUtil; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -73,12 +74,12 @@ public void testReadAndWriteRegionInfoFile() throws IOException, InterruptedExce FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(FS, ROOT_DIR); FSTableDescriptors.tryUpdateAndGetMetaTableDescriptor(CONF, FS, ROOT_DIR); HRegion r = HBaseTestingUtil.createRegionAndWAL(ri, ROOT_DIR, CONF, - fsTableDescriptors.get(TableName.META_TABLE_NAME)); + fsTableDescriptors.get(MetaTableName.getInstance())); // Get modtime on the file. long modtime = getModTime(r); HBaseTestingUtil.closeRegionAndWAL(r); Thread.sleep(1001); - r = HRegion.openHRegion(ROOT_DIR, ri, fsTableDescriptors.get(TableName.META_TABLE_NAME), null, + r = HRegion.openHRegion(ROOT_DIR, ri, fsTableDescriptors.get(MetaTableName.getInstance()), null, CONF); // Ensure the file is not written for a second time. long modtime2 = getModTime(r); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java index 60fe39ecc77f..3a31d1f7fd2d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.RegionInfoDisplay; @@ -71,7 +72,7 @@ public class TestRegionInfo { public void testIsStart() { assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isFirst()); org.apache.hadoop.hbase.client.RegionInfo ri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(TableName.META_TABLE_NAME).setStartKey(Bytes.toBytes("not_start")).build(); + .newBuilder(MetaTableName.getInstance()).setStartKey(Bytes.toBytes("not_start")).build(); assertFalse(ri.isFirst()); } @@ -79,7 +80,7 @@ public void testIsStart() { public void testIsEnd() { assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isFirst()); org.apache.hadoop.hbase.client.RegionInfo ri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(TableName.META_TABLE_NAME).setEndKey(Bytes.toBytes("not_end")).build(); + .newBuilder(MetaTableName.getInstance()).setEndKey(Bytes.toBytes("not_end")).build(); assertFalse(ri.isLast()); } @@ -87,9 +88,9 @@ public void testIsEnd() { public void testIsNext() { byte[] bytes = Bytes.toBytes("row"); org.apache.hadoop.hbase.client.RegionInfo ri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(TableName.META_TABLE_NAME).setEndKey(bytes).build(); + .newBuilder(MetaTableName.getInstance()).setEndKey(bytes).build(); org.apache.hadoop.hbase.client.RegionInfo ri2 = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(TableName.META_TABLE_NAME).setStartKey(bytes).build(); + .newBuilder(MetaTableName.getInstance()).setStartKey(bytes).build(); assertFalse(ri.isNext(RegionInfoBuilder.FIRST_META_REGIONINFO)); assertTrue(ri.isNext(ri2)); } @@ -102,18 +103,18 @@ public void testIsOverlap() { byte[] d = Bytes.toBytes("d"); org.apache.hadoop.hbase.client.RegionInfo all = RegionInfoBuilder.FIRST_META_REGIONINFO; org.apache.hadoop.hbase.client.RegionInfo ari = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(TableName.META_TABLE_NAME).setEndKey(a).build(); + .newBuilder(MetaTableName.getInstance()).setEndKey(a).build(); org.apache.hadoop.hbase.client.RegionInfo abri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(MetaTableName.getInstance()) .setStartKey(a).setEndKey(b).build(); org.apache.hadoop.hbase.client.RegionInfo adri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(MetaTableName.getInstance()) .setStartKey(a).setEndKey(d).build(); org.apache.hadoop.hbase.client.RegionInfo cdri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(MetaTableName.getInstance()) .setStartKey(c).setEndKey(d).build(); org.apache.hadoop.hbase.client.RegionInfo dri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(TableName.META_TABLE_NAME).setStartKey(d).build(); + .newBuilder(MetaTableName.getInstance()).setStartKey(d).build(); assertTrue(all.isOverlap(all)); assertTrue(all.isOverlap(abri)); assertFalse(abri.isOverlap(cdri)); @@ -140,17 +141,17 @@ public void testIsOverlaps() { byte[] e = Bytes.toBytes("e"); byte[] f = Bytes.toBytes("f"); org.apache.hadoop.hbase.client.RegionInfo ari = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(TableName.META_TABLE_NAME).setEndKey(a).build(); + .newBuilder(MetaTableName.getInstance()).setEndKey(a).build(); org.apache.hadoop.hbase.client.RegionInfo abri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(MetaTableName.getInstance()) .setStartKey(a).setEndKey(b).build(); org.apache.hadoop.hbase.client.RegionInfo eri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(TableName.META_TABLE_NAME).setEndKey(e).build(); + .newBuilder(MetaTableName.getInstance()).setEndKey(e).build(); org.apache.hadoop.hbase.client.RegionInfo cdri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(MetaTableName.getInstance()) .setStartKey(c).setEndKey(d).build(); org.apache.hadoop.hbase.client.RegionInfo efri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(MetaTableName.getInstance()) .setStartKey(e).setEndKey(f).build(); assertFalse(ari.isOverlap(abri)); assertTrue(abri.isOverlap(eri)); @@ -175,12 +176,12 @@ public void testReadAndWriteHRegionInfoFile() throws IOException, InterruptedExc FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(htu.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(htu.getConfiguration()); HRegion r = HBaseTestingUtil.createRegionAndWAL(hri, basedir, htu.getConfiguration(), - fsTableDescriptors.get(TableName.META_TABLE_NAME)); + fsTableDescriptors.get(MetaTableName.getInstance())); // Get modtime on the file. long modtime = getModTime(r); HBaseTestingUtil.closeRegionAndWAL(r); Thread.sleep(1001); - r = HRegion.openHRegion(basedir, hri, fsTableDescriptors.get(TableName.META_TABLE_NAME), null, + r = HRegion.openHRegion(basedir, hri, fsTableDescriptors.get(MetaTableName.getInstance()), null, htu.getConfiguration()); // Ensure the file is not written for a second time. long modtime2 = getModTime(r); @@ -254,7 +255,7 @@ public void testContainsRange() { @Test public void testContainsRangeForMetaTable() { TableDescriptor tableDesc = - TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).build(); + TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).build(); RegionInfo hri = RegionInfoBuilder.newBuilder(tableDesc.getTableName()).build(); byte[] startRow = HConstants.EMPTY_START_ROW; byte[] row1 = Bytes.toBytes("a,a,0"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java index 68c6b6434c4f..2ef40ffeb257 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TestMetaTableAccessor; import org.apache.hadoop.hbase.client.Consistency; import org.apache.hadoop.hbase.client.Get; @@ -147,7 +148,7 @@ public void testRegionReplicaUpdatesMetaLocation() throws Exception { openRegion(HTU, getRS(), hriSecondary); Table meta = null; try { - meta = HTU.getConnection().getTable(TableName.META_TABLE_NAME); + meta = HTU.getConnection().getTable(MetaTableName.getInstance()); TestMetaTableAccessor.assertMetaLocation(meta, hriPrimary.getRegionName(), getRS().getServerName(), -1, 1, false); } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java index 3fad6e16bf76..f5b3611fad29 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; @@ -65,7 +66,7 @@ public static void setUp() throws Exception { UTIL.createTable(TABLE_NAME, CF); UTIL.waitTableAvailable(TABLE_NAME); HRegionServer rs = UTIL.getRSForFirstRegionInTable(TABLE_NAME); - if (!rs.getRegions(TableName.META_TABLE_NAME).isEmpty()) { + if (!rs.getRegions(MetaTableName.getInstance()).isEmpty()) { HRegionServer rs1 = UTIL.getOtherRegionServer(rs); UTIL.moveRegionAndWait( UTIL.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0).getRegionInfo(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java index 522b0ea884b3..96a8db5cabe6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionLocator; @@ -97,11 +98,11 @@ public static void stopMasterAndCacheMetaLocation(HBaseTestingUtil HTU) // cache meta location, so we will not go to master to lookup meta region location for (JVMClusterUtil.RegionServerThread t : HTU.getMiniHBaseCluster().getRegionServerThreads()) { try (RegionLocator locator = - t.getRegionServer().getConnection().getRegionLocator(TableName.META_TABLE_NAME)) { + t.getRegionServer().getConnection().getRegionLocator(MetaTableName.getInstance())) { locator.getAllRegionLocations(); } } - try (RegionLocator locator = HTU.getConnection().getRegionLocator(TableName.META_TABLE_NAME)) { + try (RegionLocator locator = HTU.getConnection().getRegionLocator(MetaTableName.getInstance())) { locator.getAllRegionLocations(); } // Stop master diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java index 61da536310a8..177b2c7d7787 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -135,7 +136,7 @@ public void testRejectRequestsOnAbort() throws Exception { .getRegionServerThreads()) { HRegionServer regionServer = regionServerThread.getRegionServer(); if ( - regionServer.getRegions(TableName.META_TABLE_NAME).isEmpty() + regionServer.getRegions(MetaTableName.getInstance()).isEmpty() && !regionServer.getRegions(TABLE_NAME).isEmpty() ) { serverWithoutMeta = regionServer; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java index 0bc7deccc121..37d9d5954d90 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.YouAreDeadException; import org.apache.hadoop.hbase.client.Table; @@ -135,7 +136,7 @@ public void test() throws Exception { RegionServerThread rst1 = UTIL.getMiniHBaseCluster().getRegionServerThreads().get(1); HRegionServer liveRS; RegionServerThread toKillRSThread; - if (rst1.getRegionServer().getRegions(TableName.META_TABLE_NAME).isEmpty()) { + if (rst1.getRegionServer().getRegions(MetaTableName.getInstance()).isEmpty()) { liveRS = rst0.getRegionServer(); toKillRSThread = rst1; } else { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java index 2a5aec458828..563f5f03dfcb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -338,7 +339,7 @@ void validateData(Table table, int rownum) throws IOException { public void testCompactionRecordDoesntBlockRolling() throws Exception { // When the hbase:meta table can be opened, the region servers are running - try (Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); + try (Table t = TEST_UTIL.getConnection().getTable(MetaTableName.getInstance()); Table table = createTestTable(getName())) { server = TEST_UTIL.getRSForFirstRegionInTable(table.getName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java index 3c3dbe1ead9e..511b58c9afd5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; @@ -162,7 +163,7 @@ public void testRSAbortWithUnflushedEdits() throws Exception { LOG.info("Starting testRSAbortWithUnflushedEdits()"); // When the hbase:meta table can be opened, the region servers are running - TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME).close(); + TEST_UTIL.getConnection().getTable(MetaTableName.getInstance()).close(); // Create the test table and open it TableName tableName = TableName.valueOf(this.getClass().getSimpleName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java index 43477f21f7f8..ad72bea40c89 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; @@ -322,7 +323,7 @@ public void testLogRollOnPipelineRestart() throws Exception { fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) > 1); LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS())); // When the hbase:meta table can be opened, the region servers are running - Table t = TEST_UTIL.getConnection().getTable(TableName.META_TABLE_NAME); + Table t = TEST_UTIL.getConnection().getTable(MetaTableName.getInstance()); try { this.server = cluster.getRegionServer(0); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java index dd4fe77c8a38..ac62e0395325 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; @@ -95,7 +96,7 @@ public void testContendedLogRolling() throws Exception { CommonFSUtils.setRootDir(conf, dir); FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(TEST_UTIL.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration()); - TableDescriptor metaTableDescriptor = fsTableDescriptors.get(TableName.META_TABLE_NAME); + TableDescriptor metaTableDescriptor = fsTableDescriptors.get(MetaTableName.getInstance()); conf.set(FSHLogProvider.WRITER_IMPL, HighLatencySyncWriter.class.getName()); final WALFactory wals = new WALFactory(conf, TestLogRollingNoCluster.class.getName()); final WAL wal = wals.getWAL(null); @@ -159,7 +160,7 @@ public void run() { try { TableDescriptors tds = new FSTableDescriptors(TEST_UTIL.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration()); - TableDescriptor htd = tds.get(TableName.META_TABLE_NAME); + TableDescriptor htd = tds.get(MetaTableName.getInstance()); for (int i = 0; i < this.count; i++) { long now = EnvironmentEdgeManager.currentTime(); // Roll every ten edits @@ -176,7 +177,7 @@ public void run() { scopes.put(fam, 0); } final long txid = wal.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), - TableName.META_TABLE_NAME, now, mvcc, scopes), edit); + MetaTableName.getInstance(), now, mvcc, scopes), edit); Threads.sleep(ThreadLocalRandom.current().nextInt(5)); wal.sync(txid); } From d7806f61c3d05b61b71b726c2fab14111e6477e3 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Mon, 15 Dec 2025 19:10:34 -0500 Subject: [PATCH 05/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: rest --- .../backup/impl/IncrementalBackupManager.java | 3 +- .../favored/FavoredNodeAssignmentHelper.java | 3 +- .../SnapshotOfRegionAssignmentFromMeta.java | 9 +- ...rgeClusterBalancingMetaTableIsolation.java | 5 +- ...gTableIsolationAndReplicaDistribution.java | 5 +- .../hbase/security/token/ClientTokenUtil.java | 5 +- .../hbase/shaded/protobuf/ProtobufUtil.java | 3 +- .../hbase/client/TestCompactFromClient.java | 4 +- .../apache/hadoop/hbase/MetaTableName.java | 56 +++++------- .../org/apache/hadoop/hbase/TableName.java | 4 +- .../master/MetricsMasterFileSystemSource.java | 4 +- .../hbase/mapreduce/TestImportExport.java | 3 +- .../hadoop/hbase/rest/TestStatusResource.java | 3 +- .../model/TestStorageClusterStatusModel.java | 5 +- .../hbase/coprocessor/MetaTableMetrics.java | 5 +- .../ReplicationBarrierFamilyFormat.java | 5 +- .../hbase/security/access/AccessChecker.java | 3 +- .../hadoop/hbase/tool/BulkLoadHFilesTool.java | 13 +-- .../hadoop/hbase/util/FSTableDescriptors.java | 11 +-- .../org/apache/hadoop/hbase/util/FSUtils.java | 1 + .../apache/hadoop/hbase/util/HBaseFsck.java | 88 +++++++++++-------- .../hadoop/hbase/util/HBaseFsckRepair.java | 3 +- .../apache/hadoop/hbase/util/RegionMover.java | 9 +- .../wal/BoundedRecoveredHFilesOutputSink.java | 6 +- .../hbase-webapps/master/catalogTables.jsp | 1 + .../resources/hbase-webapps/master/table.jsp | 7 +- .../hadoop/hbase/HBaseClusterInterface.java | 2 +- .../hadoop/hbase/TestHBaseMetaEdit.java | 18 ++-- .../TestMetaUpdatesGoToPriorityQueue.java | 2 +- .../apache/hadoop/hbase/TestNamespace.java | 2 +- .../hbase/TestServerInternalsTracing.java | 2 +- .../hadoop/hbase/http/TestInfoServersACL.java | 5 +- .../TestReplicationWALEntryFilters.java | 3 +- .../TestMetaRegionReplicaReplication.java | 27 +++--- .../regionserver/TestReplicationSource.java | 3 +- .../TestSerialReplicationChecker.java | 7 +- .../rsgroup/TestRSGroupsCPHookCalled.java | 3 +- .../hbase/rsgroup/TestRSGroupsKillRS.java | 5 +- .../security/access/TestRpcAccessChecks.java | 5 +- .../token/TestGenerateDelegationToken.java | 3 +- .../snapshot/TestRegionSnapshotTask.java | 3 +- .../hadoop/hbase/util/BaseTestHBaseFsck.java | 9 +- .../hbase/util/TestFSTableDescriptors.java | 7 +- ...TestHBaseFsckCleanReplicationBarriers.java | 3 +- .../hadoop/hbase/util/TestHBaseFsckMOB.java | 3 +- .../hadoop/hbase/util/TestRegionMover1.java | 3 +- .../TestRegionMoverWithRSGroupEnable.java | 3 +- .../apache/hadoop/hbase/wal/TestWALSplit.java | 5 +- hbase-shell/src/main/ruby/hbase/table.rb | 2 +- .../thrift/ThriftHBaseServiceHandler.java | 5 +- .../hbase/zookeeper/MetaTableLocator.java | 19 ++-- .../apache/hadoop/hbase/zookeeper/ZKDump.java | 3 +- 52 files changed, 230 insertions(+), 186 deletions(-) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java index 20884edf836e..76599704d87e 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore; @@ -169,7 +170,7 @@ private List getLogFilesForNewBackup(Map olderTimestamps, LOG.debug("currentLogFile: " + log.getPath().toString()); if (AbstractFSWALProvider.isMetaFile(log.getPath())) { if (LOG.isDebugEnabled()) { - LOG.debug("Skip hbase:meta log file: " + log.getPath().getName()); + LOG.debug("Skip {} log file: {}", MetaTableName.getInstance(), log.getPath().getName()); } continue; } diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java index 6c021bf622a5..8a36477913c9 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; @@ -131,7 +132,7 @@ public static void updateMetaWithFavoredNodesInfo( puts.add(put); } } - try (Table table = connection.getTable(TableName.META_TABLE_NAME)) { + try (Table table = connection.getTable(MetaTableName.getInstance())) { table.put(puts); } LOG.info("Added " + puts.size() + " region favored nodes in META"); diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java index 02c18c73bfb5..783ad2cbbc23 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.Result; @@ -170,9 +171,10 @@ private void processMetaRecord(Result result) throws IOException { * Initialize the region assignment snapshot by scanning the hbase:meta table */ public void initialize() throws IOException { - LOG.info("Start to scan the hbase:meta for the current region assignment " + "snappshot"); + LOG.info("Start to scan {} for the current region assignment snapshot", + MetaTableName.getInstance()); // Scan hbase:meta to pick up user regions - try (Table metaTable = connection.getTable(TableName.META_TABLE_NAME); + try (Table metaTable = connection.getTable(MetaTableName.getInstance()); ResultScanner scanner = metaTable.getScanner(HConstants.CATALOG_FAMILY)) { for (;;) { Result result = scanner.next(); @@ -187,7 +189,8 @@ public void initialize() throws IOException { } } } - LOG.info("Finished to scan the hbase:meta for the current region assignment" + "snapshot"); + LOG.info("Finished scanning {} for the current region assignment snapshot", + MetaTableName.getInstance()); } private void addRegion(RegionInfo regionInfo) { diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java index e4d2e168e2b6..c62fbea144a4 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.master.balancer.BalancerTestBase.MockMapping; @@ -70,7 +71,7 @@ public static void setup() { // Create regions List allRegions = new ArrayList<>(); for (int i = 0; i < NUM_REGIONS; i++) { - TableName tableName = i < 3 ? TableName.META_TABLE_NAME : NON_META_TABLE_NAME; + TableName tableName = i < 3 ? MetaTableName.getInstance() : NON_META_TABLE_NAME; byte[] startKey = new byte[1]; startKey[0] = (byte) i; byte[] endKey = new byte[1]; @@ -98,7 +99,7 @@ public void testMetaTableIsolation() { } private boolean isMetaTableIsolated(BalancerClusterState cluster) { - return isTableIsolated(cluster, TableName.META_TABLE_NAME, "Meta"); + return isTableIsolated(cluster, MetaTableName.getInstance(), "Meta"); } } diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java index cccc0c55c2c5..efb5ccabe115 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.master.balancer.BalancerTestBase.MockMapping; @@ -74,7 +75,7 @@ public static void setup() { for (int i = 0; i < NUM_REGIONS; i++) { TableName tableName; if (i < 1) { - tableName = TableName.META_TABLE_NAME; + tableName = MetaTableName.getInstance(); } else if (i < 10) { tableName = SYSTEM_TABLE_NAME; } else { @@ -119,7 +120,7 @@ public void testTableIsolationAndReplicaDistribution() { * Validates whether all meta table regions are isolated. */ private boolean isMetaTableIsolated(BalancerClusterState cluster) { - return isTableIsolated(cluster, TableName.META_TABLE_NAME, "Meta"); + return isTableIsolated(cluster, MetaTableName.getInstance(), "Meta"); } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java index 40ff0373c36c..a4dabfb2465e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java @@ -23,6 +23,7 @@ import java.util.concurrent.CompletableFuture; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.AsyncConnection; import org.apache.hadoop.hbase.client.AsyncTable; import org.apache.hadoop.hbase.client.Connection; @@ -73,7 +74,7 @@ private static void injectFault() throws ServiceException { future.completeExceptionally(ProtobufUtil.handleRemoteException(injectedException)); return future; } - AsyncTable table = conn.getTable(TableName.META_TABLE_NAME); + AsyncTable table = conn.getTable(MetaTableName.getInstance()); table. coprocessorService( AuthenticationProtos.AuthenticationService::newStub, @@ -102,7 +103,7 @@ static Token obtainToken(Connection conn) throws try { injectFault(); - meta = conn.getTable(TableName.META_TABLE_NAME); + meta = conn.getTable(MetaTableName.getInstance()); CoprocessorRpcChannel rpcChannel = meta.coprocessorService(HConstants.EMPTY_START_ROW); AuthenticationProtos.AuthenticationService.BlockingInterface service = AuthenticationProtos.AuthenticationService.newBlockingStub(rpcChannel); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java index 60175137ad2c..cd341a911ce0 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java @@ -70,6 +70,7 @@ import org.apache.hadoop.hbase.ServerTask; import org.apache.hadoop.hbase.ServerTaskBuilder; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.BalanceRequest; import org.apache.hadoop.hbase.client.BalanceResponse; @@ -3325,7 +3326,7 @@ public static String toLockJson(List lockedRes long regionId = proto.getRegionId(); int defaultReplicaId = org.apache.hadoop.hbase.client.RegionInfo.DEFAULT_REPLICA_ID; int replicaId = proto.hasReplicaId() ? proto.getReplicaId() : defaultReplicaId; - if (tableName.equals(TableName.META_TABLE_NAME) && replicaId == defaultReplicaId) { + if (tableName.equals(MetaTableName.getInstance()) && replicaId == defaultReplicaId) { return RegionInfoBuilder.FIRST_META_REGIONINFO; } byte[] startKey = null; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java index 40617d78950a..c8e6b2158ce1 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.client; -import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -34,6 +33,7 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.hbase.ClientMetaTableAccessor; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -94,7 +94,7 @@ public void testCompactTableWithNullLocations() throws Exception { mockedMeta.when(() -> ClientMetaTableAccessor.getTableHRegionLocations(any(AsyncTable.class), any(TableName.class))).thenReturn(nullLocationsFuture); AsyncTable metaTable = mock(AsyncTable.class); - when(connection.getTable(META_TABLE_NAME)).thenReturn(metaTable); + when(connection.getTable(MetaTableName.getInstance())).thenReturn(metaTable); HashedWheelTimer hashedWheelTimer = mock(HashedWheelTimer.class); AsyncAdminBuilderBase asyncAdminBuilderBase = mock(AsyncAdminBuilderBase.class); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaTableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaTableName.java index f4afd8bbe1c7..90bb4ccc0630 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaTableName.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaTableName.java @@ -1,33 +1,38 @@ +/* + * 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 + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.hadoop.hbase; import org.apache.hadoop.conf.Configuration; -import org.apache.hbase.thirdparty.com.google.common.base.Strings; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * Singleton class for managing the META_TABLE_NAME instance. - * This allows the meta table name to be overridden for testing using reflection. - */ + @InterfaceAudience.Public public class MetaTableName { private static final Logger LOG = LoggerFactory.getLogger(MetaTableName.class); - - /** - * The singleton instance of the meta table name. - * This field can be overridden for testing using reflection. - */ private static volatile TableName instance; private MetaTableName() { - // Private constructor to prevent instantiation } /** * Get the singleton instance of the meta table name. - * Initializes lazily using the default configuration if not already set. - * * @return The meta table name instance */ public static TableName getInstance() { @@ -35,7 +40,7 @@ public static TableName getInstance() { synchronized (MetaTableName.class) { if (instance == null) { instance = initializeHbaseMetaTableName(HBaseConfiguration.create()); - LOG.info("Meta table name initialized: {}", instance); + LOG.info("Meta table name initialized: {}", instance.getName()); } } } @@ -49,26 +54,9 @@ public static TableName getInstance() { * @return The initialized meta table name */ private static TableName initializeHbaseMetaTableName(Configuration conf) { - String suffix_val = conf.get(HConstants.HBASE_META_TABLE_SUFFIX, - HConstants.HBASE_META_TABLE_SUFFIX_DEFAULT_VALUE); - LOG.info("Meta table suffix value: {}", suffix_val); - if (Strings.isNullOrEmpty(suffix_val)) { - return TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); - } else { - return TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta_" + suffix_val); - } - } - - /** - * Get the instance field for reflection-based testing. - * This method is package-private to allow test classes to access the field. - * - * @return The Field object for the instance field - */ - static java.lang.reflect.Field getInstanceField() throws NoSuchFieldException { - java.lang.reflect.Field field = MetaTableName.class.getDeclaredField("instance"); - field.setAccessible(true); - return field; + TableName metaTableName = TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); + LOG.info("Meta table suffix value: {}", metaTableName); + return metaTableName; } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java index 6fdfc1edf001..0b798c505819 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java @@ -84,7 +84,7 @@ public final class TableName implements Comparable { * future version. */ @Deprecated - public static TableName META_TABLE_NAME; + public static TableName META_TABLE_NAME = TableName.valueOf("hbase:meta"); /** * The Namespace table's name. @@ -306,7 +306,7 @@ private TableName(ByteBuffer namespace, ByteBuffer qualifier) throws IllegalArgu } if (qualifierAsString.equals(OLD_META_STR)) { throw new IllegalArgumentException( - OLD_META_STR + " no longer exists. The table has been " + "renamed to " + META_TABLE_NAME); + OLD_META_STR + " no longer exists. The table has been " + "renamed to " + MetaTableName.getInstance()); } if (Bytes.equals(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME, namespace)) { diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java index 53ed8a25ed0e..9bc4a90c8cf5 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hbase.master; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.metrics.BaseSource; import org.apache.yetus.audience.InterfaceAudience; @@ -49,7 +51,7 @@ public interface MetricsMasterFileSystemSource extends BaseSource { String SPLIT_SIZE_NAME = "hlogSplitSize"; String META_SPLIT_TIME_DESC = "Time it takes to finish splitMetaLog()"; - String META_SPLIT_SIZE_DESC = "Size of hbase:meta WAL files being split"; + String META_SPLIT_SIZE_DESC = "Size of " + MetaTableName.getInstance() + " WAL files being split"; String SPLIT_TIME_DESC = "Time it takes to finish WAL.splitLog()"; String SPLIT_SIZE_DESC = "Size of WAL files being split"; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java index d4ccac901436..09ce98ddc172 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java @@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.client.ClientInternalHelper; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -256,7 +257,7 @@ public void testSimpleCase() throws Throwable { @Test public void testMetaExport() throws Throwable { String[] args = - new String[] { TableName.META_TABLE_NAME.getNameAsString(), FQ_OUTPUT_DIR, "1", "0", "0" }; + new String[] { MetaTableName.getInstance().getNameAsString(), FQ_OUTPUT_DIR, "1", "0", "0" }; assertTrue(runExport(args)); } diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java index a115fd17af3f..b493be5c9f06 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.rest.client.Client; import org.apache.hadoop.hbase.rest.client.Cluster; @@ -55,7 +56,7 @@ public class TestStatusResource { private static final Logger LOG = LoggerFactory.getLogger(TestStatusResource.class); - private static final byte[] META_REGION_NAME = Bytes.toBytes(TableName.META_TABLE_NAME + ",,1"); + private static final byte[] META_REGION_NAME = Bytes.toBytes(MetaTableName.getInstance() + ",,1"); private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); private static final HBaseRESTTestingUtility REST_TEST_UTIL = new HBaseRESTTestingUtility(); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java index 8310232890dd..c3761c397c8a 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java @@ -24,6 +24,7 @@ import java.util.Iterator; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -89,7 +90,7 @@ protected StorageClusterStatusModel buildTestModel() { model.addLiveNode("test1", 1245219839331L, 128, 1024).addRegion(Bytes.toBytes("hbase:root,,0"), 1, 1, 0, 0, 0, 1, 1, 2, 1, 1, 1, 1, 1); model.addLiveNode("test2", 1245239331198L, 512, 1024).addRegion( - Bytes.toBytes(TableName.META_TABLE_NAME + ",,1246000043724"), 1, 1, 0, 0, 0, 1, 1, 2, 1, 1, 1, + Bytes.toBytes(MetaTableName.getInstance() + ",,1246000043724"), 1, 1, 0, 0, 0, 1, 1, 2, 1, 1, 1, 1, 1); return model; } @@ -128,7 +129,7 @@ protected void checkModel(StorageClusterStatusModel model) { assertEquals(1024, node.getMaxHeapSizeMB()); regions = node.getRegions().iterator(); region = regions.next(); - assertEquals(Bytes.toString(region.getName()), TableName.META_TABLE_NAME + ",,1246000043724"); + assertEquals(Bytes.toString(region.getName()), MetaTableName.getInstance() + ",,1246000043724"); assertEquals(1, region.getStores()); assertEquals(1, region.getStorefiles()); assertEquals(0, region.getStorefileSizeMB()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java index 3cac1f319dae..fc2d164e922d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; @@ -133,7 +134,7 @@ private String getRegionIdFromOp(Row op) { } private boolean isMetaTableOp(ObserverContext e) { - return TableName.META_TABLE_NAME.equals(e.getEnvironment().getRegionInfo().getTable()); + return MetaTableName.getInstance().equals(e.getEnvironment().getRegionInfo().getTable()); } private void clientMetricRegisterAndMark() { @@ -268,7 +269,7 @@ public void start(CoprocessorEnvironment env) throws IOException { env instanceof RegionCoprocessorEnvironment && ((RegionCoprocessorEnvironment) env).getRegionInfo().getTable() != null && ((RegionCoprocessorEnvironment) env).getRegionInfo().getTable() - .equals(TableName.META_TABLE_NAME) + .equals(MetaTableName.getInstance()) ) { RegionCoprocessorEnvironment regionCoprocessorEnv = (RegionCoprocessorEnvironment) env; registry = regionCoprocessorEnv.getMetricRegistryForRegionServer(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java index 8bf32baada22..f82ac9bd42ec 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -192,7 +193,7 @@ public static ReplicationBarrierResult getReplicationBarrierResult(Connection co .addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER) .addFamily(HConstants.REPLICATION_BARRIER_FAMILY).readAllVersions().setReversed(true) .setCaching(10); - try (Table table = conn.getTable(TableName.META_TABLE_NAME); + try (Table table = conn.getTable(MetaTableName.getInstance()); ResultScanner scanner = table.getScanner(scan)) { for (Result result;;) { result = scanner.next(); @@ -215,7 +216,7 @@ public static ReplicationBarrierResult getReplicationBarrierResult(Connection co public static long[] getReplicationBarriers(Connection conn, byte[] regionName) throws IOException { - try (Table table = conn.getTable(TableName.META_TABLE_NAME)) { + try (Table table = conn.getTable(MetaTableName.getInstance())) { Result result = table.get(new Get(regionName) .addColumn(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER) .readAllVersions()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java index 57d156ab1c2e..637785c2b62e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.security.AccessDeniedException; @@ -544,7 +545,7 @@ public AuthResult permissionGranted(String request, User user, Action permReques TableName tableName, Map> families) { // 1. All users need read access to hbase:meta table. // this is a very common operation, so deal with it quickly. - if (TableName.META_TABLE_NAME.equals(tableName)) { + if (MetaTableName.getInstance().equals(tableName)) { if (permRequest == Action.READ) { return AuthResult.allow(request, "All users allowed", user, permRequest, tableName, families); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java index 4d6f57e22edc..6c206557027e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java @@ -66,6 +66,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.AsyncAdmin; import org.apache.hadoop.hbase.client.AsyncClusterConnection; @@ -660,21 +661,21 @@ private int getRegionIndex(List> startEndKeys, byte[] key) private void checkRegionIndexValid(int idx, List> startEndKeys, TableName tableName) throws IOException { if (idx < 0) { - throw new IOException("The first region info for table " + tableName - + " can't be found in hbase:meta.Please use hbck tool to fix it first."); + throw new IOException("The first region info for table " + tableName + " can't be found in " + + MetaTableName.getInstance() + ". Please use hbck tool to fix it" + " first."); } else if ( (idx == startEndKeys.size() - 1) && !Bytes.equals(startEndKeys.get(idx).getSecond(), HConstants.EMPTY_BYTE_ARRAY) ) { - throw new IOException("The last region info for table " + tableName - + " can't be found in hbase:meta.Please use hbck tool to fix it first."); + throw new IOException("The last region info for table " + tableName + " can't be found in " + + MetaTableName.getInstance() + ". Please use hbck tool to fix it" + " first."); } else if ( idx + 1 < startEndKeys.size() && !(Bytes.compareTo(startEndKeys.get(idx).getSecond(), startEndKeys.get(idx + 1).getFirst()) == 0) ) { throw new IOException("The endkey of one region for table " + tableName - + " is not equal to the startkey of the next region in hbase:meta." - + "Please use hbck tool to fix it first."); + + " is not equal to the startkey of the next region in " + MetaTableName.getInstance() + "." + + " Please use hbck tool to fix it first."); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java index 75bf721ef41e..28cb4bcb69e5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.CoprocessorDescriptorBuilder; @@ -147,20 +148,20 @@ public static TableDescriptor tryUpdateAndGetMetaTableDescriptor(Configuration c FileSystem fs, Path rootdir) throws IOException { // see if we already have meta descriptor on fs. Write one if not. Optional> opt = getTableDescriptorFromFs(fs, - CommonFSUtils.getTableDir(rootdir, TableName.META_TABLE_NAME), false); + CommonFSUtils.getTableDir(rootdir, MetaTableName.getInstance()), false); if (opt.isPresent()) { return opt.get().getSecond(); } TableDescriptorBuilder builder = createMetaTableDescriptorBuilder(conf); TableDescriptor td = StoreFileTrackerFactory.updateWithTrackerConfigs(conf, builder.build()); - LOG.info("Creating new hbase:meta table descriptor {}", td); + LOG.info("Creating new {} table descriptor {}", MetaTableName.getInstance(), td); TableName tableName = td.getTableName(); Path tableDir = CommonFSUtils.getTableDir(rootdir, tableName); Path p = writeTableDescriptor(fs, td, tableDir, null); if (p == null) { - throw new IOException("Failed update hbase:meta table descriptor"); + throw new IOException("Failed update " + MetaTableName.getInstance() + " table descriptor"); } - LOG.info("Updated hbase:meta table descriptor to {}", p); + LOG.info("Updated {} table descriptor to {}", MetaTableName.getInstance(), p); return td; } @@ -198,7 +199,7 @@ private static TableDescriptorBuilder createMetaTableDescriptorBuilder(final Con // TODO We used to set CacheDataInL1 for META table. When we have BucketCache in file mode, now // the META table data goes to File mode BC only. Test how that affect the system. If too much, // we have to rethink about adding back the setCacheDataInL1 for META table CFs. - return TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME) + return TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()) .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(HConstants.CATALOG_FAMILY) .setMaxVersions( conf.getInt(HConstants.HBASE_META_VERSIONS, HConstants.DEFAULT_HBASE_META_VERSIONS)) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java index 3b446826b775..0b502c3c5ca6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java @@ -69,6 +69,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index c3eafa7c11d1..9bfbef4a2536 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -84,6 +84,7 @@ import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; @@ -540,7 +541,7 @@ public void run() { connection = ConnectionFactory.createConnection(getConf()); admin = connection.getAdmin(); - meta = connection.getTable(TableName.META_TABLE_NAME); + meta = connection.getTable(MetaTableName.getInstance()); status = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS, Option.DEAD_SERVERS, Option.MASTER, Option.BACKUP_MASTERS, Option.REGIONS_IN_TRANSITION, Option.HBASE_VERSION)); } @@ -660,17 +661,19 @@ public int onlineConsistencyRepair() throws IOException, KeeperException, Interr reportUnknownServers(); // Check if hbase:meta is found only once and in the right place if (!checkMetaRegion()) { - String errorMsg = "hbase:meta table is not consistent. "; + String errorMsg = MetaTableName.getInstance() + " table is not consistent. "; if (shouldFixAssignments()) { - errorMsg += "HBCK will try fixing it. Rerun once hbase:meta is back to consistent state."; + errorMsg += "HBCK will try fixing it. Rerun once " + MetaTableName.getInstance() + " is back " + + "to consistent state."; } else { - errorMsg += "Run HBCK with proper fix options to fix hbase:meta inconsistency."; + errorMsg += "Run HBCK with proper fix options to fix " + MetaTableName.getInstance() + + " inconsistency."; } errors.reportError(errorMsg + " Exiting..."); return -2; } // Not going with further consistency check for tables when hbase:meta itself is not consistent. - LOG.info("Loading regionsinfo from the hbase:meta table"); + LOG.info("Loading regionsinfo from the {} table", MetaTableName.getInstance()); boolean success = loadMetaEntries(); if (!success) return -1; @@ -1219,7 +1222,7 @@ private boolean sidelineFile(FileSystem fs, Path hbaseRoot, Path path) throws IO * TODO -- need to add tests for this. */ private void reportEmptyMetaCells() { - errors.print("Number of empty REGIONINFO_QUALIFIER rows in hbase:meta: " + errors.print("Number of empty REGIONINFO_QUALIFIER rows in " + MetaTableName.getInstance() + ": " + emptyRegionInfoQualifiers.size()); if (details) { for (Result r : emptyRegionInfoQualifiers) { @@ -1371,7 +1374,7 @@ private boolean fabricateTableInfo(FSTableDescriptors fstd, TableName tableName, */ public void fixEmptyMetaCells() throws IOException { if (shouldFixEmptyMetaCells() && !emptyRegionInfoQualifiers.isEmpty()) { - LOG.info("Trying to fix empty REGIONINFO_QUALIFIER hbase:meta rows."); + LOG.info("Trying to fix empty REGIONINFO_QUALIFIER {} rows.", MetaTableName.getInstance()); for (Result region : emptyRegionInfoQualifiers) { deleteMetaRegion(region.getRow()); errors.getErrorList().remove(ERROR_CODE.EMPTY_META_CELL); @@ -1574,8 +1577,8 @@ private void loadTableStates() throws IOException { // Add hbase:meta so this tool keeps working. In hbase2, meta is always enabled though it // has no entry in the table states. HBCK doesn't work right w/ hbase2 but just do this in // meantime. - this.tableStates.put(TableName.META_TABLE_NAME, - new TableState(TableName.META_TABLE_NAME, TableState.State.ENABLED)); + this.tableStates.put(MetaTableName.getInstance(), + new TableState(MetaTableName.getInstance(), TableState.State.ENABLED)); } /** @@ -1604,7 +1607,7 @@ public void loadHdfsRegionDirs() throws IOException, InterruptedException { TableName tableName = CommonFSUtils.getTableName(path); if ( (!checkMetaOnly && isTableIncluded(tableName)) - || tableName.equals(TableName.META_TABLE_NAME) + || tableName.equals(MetaTableName.getInstance()) ) { tableDirs.add(fs.getFileStatus(path)); } @@ -1649,7 +1652,7 @@ public void loadHdfsRegionDirs() throws IOException, InterruptedException { */ private boolean recordMetaRegion() throws IOException { List locs; - try (RegionLocator locator = connection.getRegionLocator(TableName.META_TABLE_NAME)) { + try (RegionLocator locator = connection.getRegionLocator(MetaTableName.getInstance())) { locs = locator.getRegionLocations(HConstants.EMPTY_START_ROW, true); } if (locs == null || locs.isEmpty()) { @@ -2019,9 +2022,11 @@ void closeRegion(HbckRegionInfo hi) throws IOException, InterruptedException { } RegionInfo hri = h.getRegion(); if (hri == null) { - LOG.warn("Unable to close region " + hi.getRegionNameAsString() - + " because hbase:meta had invalid or missing " + HConstants.CATALOG_FAMILY_STR + ":" - + Bytes.toString(HConstants.REGIONINFO_QUALIFIER) + " qualifier value."); + LOG.warn( + "Unable to close region " + hi.getRegionNameAsString() + + " because {} had invalid or missing " + HConstants.CATALOG_FAMILY_STR + ":" + + Bytes.toString(HConstants.REGIONINFO_QUALIFIER) + " qualifier value.", + MetaTableName.getInstance()); continue; } // close the region -- close files and remove assignment @@ -2140,8 +2145,9 @@ else if (!inMeta && !inHdfs && !isDeployed) { assert false : "Entry for region with no data"; } else if (!inMeta && !inHdfs && isDeployed) { errors.reportError(ERROR_CODE.NOT_IN_META_HDFS, - "Region " + descriptiveName + ", key=" + key + ", not on HDFS or in hbase:meta but " - + "deployed on " + Joiner.on(", ").join(hbi.getDeployedOn())); + "Region " + descriptiveName + ", key=" + key + ", not on HDFS or in " + + MetaTableName.getInstance() + " but " + "deployed on " + + Joiner.on(", ").join(hbi.getDeployedOn())); if (shouldFixAssignments()) { undeployRegions(hbi); } @@ -2155,8 +2161,9 @@ else if (!inMeta && !inHdfs && !isDeployed) { + " got merge recently, its file(s) will be cleaned by CatalogJanitor later"); return; } - errors.reportError(ERROR_CODE.NOT_IN_META_OR_DEPLOYED, "Region " + descriptiveName - + " on HDFS, but not listed in hbase:meta " + "or deployed on any region server"); + errors.reportError(ERROR_CODE.NOT_IN_META_OR_DEPLOYED, + "Region " + descriptiveName + " on HDFS, but not listed in " + MetaTableName.getInstance() + + " or deployed on any region server"); // restore region consistency of an adopted orphan if (shouldFixMeta()) { if (!hbi.isHdfsRegioninfoPresent()) { @@ -2196,7 +2203,7 @@ else if (!inMeta && !inHdfs && !isDeployed) { } } } - LOG.info("Patching hbase:meta with .regioninfo: " + hbi.getHdfsHRI()); + LOG.info("Patching {} with .regioninfo: " + hbi.getHdfsHRI(), MetaTableName.getInstance()); int numReplicas = admin.getDescriptor(hbi.getTableName()).getRegionReplication(); HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(), admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet(), @@ -2224,7 +2231,8 @@ else if (!inMeta && !inHdfs && !isDeployed) { return; } - LOG.info("Patching hbase:meta with with .regioninfo: " + hbi.getHdfsHRI()); + LOG.info("Patching {} with with .regioninfo: " + hbi.getHdfsHRI(), + MetaTableName.getInstance()); int numReplicas = admin.getDescriptor(hbi.getTableName()).getRegionReplication(); HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(), admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet(), @@ -2301,9 +2309,9 @@ else if (!inMeta && !inHdfs && !isDeployed) { } } else if (inMeta && inHdfs && isMultiplyDeployed) { errors.reportError(ERROR_CODE.MULTI_DEPLOYED, - "Region " + descriptiveName + " is listed in hbase:meta on region server " - + hbi.getMetaEntry().regionServer + " but is multiply assigned to region servers " - + Joiner.on(", ").join(hbi.getDeployedOn())); + "Region " + descriptiveName + " is listed in " + MetaTableName.getInstance() + + " on region server " + hbi.getMetaEntry().regionServer + " but is multiply assigned" + + " to region servers " + Joiner.on(", ").join(hbi.getDeployedOn())); // If we are trying to fix the errors if (shouldFixAssignments()) { errors.print("Trying to fix assignment error..."); @@ -2313,8 +2321,8 @@ else if (!inMeta && !inHdfs && !isDeployed) { } } else if (inMeta && inHdfs && isDeployed && !deploymentMatchesMeta) { errors.reportError(ERROR_CODE.SERVER_DOES_NOT_MATCH_META, - "Region " + descriptiveName + " listed in hbase:meta on region server " - + hbi.getMetaEntry().regionServer + " but found on region server " + "Region " + descriptiveName + " listed in " + MetaTableName.getInstance() + + " on region server " + hbi.getMetaEntry().regionServer + " but found on region server " + hbi.getDeployedOn().get(0)); // If we are trying to fix the errors if (shouldFixAssignments()) { @@ -2599,7 +2607,7 @@ boolean checkMetaRegion() throws IOException, KeeperException, InterruptedExcept metaRegions.put(value.getReplicaId(), value); } } - int metaReplication = admin.getDescriptor(TableName.META_TABLE_NAME).getRegionReplication(); + int metaReplication = admin.getDescriptor(MetaTableName.getInstance()).getRegionReplication(); boolean noProblem = true; // There will be always entries in regionInfoMap corresponding to hbase:meta & its replicas // Check the deployed servers. It should be exactly one server for each replica. @@ -2614,11 +2622,12 @@ boolean checkMetaRegion() throws IOException, KeeperException, InterruptedExcept if (servers.isEmpty()) { assignMetaReplica(i); } else if (servers.size() > 1) { - errors.reportError(ERROR_CODE.MULTI_META_REGION, "hbase:meta, replicaId " - + metaHbckRegionInfo.getReplicaId() + " is found on more than one region."); + errors.reportError(ERROR_CODE.MULTI_META_REGION, + MetaTableName.getInstance() + ", replicaId " + metaHbckRegionInfo.getReplicaId() + + " is found on more than one region."); if (shouldFixAssignments()) { - errors.print("Trying to fix a problem with hbase:meta, replicaId " - + metaHbckRegionInfo.getReplicaId() + ".."); + errors.print("Trying to fix a problem with " + MetaTableName.getInstance() + + ", replicaId " + metaHbckRegionInfo.getReplicaId() + ".."); setShouldRerun(); // try fix it (treat is a dupe assignment) HBaseFsckRepair.fixMultiAssignment(connection, @@ -2631,11 +2640,11 @@ boolean checkMetaRegion() throws IOException, KeeperException, InterruptedExcept for (Map.Entry entry : metaRegions.entrySet()) { noProblem = false; errors.reportError(ERROR_CODE.SHOULD_NOT_BE_DEPLOYED, - "hbase:meta replicas are deployed in excess. Configured " + metaReplication + ", deployed " - + metaRegions.size()); + MetaTableName.getInstance() + " replicas are deployed in excess. Configured " + + metaReplication + ", deployed " + metaRegions.size()); if (shouldFixAssignments()) { - errors.print( - "Trying to undeploy excess replica, replicaId: " + entry.getKey() + " of hbase:meta.."); + errors.print("Trying to undeploy excess replica, replicaId: " + entry.getKey() + " of " + + MetaTableName.getInstance() + ".."); setShouldRerun(); unassignMetaReplica(entry.getValue()); } @@ -2655,9 +2664,9 @@ private void unassignMetaReplica(HbckRegionInfo hi) private void assignMetaReplica(int replicaId) throws IOException, KeeperException, InterruptedException { errors.reportError(ERROR_CODE.NO_META_REGION, - "hbase:meta, replicaId " + replicaId + " is not found on any region."); + MetaTableName.getInstance() + ", replicaId " + replicaId + " is not found on any region."); if (shouldFixAssignments()) { - errors.print("Trying to fix a problem with hbase:meta.."); + errors.print("Trying to fix a problem with " + MetaTableName.getInstance() + ".."); setShouldRerun(); // try to fix it (treat it as unassigned region) RegionInfo h = RegionReplicaUtil @@ -2693,7 +2702,7 @@ public boolean visit(Result result) throws IOException { if (rl == null) { emptyRegionInfoQualifiers.add(result); errors.reportError(ERROR_CODE.EMPTY_META_CELL, - "Empty REGIONINFO_QUALIFIER found in hbase:meta"); + "Empty REGIONINFO_QUALIFIER found in " + MetaTableName.getInstance()); return true; } ServerName sn = null; @@ -2703,7 +2712,7 @@ public boolean visit(Result result) throws IOException { ) { emptyRegionInfoQualifiers.add(result); errors.reportError(ERROR_CODE.EMPTY_META_CELL, - "Empty REGIONINFO_QUALIFIER found in hbase:meta"); + "Empty REGIONINFO_QUALIFIER found in " + MetaTableName.getInstance()); return true; } RegionInfo hri = rl.getRegionLocation(RegionInfo.DEFAULT_REPLICA_ID).getRegion(); @@ -2731,7 +2740,8 @@ public boolean visit(Result result) throws IOException { } else if (previous.getMetaEntry() == null) { previous.setMetaEntry(m); } else { - throw new IOException("Two entries in hbase:meta are same " + previous); + throw new IOException( + "Two entries in " + MetaTableName.getInstance() + " are same " + previous); } } List mergeParents = CatalogFamilyFormat.getMergeRegions(result.rawCells()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java index 6ead66c16d9e..8550220f2a5f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.AsyncClusterConnection; @@ -149,7 +150,7 @@ public static void closeRegionSilentlyAndWait(Connection connection, ServerName public static void fixMetaHoleOnlineAndAddReplicas(Configuration conf, RegionInfo hri, Collection servers, int numReplicas) throws IOException { Connection conn = ConnectionFactory.createConnection(conf); - Table meta = conn.getTable(TableName.META_TABLE_NAME); + Table meta = conn.getTable(MetaTableName.getInstance()); Put put = MetaTableAccessor.makePutFromRegionInfo(hri); if (numReplicas > 1) { Random rand = ThreadLocalRandom.current(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java index c1f98edd75ab..dda03bad7525 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java @@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; @@ -586,13 +587,13 @@ private void unloadRegions(ServerName server, List regionServers, // For isolating hbase:meta, it should move explicitly in Ack mode, // hence the forceMoveRegionByAck = true. if (!metaSeverName.equals(server)) { - LOG.info("Region of hbase:meta " + metaRegionInfo.getEncodedName() + " is on server " - + metaSeverName + " moving to " + server); + LOG.info("Region of {} {} is on server {} moving to {}", MetaTableName.getInstance(), + metaRegionInfo.getEncodedName(), metaSeverName, server); submitRegionMovesWhileUnloading(metaSeverName, Collections.singletonList(server), movedRegions, Collections.singletonList(metaRegionInfo), true); } else { - LOG.info("Region of hbase:meta " + metaRegionInfo.getEncodedName() + " already exists" - + " on server : " + server); + LOG.info("Region of {} {} already exists on server: {}", MetaTableName.getInstance(), + metaRegionInfo.getEncodedName(), server); } isolateRegionInfoList.add(RegionInfoBuilder.FIRST_META_REGIONINFO); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java index b8f095eb03df..c14cf914b497 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java @@ -17,8 +17,7 @@ */ package org.apache.hadoop.hbase.wal; -import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; - +import org.apache.hadoop.hbase.MetaTableName; import java.io.IOException; import java.io.InterruptedIOException; import java.util.HashMap; @@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.MetaCellComparator; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; @@ -78,7 +78,7 @@ public BoundedRecoveredHFilesOutputSink(WALSplitter walSplitter, void append(RegionEntryBuffer buffer) throws IOException { Map> familyCells = new HashMap<>(); Map familySeqIds = new HashMap<>(); - boolean isMetaTable = buffer.tableName.equals(META_TABLE_NAME); + boolean isMetaTable = buffer.tableName.equals(MetaTableName.getInstance()); // First iterate all Cells to find which column families are present and to stamp Cell with // sequence id. for (WAL.Entry entry : buffer.entryBuffer) { diff --git a/hbase-server/src/main/resources/hbase-webapps/master/catalogTables.jsp b/hbase-server/src/main/resources/hbase-webapps/master/catalogTables.jsp index 1f1aad61ba70..f157f327103e 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/catalogTables.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/catalogTables.jsp @@ -20,6 +20,7 @@ <%@ page contentType="text/html;charset=UTF-8" import="java.util.*" + import="org.apache.hadoop.hbase.MetaTableName" import="org.apache.hadoop.hbase.NamespaceDescriptor" import="org.apache.hadoop.hbase.TableName" import="org.apache.hadoop.hbase.master.HMaster" diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp index d88d968e199f..d838cd3c50bd 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp @@ -41,6 +41,7 @@ import="org.apache.hadoop.hbase.ServerMetrics" import="org.apache.hadoop.hbase.ServerName" import="org.apache.hadoop.hbase.Size" + import="org.apache.hadoop.hbase.MetaTableName" import="org.apache.hadoop.hbase.TableName" import="org.apache.hadoop.hbase.client.AsyncAdmin" import="org.apache.hadoop.hbase.client.AsyncConnection" @@ -196,7 +197,7 @@ boolean showFragmentation = conf.getBoolean("hbase.master.ui.fragmentation.enabled", false); boolean readOnly = !InfoServer.canUserModifyUI(request, getServletContext(), conf); int numMetaReplicas = - master.getTableDescriptors().get(TableName.META_TABLE_NAME).getRegionReplication(); + master.getTableDescriptors().get(MetaTableName.getInstance()).getRegionReplication(); Map frags = null; if (showFragmentation) { frags = FSUtils.getTableFragmentation(master); @@ -317,7 +318,7 @@
<% //Meta table. - if(fqtn.equals(TableName.META_TABLE_NAME.getNameAsString())) { %> + if(fqtn.equals(MetaTableName.getInstance().getNameAsString())) { %>

Table Regions

@@ -653,7 +654,7 @@
- +
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseClusterInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseClusterInterface.java index f56fc57dd2d9..018d4e1182f1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseClusterInterface.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseClusterInterface.java @@ -388,7 +388,7 @@ public boolean restoreClusterMetrics(ClusterMetrics desiredStatus) throws IOExce * Get the ServerName of region server serving the first hbase:meta region */ public ServerName getServerHoldingMeta() throws IOException { - return getServerHoldingRegion(TableName.META_TABLE_NAME, + return getServerHoldingRegion(MetaTableName.getInstance(), RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java index 2a438461b4e7..b90831e32c9f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java @@ -68,11 +68,11 @@ public void after() throws Exception { // make sure that with every possible way, we get the same meta table descriptor. private TableDescriptor getMetaDescriptor() throws TableNotFoundException, IOException { Admin admin = UTIL.getAdmin(); - TableDescriptor get = admin.getDescriptor(TableName.META_TABLE_NAME); + TableDescriptor get = admin.getDescriptor(MetaTableName.getInstance()); TableDescriptor list = admin.listTableDescriptors(true).stream().filter(td -> td.isMetaTable()).findAny().get(); TableDescriptor listByName = - admin.listTableDescriptors(Collections.singletonList(TableName.META_TABLE_NAME)).get(0); + admin.listTableDescriptors(Collections.singletonList(MetaTableName.getInstance())).get(0); TableDescriptor listByNs = admin.listTableDescriptorsByNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME).stream() .filter(td -> td.isMetaTable()).findAny().get(); @@ -91,7 +91,7 @@ private TableDescriptor getMetaDescriptor() throws TableNotFoundException, IOExc @Test public void testEditMeta() throws IOException { Admin admin = UTIL.getAdmin(); - admin.tableExists(TableName.META_TABLE_NAME); + admin.tableExists(MetaTableName.getInstance()); TableDescriptor originalDescriptor = getMetaDescriptor(); ColumnFamilyDescriptor cfd = originalDescriptor.getColumnFamily(HConstants.CATALOG_FAMILY); int oldVersions = cfd.getMaxVersions(); @@ -100,11 +100,11 @@ public void testEditMeta() throws IOException { .setConfiguration(ColumnFamilyDescriptorBuilder.DATA_BLOCK_ENCODING, DataBlockEncoding.ROW_INDEX_V1.toString()) .build(); - admin.modifyColumnFamily(TableName.META_TABLE_NAME, cfd); + admin.modifyColumnFamily(MetaTableName.getInstance(), cfd); byte[] extraColumnFamilyName = Bytes.toBytes("xtra"); ColumnFamilyDescriptor newCfd = ColumnFamilyDescriptorBuilder.newBuilder(extraColumnFamilyName).build(); - admin.addColumnFamily(TableName.META_TABLE_NAME, newCfd); + admin.addColumnFamily(MetaTableName.getInstance(), newCfd); TableDescriptor descriptor = getMetaDescriptor(); // Assert new max versions is == old versions plus 1. assertEquals(oldVersions + 1, @@ -126,11 +126,11 @@ public void testEditMeta() throws IOException { assertEquals(encoding, DataBlockEncoding.ROW_INDEX_V1.toString()); assertTrue(r.getStore(extraColumnFamilyName) != null); // Assert we can't drop critical hbase:meta column family but we can drop any other. - admin.deleteColumnFamily(TableName.META_TABLE_NAME, newCfd.getName()); + admin.deleteColumnFamily(MetaTableName.getInstance(), newCfd.getName()); descriptor = getMetaDescriptor(); assertTrue(descriptor.getColumnFamily(newCfd.getName()) == null); try { - admin.deleteColumnFamily(TableName.META_TABLE_NAME, HConstants.CATALOG_FAMILY); + admin.deleteColumnFamily(MetaTableName.getInstance(), HConstants.CATALOG_FAMILY); fail("Should not reach here"); } catch (HBaseIOException hioe) { assertTrue(hioe.getMessage().contains("Delete of hbase:meta")); @@ -144,7 +144,7 @@ public void testEditMeta() throws IOException { @Test public void testAlterMetaWithReadOnly() throws IOException { Admin admin = UTIL.getAdmin(); - TableDescriptor origMetaTableDesc = admin.getDescriptor(TableName.META_TABLE_NAME); + TableDescriptor origMetaTableDesc = admin.getDescriptor(MetaTableName.getInstance()); assertFalse(origMetaTableDesc.isReadOnly()); TableDescriptor newTD = TableDescriptorBuilder.newBuilder(origMetaTableDesc).setReadOnly(true).build(); @@ -152,7 +152,7 @@ public void testAlterMetaWithReadOnly() throws IOException { admin.modifyTable(newTD); fail("Meta table can't be set as read only"); } catch (Exception e) { - assertFalse(admin.getDescriptor(TableName.META_TABLE_NAME).isReadOnly()); + assertFalse(admin.getDescriptor(MetaTableName.getInstance()).isReadOnly()); } // Create a table to check region assignment & meta operation diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java index 91e040f1db13..e1c76567be7f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java @@ -87,7 +87,7 @@ private void multiMutate(byte[] row, List mutations) throws IOExceptio } } MutateRowsRequest request = builder.build(); - AsyncTable table = UTIL.getAsyncConnection().getTable(TableName.META_TABLE_NAME); + AsyncTable table = UTIL.getAsyncConnection().getTable(MetaTableName.getInstance()); CompletableFuture future = table. coprocessorService(MultiRowMutationService::newStub, (stub, controller, done) -> stub.mutateRows(controller, request, done), row); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java index 0005a2becde7..67110aca8181 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java @@ -117,7 +117,7 @@ public void verifyReservedNS() throws IOException { assertEquals(2, admin.listNamespaceDescriptors().length); // verify existence of system tables - Set systemTables = Sets.newHashSet(TableName.META_TABLE_NAME); + Set systemTables = Sets.newHashSet(MetaTableName.getInstance()); List descs = admin.listTableDescriptorsByNamespace( Bytes.toBytes(NamespaceDescriptor.SYSTEM_NAMESPACE.getName())); assertEquals(systemTables.size(), descs.size()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerInternalsTracing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerInternalsTracing.java index 69e6e4ac83df..16ede9b98c60 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerInternalsTracing.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerInternalsTracing.java @@ -77,7 +77,7 @@ public Setup(final Supplier testingUtilSupplier) { @Override protected void before() throws Throwable { final HBaseTestingUtil testingUtil = testingUtilSupplier.get(); - testingUtil.waitTableAvailable(TableName.META_TABLE_NAME); + testingUtil.waitTableAvailable(MetaTableName.getInstance()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java index b613e2a824c6..4d7cbdd2680a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.LocalHBaseCluster; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.security.HBaseKerberosUtils; @@ -200,7 +201,7 @@ public void testTableActionsAvailableForAdmins() throws Exception { @Override public Void run() throws Exception { // Check the expected content is present in the http response - Pair pair = getTablePage(TableName.META_TABLE_NAME); + Pair pair = getTablePage(MetaTableName.getInstance()); assertEquals(HttpURLConnection.HTTP_OK, pair.getFirst().intValue()); assertTrue("expected=" + expectedAuthorizedContent + ", content=" + pair.getSecond(), pair.getSecond().contains(expectedAuthorizedContent)); @@ -213,7 +214,7 @@ public Void run() throws Exception { nonAdmin.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - Pair pair = getTablePage(TableName.META_TABLE_NAME); + Pair pair = getTablePage(MetaTableName.getInstance()); assertEquals(HttpURLConnection.HTTP_OK, pair.getFirst().intValue()); assertFalse( "should not find=" + expectedAuthorizedContent + ", content=" + pair.getSecond(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java index 897166a94000..9e79200a0aa5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.security.access.PermissionStorage; import org.apache.hadoop.hbase.security.visibility.VisibilityConstants; @@ -72,7 +73,7 @@ public void testSystemTableWALEntryFilter() { // meta WALKeyImpl key1 = new WALKeyImpl(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(), - TableName.META_TABLE_NAME, EnvironmentEdgeManager.currentTime()); + MetaTableName.getInstance(), EnvironmentEdgeManager.currentTime()); Entry metaEntry = new Entry(key1, null); assertNull(filter.filter(metaEntry)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java index ab7935ddad41..e91fb0080982 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -98,9 +99,9 @@ public void before() throws Exception { // conf.setInt(HConstants.META_REPLICAS_NUM, numOfMetaReplica); HTU.startMiniCluster(NB_SERVERS); // Enable hbase:meta replication. - HBaseTestingUtil.setReplicas(HTU.getAdmin(), TableName.META_TABLE_NAME, numOfMetaReplica); + HBaseTestingUtil.setReplicas(HTU.getAdmin(), MetaTableName.getInstance(), numOfMetaReplica); - HTU.waitFor(30000, () -> HTU.getMiniHBaseCluster().getRegions(TableName.META_TABLE_NAME).size() + HTU.waitFor(30000, () -> HTU.getMiniHBaseCluster().getRegions(MetaTableName.getInstance()).size() >= numOfMetaReplica); } @@ -118,35 +119,35 @@ public void testHBaseMetaReplicates() throws Exception { try (Table table = HTU.createTable(TableName.valueOf(this.name.getMethodName() + "_0"), HConstants.CATALOG_FAMILY, Arrays.copyOfRange(HBaseTestingUtil.KEYS, 1, HBaseTestingUtil.KEYS.length))) { - verifyReplication(TableName.META_TABLE_NAME, numOfMetaReplica, getMetaCells(table.getName())); + verifyReplication(MetaTableName.getInstance(), numOfMetaReplica, getMetaCells(table.getName())); } try (Table table = HTU.createTable(TableName.valueOf(this.name.getMethodName() + "_1"), HConstants.CATALOG_FAMILY, Arrays.copyOfRange(HBaseTestingUtil.KEYS, 1, HBaseTestingUtil.KEYS.length))) { - verifyReplication(TableName.META_TABLE_NAME, numOfMetaReplica, getMetaCells(table.getName())); + verifyReplication(MetaTableName.getInstance(), numOfMetaReplica, getMetaCells(table.getName())); // Try delete. HTU.deleteTableIfAny(table.getName()); - verifyDeletedReplication(TableName.META_TABLE_NAME, numOfMetaReplica, table.getName()); + verifyDeletedReplication(MetaTableName.getInstance(), numOfMetaReplica, table.getName()); } } @Test public void testCatalogReplicaReplicationWithFlushAndCompaction() throws Exception { try (Connection connection = ConnectionFactory.createConnection(HTU.getConfiguration()); - Table table = connection.getTable(TableName.META_TABLE_NAME)) { + Table table = connection.getTable(MetaTableName.getInstance())) { // load the data to the table for (int i = 0; i < 5; i++) { LOG.info("Writing data from " + i * 1000 + " to " + (i * 1000 + 1000)); HTU.loadNumericRows(table, HConstants.CATALOG_FAMILY, i * 1000, i * 1000 + 1000); LOG.info("flushing table"); - HTU.flush(TableName.META_TABLE_NAME); + HTU.flush(MetaTableName.getInstance()); LOG.info("compacting table"); if (i < 4) { - HTU.compact(TableName.META_TABLE_NAME, false); + HTU.compact(MetaTableName.getInstance(), false); } } - verifyReplication(TableName.META_TABLE_NAME, numOfMetaReplica, 0, 5000, + verifyReplication(MetaTableName.getInstance(), numOfMetaReplica, 0, 5000, HConstants.CATALOG_FAMILY); } } @@ -181,7 +182,7 @@ public void testCatalogReplicaReplicationWithReplicaMoved() throws Exception { } } try (Connection connection = ConnectionFactory.createConnection(HTU.getConfiguration()); - Table table = connection.getTable(TableName.META_TABLE_NAME)) { + Table table = connection.getTable(MetaTableName.getInstance())) { // load the data to the table for (int i = 0; i < 5; i++) { LOG.info("Writing data from " + i * 1000 + " to " + (i * 1000 + 1000)); @@ -191,7 +192,7 @@ public void testCatalogReplicaReplicationWithReplicaMoved() throws Exception { } } - verifyReplication(TableName.META_TABLE_NAME, numOfMetaReplica, 0, 5000, + verifyReplication(MetaTableName.getInstance(), numOfMetaReplica, 0, 5000, HConstants.CATALOG_FAMILY); } } @@ -413,7 +414,7 @@ private void getMetaReplicaReadRequests(final Region[] metaRegions, final long[] @Test public void testHBaseMetaReplicaGets() throws Exception { TableName tn = TableName.valueOf(this.name.getMethodName()); - final Region[] metaRegions = getAllRegions(TableName.META_TABLE_NAME, numOfMetaReplica); + final Region[] metaRegions = getAllRegions(MetaTableName.getInstance(), numOfMetaReplica); long[] readReqsForMetaReplicas = new long[numOfMetaReplica]; long[] readReqsForMetaReplicasAfterGet = new long[numOfMetaReplica]; long[] readReqsForMetaReplicasAfterGetAllLocations = new long[numOfMetaReplica]; @@ -426,7 +427,7 @@ public void testHBaseMetaReplicaGets() throws Exception { try (Table table = HTU.createTable(tn, HConstants.CATALOG_FAMILY, Arrays.copyOfRange(HBaseTestingUtil.KEYS, 1, HBaseTestingUtil.KEYS.length))) { - verifyReplication(TableName.META_TABLE_NAME, numOfMetaReplica, getMetaCells(table.getName())); + verifyReplication(MetaTableName.getInstance(), numOfMetaReplica, getMetaCells(table.getName())); // load different values HTU.loadTable(table, new byte[][] { HConstants.CATALOG_FAMILY }, VALUE); for (int i = 0; i < NB_SERVERS; i++) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java index 37af52eb93b9..9feb2581900e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.regionserver.HRegionServer; @@ -196,7 +197,7 @@ public void testWALEntryFilter() throws IOException { assertTrue(wef.filter(e) == e); // Test system WAL edit. e = new WAL.Entry( - new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, TableName.META_TABLE_NAME, -1, -1, uuid), we); + new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, MetaTableName.getInstance(), -1, -1, uuid), we); assertNull(wef.filter(e)); } finally { rs.terminate("Done"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java index 8731adbe4c2b..40530a5ba39e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; @@ -162,7 +163,7 @@ private void addStateAndBarrier(RegionInfo region, RegionState.State state, long put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER, put.getTimestamp() - barriers.length + i, Bytes.toBytes(barriers[i])); } - try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) { + try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { table.put(put); } } @@ -171,7 +172,7 @@ private void setState(RegionInfo region, RegionState.State state) throws IOExcep Put put = new Put(region.getRegionName(), EnvironmentEdgeManager.currentTime()); put.addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER, Bytes.toBytes(state.name())); - try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) { + try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { table.put(put); } } @@ -188,7 +189,7 @@ private void addParents(RegionInfo region, List parents) throws IOEx put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, ReplicationBarrierFamilyFormat.REPLICATION_PARENT_QUALIFIER, ReplicationBarrierFamilyFormat.getParentsBytes(parents)); - try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) { + try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { table.put(put); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsCPHookCalled.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsCPHookCalled.java index cb53f77bce56..4e82d3926c2b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsCPHookCalled.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsCPHookCalled.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RSGroupTests; import org.junit.After; @@ -69,7 +70,7 @@ public void testGetRSGroupInfoCPHookCalled() throws Exception { @Test public void testGetRSGroupInfoOfTableCPHookCalled() throws Exception { - ADMIN.getRSGroup(TableName.META_TABLE_NAME); + ADMIN.getRSGroup(MetaTableName.getInstance()); assertTrue(OBSERVER.preGetRSGroupInfoOfTableCalled); assertTrue(OBSERVER.postGetRSGroupInfoOfTableCalled); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java index 3a596a02e0a6..4832252c4acd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Version; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -246,10 +247,10 @@ public void testLowerMetaGroupVersion() throws Exception { // move hbase:meta to meta_group Set toAddTables = new HashSet<>(); - toAddTables.add(TableName.META_TABLE_NAME); + toAddTables.add(MetaTableName.getInstance()); ADMIN.setRSGroup(toAddTables, groupName); assertTrue(ADMIN.getConfiguredNamespacesAndTablesInRSGroup(groupName).getSecond() - .contains(TableName.META_TABLE_NAME)); + .contains(MetaTableName.getInstance())); // restart the regionserver in meta_group, and lower its version String originVersion = ""; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java index dffeaa206a24..aea5ac746f33 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; @@ -236,7 +237,7 @@ public void testRunCleanerChore() throws Exception { public void testExecProcedure() throws Exception { verifyAdminCheckForAction((admin) -> { // Using existing table instead of creating a new one. - admin.execProcedure("flush-table-proc", TableName.META_TABLE_NAME.getNameAsString(), + admin.execProcedure("flush-table-proc", MetaTableName.getInstance().getNameAsString(), new HashMap<>()); }); } @@ -259,7 +260,7 @@ public void testExecService() throws Exception { public void testExecProcedureWithRet() throws Exception { verifyAdminCheckForAction((admin) -> { // Using existing table instead of creating a new one. - admin.execProcedureWithReturn("flush-table-proc", TableName.META_TABLE_NAME.getNameAsString(), + admin.execProcedureWithReturn("flush-table-proc", MetaTableName.getInstance().getNameAsString(), new HashMap<>()); }); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java index f132eb6964b1..808930bbcf0f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.AsyncConnection; import org.apache.hadoop.hbase.client.AsyncTable; import org.apache.hadoop.hbase.client.Connection; @@ -97,7 +98,7 @@ public void setUpBeforeMethod() { private void testToken() throws Exception { try (AsyncConnection conn = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get()) { - AsyncTable table = conn.getTable(TableName.META_TABLE_NAME); + AsyncTable table = conn.getTable(MetaTableName.getInstance()); WhoAmIResponse response = table. coprocessorService( AuthenticationService::newStub, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java index 06d64ace3be2..69557a84128f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; @@ -86,7 +87,7 @@ public static void setupBeforeClass() throws Exception { TEST_UTIL.startMiniCluster(1); TEST_UTIL.getHBaseCluster().waitForActiveAndReadyMaster(); - TEST_UTIL.waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME); + TEST_UTIL.waitUntilAllRegionsAssigned(MetaTableName.getInstance()); rootDir = CommonFSUtils.getRootDir(conf); fs = TEST_UTIL.getTestFileSystem(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java index 0b989b8029f0..bb91b5d0a759 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -198,7 +199,7 @@ protected void deleteRegion(Configuration conf, final TableDescriptor htd, byte[ } if (metaRow) { - try (Table meta = connection.getTable(TableName.META_TABLE_NAME, tableExecutorService)) { + try (Table meta = connection.getTable(MetaTableName.getInstance(), tableExecutorService)) { Delete delete = new Delete(deleteRow); meta.delete(delete); } @@ -510,7 +511,7 @@ public boolean tableHasErrors(HbckTableInfo table) { protected void deleteMetaRegion(Configuration conf, boolean unassign, boolean hdfs, boolean regionInfoOnly) throws IOException, InterruptedException { - HRegionLocation metaLocation = connection.getRegionLocator(TableName.META_TABLE_NAME) + HRegionLocation metaLocation = connection.getRegionLocator(MetaTableName.getInstance()) .getRegionLocation(HConstants.EMPTY_START_ROW); ServerName hsa = metaLocation.getServerName(); RegionInfo hri = metaLocation.getRegion(); @@ -526,7 +527,7 @@ protected void deleteMetaRegion(Configuration conf, boolean unassign, boolean hd Path rootDir = CommonFSUtils.getRootDir(conf); FileSystem fs = rootDir.getFileSystem(conf); Path p = - new Path(rootDir + "/" + TableName.META_TABLE_NAME.getNameAsString(), hri.getEncodedName()); + new Path(rootDir + "/" + MetaTableName.getInstance().getNameAsString(), hri.getEncodedName()); Path hriPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE); fs.delete(hriPath, true); } @@ -536,7 +537,7 @@ protected void deleteMetaRegion(Configuration conf, boolean unassign, boolean hd Path rootDir = CommonFSUtils.getRootDir(conf); FileSystem fs = rootDir.getFileSystem(conf); Path p = - new Path(rootDir + "/" + TableName.META_TABLE_NAME.getNameAsString(), hri.getEncodedName()); + new Path(rootDir + "/" + MetaTableName.getInstance().getNameAsString(), hri.getEncodedName()); HBaseFsck.debugLsr(conf, p); boolean success = fs.delete(p, true); LOG.info("Deleted " + p + " sucessfully? " + success); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java index 5e2b4b52950f..e506ab2512d8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; @@ -280,7 +281,7 @@ public void testGetAll() throws IOException, InterruptedException { } // add hbase:meta htds - .createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).build()); + .createTableDescriptor(TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).build()); assertEquals("getAll() didn't return all TableDescriptors, expected: " + (count + 1) + " got: " + htds.getAll().size(), count + 1, htds.getAll().size()); } @@ -299,7 +300,7 @@ public void testParallelGetAll() throws IOException, InterruptedException { } // add hbase:meta htds - .createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).build()); + .createTableDescriptor(TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).build()); int getTableDescriptorSize = htds.getAll().size(); assertEquals("getAll() didn't return all TableDescriptors, expected: " + (count + 1) + " got: " @@ -326,7 +327,7 @@ public void testGetAllOrdering() throws Exception { // Remove hbase:meta from list. It shows up now since we made it dynamic. The schema // is written into the fs by the FSTableDescriptors constructor now where before it // didn't. - tables.remove(TableName.META_TABLE_NAME.getNameAsString()); + tables.remove(MetaTableName.getInstance().getNameAsString()); assertEquals(4, tables.size()); String[] tableNamesOrdered = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java index c8e96383492a..2f73befff8d1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; @@ -198,7 +199,7 @@ private void addStateAndBarrier(RegionInfo region, RegionState.State state, long put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER, put.getTimestamp() - barriers.length + i, Bytes.toBytes(barriers[i])); } - try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) { + try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { table.put(put); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java index b24b721762d3..d28a533314df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.io.hfile.TestHFile; import org.apache.hadoop.hbase.master.assignment.AssignmentManager; @@ -84,7 +85,7 @@ public static void setUpBeforeClass() throws Exception { admin = connection.getAdmin(); admin.balancerSwitch(false, true); - TEST_UTIL.waitUntilAllRegionsAssigned(TableName.META_TABLE_NAME); + TEST_UTIL.waitUntilAllRegionsAssigned(MetaTableName.getInstance()); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java index 503f0dbdd971..e6a79cec995c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java @@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; @@ -287,7 +288,7 @@ public void testRegionServerPort() throws Exception { public void testLoadMetaRegion() throws Exception { HRegionServer rsWithMeta = TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads().stream() .map(t -> t.getRegionServer()) - .filter(rs -> rs.getRegions(TableName.META_TABLE_NAME).size() > 0).findFirst().get(); + .filter(rs -> rs.getRegions(MetaTableName.getInstance()).size() > 0).findFirst().get(); int onlineRegions = rsWithMeta.getNumberOfOnlineRegions(); String rsName = rsWithMeta.getServerName().getAddress().toString(); try (RegionMover rm = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java index 56e103aa612e..809726759025 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; @@ -86,7 +87,7 @@ public void setUp() throws Exception { // Remove rs contains hbase:meta, otherwise test looks unstable and buggy in test env. ServerName rsContainMeta = TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads().stream() .map(t -> t.getRegionServer()) - .filter(rs -> rs.getRegions(TableName.META_TABLE_NAME).size() > 0).findFirst().get() + .filter(rs -> rs.getRegions(MetaTableName.getInstance()).size() > 0).findFirst().get() .getServerName(); LOG.info("{} contains hbase:meta", rsContainMeta); List modifiable = new ArrayList<>(allServers); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java index cc1fefc266c4..b3a5b0384dd2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java @@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination; @@ -456,7 +457,7 @@ public void testRecoveredEditsPathForMeta() throws IOException { @Test public void testOldRecoveredEditsFileSidelined() throws IOException { Path p = createRecoveredEditsPathForRegion(); - Path tdir = CommonFSUtils.getTableDir(HBASEDIR, TableName.META_TABLE_NAME); + Path tdir = CommonFSUtils.getTableDir(HBASEDIR, MetaTableName.getInstance()); Path regiondir = new Path(tdir, RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName()); fs.mkdirs(regiondir); Path parent = WALSplitUtil.getRegionDirRecoveredEditsDir(regiondir); @@ -469,7 +470,7 @@ public void testOldRecoveredEditsFileSidelined() throws IOException { private Path createRecoveredEditsPathForRegion() throws IOException { byte[] encoded = RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(); - Path p = WALSplitUtil.getRegionSplitEditsPath(TableName.META_TABLE_NAME, encoded, 1, + Path p = WALSplitUtil.getRegionSplitEditsPath(MetaTableName.getInstance(), encoded, 1, FILENAME_BEING_SPLIT, TMPDIRNAME, conf, ""); return p; } diff --git a/hbase-shell/src/main/ruby/hbase/table.rb b/hbase-shell/src/main/ruby/hbase/table.rb index aed5e1c26e59..ff852b357ffc 100644 --- a/hbase-shell/src/main/ruby/hbase/table.rb +++ b/hbase-shell/src/main/ruby/hbase/table.rb @@ -748,7 +748,7 @@ def get_all_columns # Checks if current table is one of the 'meta' tables def is_meta_table? - org.apache.hadoop.hbase.TableName::META_TABLE_NAME.equals(@table.getName) + org.apache.hadoop.hbase.MetaTableName.getInstance.equals(@table.getName) end # Given a column specification in the format FAMILY[:QUALIFIER[:CONVERTER]] diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java index 1900c6c0f8da..d4f11144e7cb 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -1073,11 +1074,11 @@ public TRegionInfo getRegionInfo(ByteBuffer searchRow) throws IOError { try { byte[] row = getBytes(searchRow); Result startRowResult = - getReverseScanResult(TableName.META_TABLE_NAME.getName(), row, HConstants.CATALOG_FAMILY); + getReverseScanResult(MetaTableName.getInstance().getName(), row, HConstants.CATALOG_FAMILY); if (startRowResult == null) { throw new IOException( - "Cannot find row in " + TableName.META_TABLE_NAME + ", row=" + Bytes.toStringBinary(row)); + "Cannot find row in " + MetaTableName.getInstance() + ", row=" + Bytes.toStringBinary(row)); } // find region start and end keys diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java index d34600bc5d3a..9ab12c6daf18 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java @@ -21,6 +21,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.master.RegionState; @@ -165,11 +166,12 @@ public static void setMetaLocation(ZKWatcher zookeeper, ServerName serverName, public static void setMetaLocation(ZKWatcher zookeeper, ServerName serverName, int replicaId, RegionState.State state) throws KeeperException { if (serverName == null) { - LOG.warn("Tried to set null ServerName in hbase:meta; skipping -- ServerName required"); + LOG.warn("Tried to set null ServerName in {}; skipping -- ServerName required", + MetaTableName.getInstance()); return; } - LOG.info("Setting hbase:meta replicaId={} location in ZooKeeper as {}, state={}", replicaId, - serverName, state); + LOG.info("Setting {} replicaId={} location in ZooKeeper as {}, state={}", + MetaTableName.getInstance(), replicaId, serverName, state); // Make the MetaRegionServer pb and then get its bytes and save this as // the znode content. MetaRegionServer pbrsr = @@ -180,10 +182,10 @@ public static void setMetaLocation(ZKWatcher zookeeper, ServerName serverName, i ZKUtil.setData(zookeeper, zookeeper.getZNodePaths().getZNodeForReplica(replicaId), data); } catch (KeeperException.NoNodeException nne) { if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) { - LOG.debug("hbase:meta region location doesn't exist, create it"); + LOG.debug("{} region location doesn't exist, create it", MetaTableName.getInstance()); } else { - LOG.debug( - "hbase:meta region location doesn't exist for replicaId=" + replicaId + ", create it"); + LOG.debug("{} region location doesn't exist for replicaId={}, create it", + MetaTableName.getInstance(), replicaId); } ZKUtil.createAndWatch(zookeeper, zookeeper.getZNodePaths().getZNodeForReplica(replicaId), data); @@ -233,9 +235,10 @@ public static void deleteMetaLocation(ZKWatcher zookeeper) throws KeeperExceptio public static void deleteMetaLocation(ZKWatcher zookeeper, int replicaId) throws KeeperException { if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) { - LOG.info("Deleting hbase:meta region location in ZooKeeper"); + LOG.info("Deleting {} region location in ZooKeeper", MetaTableName.getInstance()); } else { - LOG.info("Deleting hbase:meta for {} region location in ZooKeeper", replicaId); + LOG.info("Deleting {} for {} region location in ZooKeeper", MetaTableName.getInstance(), + replicaId); } try { // Just delete the node. Don't need any watches. diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java index cd2b0b1a0149..f5f250ff1354 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java @@ -32,6 +32,7 @@ import java.util.LinkedList; import java.util.List; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; @@ -74,7 +75,7 @@ public static String dump(final ZKWatcher zkWatcher) { sb.append("\n ").append(child); } } - sb.append("\nRegion server holding hbase:meta:"); + sb.append("\nRegion server holding ").append(MetaTableName.getInstance()).append(":"); sb.append("\n ").append(MetaTableLocator.getMetaRegionLocation(zkWatcher)); int numMetaReplicas = zkWatcher.getMetaReplicaNodes().size(); for (int i = 1; i < numMetaReplicas; i++) { From cbcf57beedc1c716192b69aead2274117a862908 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Tue, 16 Dec 2025 13:08:26 -0500 Subject: [PATCH 06/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: fix spotless --- .../backup/impl/IncrementalBackupManager.java | 2 +- .../favored/FavoredNodeAssignmentHelper.java | 3 +-- .../SnapshotOfRegionAssignmentFromMeta.java | 2 +- ...rgeClusterBalancingMetaTableIsolation.java | 2 +- ...gTableIsolationAndReplicaDistribution.java | 2 +- .../client/AsyncNonMetaRegionLocator.java | 5 +++-- .../hbase/client/AsyncRegionLocator.java | 2 +- .../hbase/client/RawAsyncHBaseAdmin.java | 2 +- .../hbase/client/RegionInfoDisplay.java | 1 - .../hbase/security/token/ClientTokenUtil.java | 1 - .../hbase/shaded/protobuf/ProtobufUtil.java | 2 +- .../client/TestAsyncRegionLocatorTracing.java | 3 +-- .../apache/hadoop/hbase/MetaTableName.java | 8 +++---- .../org/apache/hadoop/hbase/TableName.java | 22 +++++-------------- .../hadoop/hbase/TestCellComparator.java | 18 +++++++-------- .../org/apache/hadoop/hbase/TestKeyValue.java | 3 ++- .../util/ProcessBasedLocalHBaseCluster.java | 1 - .../master/MetricsMasterFileSystemSource.java | 1 - .../hbase/mapreduce/TestImportExport.java | 2 +- .../hadoop/hbase/rest/TestStatusResource.java | 2 +- .../model/TestStorageClusterStatusModel.java | 5 ++--- .../hbase/coprocessor/MetaTableMetrics.java | 1 - .../apache/hadoop/hbase/master/HMaster.java | 6 +++-- .../master/RegionPlacementMaintainer.java | 2 +- .../hbase/master/TableNamespaceManager.java | 2 +- .../master/assignment/AssignmentManager.java | 2 +- .../MergeTableRegionsProcedure.java | 2 +- .../master/assignment/RegionStateStore.java | 2 +- .../assignment/SplitTableRegionProcedure.java | 2 +- .../cleaner/ReplicationBarrierCleaner.java | 2 +- .../hbase/master/janitor/CatalogJanitor.java | 2 +- .../master/janitor/ReportMakingVisitor.java | 2 +- .../procedure/DeleteTableProcedure.java | 3 ++- .../procedure/HBCKServerCrashProcedure.java | 2 +- .../procedure/MasterProcedureScheduler.java | 2 +- .../procedure/ModifyTableProcedure.java | 4 ++-- .../hbase/master/procedure/SchemaLocking.java | 2 +- .../procedure/TruncateRegionProcedure.java | 1 - .../hbase/regionserver/RSRpcServices.java | 2 +- .../ReplicationBarrierFamilyFormat.java | 2 +- .../hbase/security/access/AccessChecker.java | 2 +- .../hadoop/hbase/tool/BulkLoadHFilesTool.java | 2 +- .../hadoop/hbase/util/FSTableDescriptors.java | 2 +- .../org/apache/hadoop/hbase/util/FSUtils.java | 1 - .../apache/hadoop/hbase/util/HBaseFsck.java | 10 ++++----- .../hadoop/hbase/util/HBaseFsckRepair.java | 3 +-- .../apache/hadoop/hbase/util/RegionMover.java | 2 +- .../wal/BoundedRecoveredHFilesOutputSink.java | 3 +-- .../client/AbstractTestRegionLocator.java | 2 +- .../client/MetaWithReplicasTestBase.java | 3 +-- .../hbase/client/RegionReplicaTestHelper.java | 2 +- .../apache/hadoop/hbase/client/TestAdmin.java | 2 +- .../hadoop/hbase/client/TestAdmin2.java | 2 +- .../TestAsyncAdminWithRegionReplicas.java | 2 +- .../client/TestAsyncMetaRegionLocator.java | 2 +- .../client/TestAsyncNonMetaRegionLocator.java | 2 +- .../client/TestAsyncRegionAdminApi2.java | 18 ++++++++++----- .../hbase/client/TestAsyncTableAdminApi.java | 8 ++++--- .../hbase/client/TestAsyncTableAdminApi2.java | 2 +- .../hbase/client/TestAsyncTableAdminApi3.java | 3 ++- .../client/TestAsyncTableUseMetaReplicas.java | 2 +- ...talogReplicaLoadBalanceSimpleSelector.java | 12 +++++----- .../hbase/client/TestCleanupMetaReplica.java | 1 - .../client/TestClientScannerTimeouts.java | 2 +- .../client/TestClientSideRegionScanner.java | 2 +- .../hadoop/hbase/client/TestEnableTable.java | 2 +- .../hbase/client/TestFromClientSide5.java | 2 +- .../TestIncreaseMetaReplicaThroughConfig.java | 3 +-- .../hbase/client/TestMasterRegistry.java | 3 +-- .../hadoop/hbase/client/TestMetaCache.java | 2 +- .../client/TestMetaRegionLocationCache.java | 3 +-- .../client/TestMetaWithReplicasBasic.java | 1 - .../TestMetaWithReplicasShutdownHandling.java | 2 +- .../TestMultiActionMetricsFromClient.java | 2 +- .../hbase/client/TestReplicaWithCluster.java | 2 +- .../client/TestRpcConnectionRegistry.java | 3 +-- .../client/TestSeparateClientZKCluster.java | 2 +- .../TestShutdownOfMetaReplicaHolder.java | 3 +-- .../hbase/client/TestSnapshotFromClient.java | 2 +- .../client/TestZKConnectionRegistry.java | 3 +-- .../hadoop/hbase/http/TestInfoServersACL.java | 2 +- .../hadoop/hbase/master/TestMaster.java | 2 +- ...MasterFileSystemWithStoreFileTracking.java | 1 - ...TestMasterOperationsForRegionReplicas.java | 2 +- .../hbase/master/TestMasterRepairMode.java | 2 +- .../hbase/master/TestMasterTransitions.java | 2 +- .../TestMetaAssignmentWithStopMaster.java | 3 +-- .../TestMigrateAndMirrorMetaLocations.java | 3 +-- .../master/TestMigrateNamespaceTable.java | 2 +- .../hbase/master/TestRecreateCluster.java | 2 +- .../TestRestartWithEmptyWALDirectory.java | 2 +- ...ServerCrashProcedureCarryingMetaStuck.java | 1 - .../TestCloseRegionWhileRSCrash.java | 2 +- .../TestOpenRegionProcedureBackoff.java | 2 +- .../assignment/TestRegionStateStore.java | 2 +- .../master/assignment/TestRollbackSCP.java | 2 +- .../BalancerConditionalsTestUtil.java | 2 +- .../TestFavoredStochasticLoadBalancer.java | 2 +- ...MetaTableIsolationBalancerConditional.java | 7 +++--- .../TestReplicationBarrierCleaner.java | 2 +- .../janitor/TestMetaFixerNoCluster.java | 1 - .../TestSimpleRegionNormalizer.java | 2 +- .../hbase/master/procedure/TestHBCKSCP.java | 3 +-- .../TestCompactionInDeadRegionServer.java | 2 +- .../regionserver/TestDefaultMemStore.java | 5 +++-- .../TestEndToEndSplitTransaction.java | 2 +- .../TestGetClosestAtOrBefore.java | 2 +- .../TestReadAndWriteRegionInfoFile.java | 1 - .../hbase/regionserver/TestRegionInfo.java | 2 +- .../regionserver/TestRegionReplicas.java | 2 +- .../TestRegionServerCrashDisableWAL.java | 2 +- .../TestRegionServerNoMaster.java | 5 +++-- .../TestRegionServerRejectDuringAbort.java | 2 +- .../TestShutdownWhileWALBroken.java | 2 +- .../wal/AbstractTestLogRolling.java | 2 +- .../regionserver/wal/TestLogRollAbort.java | 2 +- .../regionserver/wal/TestLogRolling.java | 2 +- .../wal/TestLogRollingNoCluster.java | 3 +-- .../TestReplicationWALEntryFilters.java | 2 +- .../TestMetaRegionReplicaReplication.java | 16 +++++++++----- .../regionserver/TestReplicationSource.java | 2 +- .../TestSerialReplicationChecker.java | 2 +- .../rsgroup/TestRSGroupsCPHookCalled.java | 3 +-- .../hbase/rsgroup/TestRSGroupsKillRS.java | 2 +- .../security/access/TestRpcAccessChecks.java | 6 ++--- .../token/TestGenerateDelegationToken.java | 1 - .../snapshot/TestRegionSnapshotTask.java | 2 +- .../hadoop/hbase/util/BaseTestHBaseFsck.java | 13 ++++++----- .../hbase/util/TestFSTableDescriptors.java | 10 ++++----- ...TestHBaseFsckCleanReplicationBarriers.java | 2 +- .../hadoop/hbase/util/TestHBaseFsckMOB.java | 2 +- .../hadoop/hbase/util/TestRegionMover1.java | 2 +- .../TestRegionMoverWithRSGroupEnable.java | 2 +- .../apache/hadoop/hbase/wal/TestWALSplit.java | 2 +- .../thrift/ThriftHBaseServiceHandler.java | 6 ++--- .../hbase/zookeeper/MetaTableLocator.java | 2 +- 136 files changed, 201 insertions(+), 222 deletions(-) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java index 76599704d87e..ec5a04a63a05 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java @@ -27,8 +27,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore; diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java index 8a36477913c9..d316075979a1 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java @@ -36,9 +36,8 @@ import org.apache.hadoop.hbase.CellBuilderType; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java index 783ad2cbbc23..e54ce7478e99 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java @@ -33,10 +33,10 @@ import org.apache.hadoop.hbase.CatalogFamilyFormat; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.Result; diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java index c62fbea144a4..1843cb7b895d 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java @@ -27,9 +27,9 @@ import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.master.balancer.BalancerTestBase.MockMapping; diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java index efb5ccabe115..4c8e9ffd5526 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java @@ -28,9 +28,9 @@ import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.master.balancer.BalancerTestBase.MockMapping; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java index 3afe3ae63bed..2ef315d2b4ac 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java @@ -51,9 +51,9 @@ import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Scan.ReadType; @@ -245,7 +245,8 @@ private boolean tryComplete(LocateRequest req, CompletableFuture { public static final String VALID_USER_TABLE_REGEX = "(?:(?:(?:" + VALID_NAMESPACE_REGEX + "\\" + NAMESPACE_DELIM + ")?)" + "(?:" + VALID_TABLE_QUALIFIER_REGEX + "))"; - /** - * The name of hbase meta table could either be hbase:meta_xxx or 'hbase:meta' otherwise. Config - * hbase.meta.table.suffix will govern the decision of adding suffix to the habase:meta - * - * This field is initialized from the MetaTableName singleton and can be overridden for testing - * by modifying the singleton instance via reflection. - * - * @deprecated Use {@link MetaTableName#getInstance()} instead. This field will be removed in a - * future version. - */ + /** The hbase:meta table's name. */ @Deprecated - public static TableName META_TABLE_NAME = TableName.valueOf("hbase:meta"); + public static TableName META_TABLE_NAME = + valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); /** * The Namespace table's name. @@ -305,8 +295,8 @@ private TableName(ByteBuffer namespace, ByteBuffer qualifier) throws IllegalArgu throw new IllegalArgumentException(OLD_ROOT_STR + " has been deprecated."); } if (qualifierAsString.equals(OLD_META_STR)) { - throw new IllegalArgumentException( - OLD_META_STR + " no longer exists. The table has been " + "renamed to " + MetaTableName.getInstance()); + throw new IllegalArgumentException(OLD_META_STR + " no longer exists. The table has been " + + "renamed to " + MetaTableName.getInstance()); } if (Bytes.equals(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME, namespace)) { diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java index 68666e3552f0..5c143d8ee065 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java @@ -196,22 +196,20 @@ public void testMetaComparisons2() { long now = EnvironmentEdgeManager.currentTime(); CellComparator c = MetaCellComparator.META_COMPARATOR; assertTrue(c.compare( - createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now)), - createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now))) - == 0); + createByteBufferKeyValueFromKeyValue(new KeyValue( + Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now)), + createByteBufferKeyValueFromKeyValue(new KeyValue( + Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now))) == 0); Cell a = createByteBufferKeyValueFromKeyValue( new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now)); Cell b = createByteBufferKeyValueFromKeyValue( new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,2"), now)); assertTrue(c.compare(a, b) < 0); assertTrue(c.compare( - createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,2"), now)), - createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now))) - > 0); + createByteBufferKeyValueFromKeyValue(new KeyValue( + Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,2"), now)), + createByteBufferKeyValueFromKeyValue(new KeyValue( + Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now))) > 0); assertTrue(c.compare( createByteBufferKeyValueFromKeyValue( new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now)), diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java index c06d5569e00d..1e65b75a9777 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java @@ -216,7 +216,8 @@ private void comparisons(final CellComparatorImpl c) { long now = EnvironmentEdgeManager.currentTime(); assertTrue(c.compare( new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now), - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now)) == 0); + new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now)) + == 0); assertTrue(c.compare( new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now), new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,2"), now)) < 0); diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java index 6407ff13bbdb..267b78dade13 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java @@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.slf4j.Logger; diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java index 9bc4a90c8cf5..7aea9b356259 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.master; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.metrics.BaseSource; import org.apache.yetus.audience.InterfaceAudience; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java index 09ce98ddc172..b861c29a9bcc 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java @@ -50,9 +50,9 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.client.ClientInternalHelper; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java index b493be5c9f06..5ddc23d36554 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java @@ -29,8 +29,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.rest.client.Client; import org.apache.hadoop.hbase.rest.client.Cluster; diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java index c3761c397c8a..1db8c371593c 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java @@ -23,7 +23,6 @@ import java.util.Iterator; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -90,8 +89,8 @@ protected StorageClusterStatusModel buildTestModel() { model.addLiveNode("test1", 1245219839331L, 128, 1024).addRegion(Bytes.toBytes("hbase:root,,0"), 1, 1, 0, 0, 0, 1, 1, 2, 1, 1, 1, 1, 1); model.addLiveNode("test2", 1245239331198L, 512, 1024).addRegion( - Bytes.toBytes(MetaTableName.getInstance() + ",,1246000043724"), 1, 1, 0, 0, 0, 1, 1, 2, 1, 1, 1, - 1, 1); + Bytes.toBytes(MetaTableName.getInstance() + ",,1246000043724"), 1, 1, 0, 0, 0, 1, 1, 2, 1, 1, + 1, 1, 1); return model; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java index fc2d164e922d..aa934cc6d462 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java @@ -26,7 +26,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CoprocessorEnvironment; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index d3498bf3d897..601da7239f3e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -83,6 +83,7 @@ import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.PleaseHoldException; import org.apache.hadoop.hbase.PleaseRestartMasterException; @@ -93,7 +94,6 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerTask; import org.apache.hadoop.hbase.ServerTaskBuilder; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; @@ -1093,7 +1093,9 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE startupTaskGroup.addTask("Initializing meta table if this is a new deploy"); InitMetaProcedure initMetaProc = null; // Print out state of hbase:meta on startup; helps debugging. - if (!this.assignmentManager.getRegionStates().hasTableRegionStates(MetaTableName.getInstance())) { + if ( + !this.assignmentManager.getRegionStates().hasTableRegionStates(MetaTableName.getInstance()) + ) { Optional optProc = procedureExecutor.getProcedures().stream() .filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny(); initMetaProc = optProc.orElseGet(() -> { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java index c1195e0a90a1..3edfc1eb67a0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java @@ -35,8 +35,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AsyncClusterConnection; import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java index 70aa46960e00..9903c84785e9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java @@ -28,8 +28,8 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Delete; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java index 7d7f2dacda89..3a50d74e8a64 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java @@ -45,9 +45,9 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.PleaseHoldException; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.client.DoNotRetryRegionException; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java index 11ae106a0b8c..e6891d08f075 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java @@ -28,8 +28,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.MetaMutationAnnotation; -import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java index 8bcf3b5a93fb..4d42ad619255 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java @@ -36,9 +36,9 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AsyncTable; import org.apache.hadoop.hbase.client.Delete; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java index 1e405a001a9e..55ac3fa2a0bb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java @@ -38,8 +38,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java index 1f141a8de06e..419f8fb15312 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java @@ -23,9 +23,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.Stoppable; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Delete; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java index 4e7925e708af..a6cec33c3efb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java @@ -33,8 +33,8 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java index f370fdc2ffe8..c74be9de50d9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java @@ -22,9 +22,9 @@ import org.apache.hadoop.hbase.ClientMetaTableAccessor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.Result; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java index 960fa3defe6d..98f43871238e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java @@ -395,7 +395,8 @@ private static void cleanRegionsInMeta(final MasterProcedureEnv env, final Table long now = EnvironmentEdgeManager.currentTime(); List deletes = new ArrayList<>(); try ( - Table metaTable = env.getMasterServices().getConnection().getTable(MetaTableName.getInstance()); + Table metaTable = + env.getMasterServices().getConnection().getTable(MetaTableName.getInstance()); ResultScanner scanner = metaTable.getScanner(tableScan)) { for (;;) { Result result = scanner.next(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java index 635dcc0a91b2..d7a4ce95c4ff 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java @@ -25,9 +25,9 @@ import org.apache.hadoop.hbase.ClientMetaTableAccessor; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.Result; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java index 7ed12a280afd..68c08cbf0e5f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java @@ -30,9 +30,9 @@ import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableExistsException; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.RegionInfo; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java index 6c7dc29e4ddf..c9e2d67b43df 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java @@ -110,8 +110,8 @@ protected void preflightChecks(MasterProcedureEnv env, Boolean enabled) throws H Set cfs = this.modifiedTableDescriptor.getColumnFamilyNames(); for (byte[] family : UNDELETABLE_META_COLUMNFAMILIES) { if (!cfs.contains(family)) { - throw new HBaseIOException( - "Delete of " + MetaTableName.getInstance() + " column family " + Bytes.toString(family)); + throw new HBaseIOException("Delete of " + MetaTableName.getInstance() + " column family " + + Bytes.toString(family)); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java index 2afaf1c20b6f..1b9b95889e05 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java @@ -25,8 +25,8 @@ import java.util.stream.Collectors; import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; -import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.locking.LockProcedure; import org.apache.hadoop.hbase.procedure2.LockAndQueue; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java index 57f4e9e4359c..4adf36efc7f6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java @@ -21,7 +21,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.MetaTableName; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterFileSystem; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 46b81988a9e7..4b7b7787f79d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -56,13 +56,13 @@ import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HBaseRpcServicesBase; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.MultiActionResultTooLarge; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.RegionTooBusyException; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownScannerException; import org.apache.hadoop.hbase.client.Append; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java index f82ac9bd42ec..ebdd54a081d0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java @@ -33,8 +33,8 @@ import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java index 637785c2b62e..d90d671feffc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java @@ -33,9 +33,9 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.security.AccessDeniedException; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java index 6c206557027e..7c0dbffb1330 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java @@ -65,8 +65,8 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.AsyncAdmin; import org.apache.hadoop.hbase.client.AsyncClusterConnection; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java index 28cb4bcb69e5..e3c13f92a017 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java @@ -46,9 +46,9 @@ import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.CoprocessorDescriptorBuilder; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java index 0b502c3c5ca6..3b446826b775 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java @@ -69,7 +69,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index 9bfbef4a2536..9eff10a0b160 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -81,10 +81,10 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; @@ -663,8 +663,8 @@ public int onlineConsistencyRepair() throws IOException, KeeperException, Interr if (!checkMetaRegion()) { String errorMsg = MetaTableName.getInstance() + " table is not consistent. "; if (shouldFixAssignments()) { - errorMsg += "HBCK will try fixing it. Rerun once " + MetaTableName.getInstance() + " is back " - + "to consistent state."; + errorMsg += "HBCK will try fixing it. Rerun once " + MetaTableName.getInstance() + + " is back " + "to consistent state."; } else { errorMsg += "Run HBCK with proper fix options to fix " + MetaTableName.getInstance() + " inconsistency."; @@ -1222,8 +1222,8 @@ private boolean sidelineFile(FileSystem fs, Path hbaseRoot, Path path) throws IO * TODO -- need to add tests for this. */ private void reportEmptyMetaCells() { - errors.print("Number of empty REGIONINFO_QUALIFIER rows in " + MetaTableName.getInstance() + ": " - + emptyRegionInfoQualifiers.size()); + errors.print("Number of empty REGIONINFO_QUALIFIER rows in " + MetaTableName.getInstance() + + ": " + emptyRegionInfoQualifiers.size()); if (details) { for (Result r : emptyRegionInfoQualifiers) { errors.print(" " + r); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java index 8550220f2a5f..e3516ed855b0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java @@ -27,9 +27,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ClusterMetrics.Option; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.AsyncClusterConnection; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java index dda03bad7525..7321d7da178d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java @@ -56,8 +56,8 @@ 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.MetaTableName; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java index c14cf914b497..36ecfbd1a247 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.wal; -import org.apache.hadoop.hbase.MetaTableName; import java.io.IOException; import java.io.InterruptedIOException; import java.util.HashMap; @@ -34,9 +33,9 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.ExtendedCell; import org.apache.hadoop.hbase.MetaCellComparator; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java index bfed5d2452cf..77fc747178fa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java @@ -26,9 +26,9 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java index 0fb896111a42..7b2aa41a2202 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java @@ -24,10 +24,9 @@ import java.util.HashSet; import java.util.Set; import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.master.assignment.AssignmentManager; import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java index 38b4ee29a434..25953227d8df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java @@ -28,11 +28,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.regionserver.Region; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java index 61a390b6ba51..986701050453 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java @@ -36,10 +36,10 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.assignment.AssignmentManager; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index 1e4f2c19cc3d..a6dd60781bd0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -37,11 +37,11 @@ import org.apache.hadoop.hbase.ClusterMetrics.Option; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java index 08f72317ef56..66ac07cdf88f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java @@ -31,8 +31,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java index 7dc75ee9d935..871590d732e7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java @@ -39,11 +39,11 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MatcherPredicate; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.MiniClusterRule; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.RegionReplicaTestHelper.Locator; import org.apache.hadoop.hbase.client.trace.StringTraceRenderer; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java index 4d4c307f4593..4da4c87daba3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java @@ -43,11 +43,11 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.client.RegionReplicaTestHelper.Locator; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java index ff869d5c9c0e..4a51b6d26465 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java @@ -86,7 +86,8 @@ public void testSplitSwitch() throws Exception { final int rows = 10000; TestAsyncRegionAdminApi.loadData(tableName, families, rows); - AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); + AsyncTable metaTable = + ASYNC_CONN.getTable(MetaTableName.getInstance()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); int originalCount = regionLocations.size(); @@ -117,7 +118,8 @@ public void testMergeSwitch() throws Exception { byte[][] families = { FAMILY }; TestAsyncRegionAdminApi.loadData(tableName, families, 1000); - AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); + AsyncTable metaTable = + ASYNC_CONN.getTable(MetaTableName.getInstance()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); int originalCount = regionLocations.size(); @@ -162,7 +164,8 @@ public void testMergeRegions() throws Exception { byte[][] splitRows = new byte[][] { Bytes.toBytes("3"), Bytes.toBytes("6") }; createTableWithDefaultConf(tableName, splitRows); - AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); + AsyncTable metaTable = + ASYNC_CONN.getTable(MetaTableName.getInstance()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); RegionInfo regionA; @@ -242,7 +245,8 @@ private void splitTest(TableName tableName, int rowCount, boolean isSplitRegion, // create table createTableWithDefaultConf(tableName); - AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); + AsyncTable metaTable = + ASYNC_CONN.getTable(MetaTableName.getInstance()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); assertEquals(1, regionLocations.size()); @@ -299,7 +303,8 @@ public void testTruncateRegion() throws Exception { final byte[][] bFamilies = new byte[][] { Bytes.toBytes(family1), Bytes.toBytes(family2) }; createTableWithDefaultConf(tableName, splitKeys, bFamilies); - AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); + AsyncTable metaTable = + ASYNC_CONN.getTable(MetaTableName.getInstance()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); RegionInfo regionToBeTruncated = regionLocations.get(0).getRegion(); @@ -333,7 +338,8 @@ public void testTruncateReplicaRegionNotAllowed() throws Exception { final byte[][] bFamilies = new byte[][] { Bytes.toBytes(family1), Bytes.toBytes(family2) }; createTableWithDefaultConf(tableName, 2, splitKeys, bFamilies); - AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); + AsyncTable metaTable = + ASYNC_CONN.getTable(MetaTableName.getInstance()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); RegionInfo primaryRegion = regionLocations.get(0).getRegion(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java index 44d01cecb92a..f6732c611194 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java @@ -31,8 +31,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; @@ -80,7 +80,8 @@ static TableState.State getStateFromMeta(TableName table) throws Exception { @Test public void testCreateTableNumberOfRegions() throws Exception { - AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); + AsyncTable metaTable = + ASYNC_CONN.getTable(MetaTableName.getInstance()); createTableWithDefaultConf(tableName); List regionLocations = @@ -128,7 +129,8 @@ public void testCreateTableWithRegions() throws Exception { boolean tableAvailable = admin.isTableAvailable(tableName).get(); assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable); - AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); + AsyncTable metaTable = + ASYNC_CONN.getTable(MetaTableName.getInstance()); List regions = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); Iterator hris = regions.iterator(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java index f56412c5d589..1b2ef352515c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java @@ -26,8 +26,8 @@ import java.util.Set; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java index 89708d40bf1e..93e684453f51 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java @@ -285,7 +285,8 @@ public void testEnableTableRetainAssignment() throws Exception { int expectedRegions = splitKeys.length + 1; createTableWithDefaultConf(tableName, splitKeys); - AsyncTable metaTable = ASYNC_CONN.getTable(MetaTableName.getInstance()); + AsyncTable metaTable = + ASYNC_CONN.getTable(MetaTableName.getInstance()); List regions = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); assertEquals( diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java index 96bb7b2bd1ba..db794ff014ff 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java @@ -27,8 +27,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java index e180d33df541..a392ff5c9a69 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java @@ -27,8 +27,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -102,7 +102,8 @@ public void testMetaChangeFromReplicaNoReplica() throws IOException, Interrupted .get(CONN.connConf.getMetaReadRpcTimeoutNs(), TimeUnit.NANOSECONDS); numOfReplicas = metaLocations.size(); } catch (Exception e) { - LOG.error("Failed to get table {}'s region replication, ", MetaTableName.getInstance(), e); + LOG.error("Failed to get table {}'s region replication, ", MetaTableName.getInstance(), + e); } return numOfReplicas; }); @@ -121,15 +122,16 @@ public void testMetaChangeFromReplicaNoReplica() throws IOException, Interrupted () -> TEST_UTIL.getMiniHBaseCluster().getRegions(MetaTableName.getInstance()).size() == 1); CatalogReplicaLoadBalanceSelector metaSelectorWithNoReplica = - CatalogReplicaLoadBalanceSelectorFactory.createSelector(replicaSelectorClass, MetaTableName.getInstance(), - CONN, () -> { + CatalogReplicaLoadBalanceSelectorFactory.createSelector(replicaSelectorClass, + MetaTableName.getInstance(), CONN, () -> { int numOfReplicas = CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS; try { RegionLocations metaLocations = CONN.registry.getMetaRegionLocations() .get(CONN.connConf.getMetaReadRpcTimeoutNs(), TimeUnit.NANOSECONDS); numOfReplicas = metaLocations.size(); } catch (Exception e) { - LOG.error("Failed to get table {}'s region replication, ", MetaTableName.getInstance(), e); + LOG.error("Failed to get table {}'s region replication, ", MetaTableName.getInstance(), + e); } return numOfReplicas; }); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java index 3c314b5aa968..3b5a1496e403 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java @@ -22,7 +22,6 @@ import java.util.List; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java index 91beea20c73c..55bf792f568f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java @@ -29,10 +29,10 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.SingleProcessHBaseCluster.MiniHBaseClusterRegionServer; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException; import org.apache.hadoop.hbase.ipc.CallTimeoutException; import org.apache.hadoop.hbase.regionserver.HRegionServer; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java index b9294f8f991b..e40208aa3b90 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java @@ -38,8 +38,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.client.metrics.ScanMetricsRegionInfo; import org.apache.hadoop.hbase.filter.FilterBase; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java index b5b3e652ea89..225c30046fde 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java @@ -27,8 +27,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java index 8a46e0f79a18..d224ad41a277 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java @@ -48,10 +48,10 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Scan.ReadType; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java index 044be8b5aa70..e14944a14e03 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java @@ -21,9 +21,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableDescriptors; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java index e14ccfa949e8..365e371bf13d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java @@ -32,10 +32,9 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.security.User; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java index 5c4d6642f311..267593311fdc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java @@ -34,12 +34,12 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.MultiActionResultTooLarge; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionTooBusyException; import org.apache.hadoop.hbase.RetryImmediatelyException; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil; import org.apache.hadoop.hbase.exceptions.RegionOpeningException; import org.apache.hadoop.hbase.quotas.RpcThrottlingException; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java index 977a3571f595..cfe193f60fb8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java @@ -30,10 +30,9 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaRegionLocationCache; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.MultithreadedTestUtil; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.security.User; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java index 799c90c57c3f..1fe06bcd3c56 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java @@ -24,7 +24,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.testclassification.MediumTests; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java index aab092dda927..240e60099edb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java @@ -28,9 +28,9 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java index 4e6f9965c6d9..5e2a11a13dc8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java @@ -22,8 +22,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java index caa7c825601e..a453646e4247 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java @@ -35,8 +35,8 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor; import org.apache.hadoop.hbase.coprocessor.ObserverContext; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java index 14c750398a3a..4c24dcc10949 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java @@ -34,9 +34,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.regionserver.BootstrapNodeManager; import org.apache.hadoop.hbase.regionserver.RSRpcServices; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java index 84430264474d..c196dded6618 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java @@ -26,11 +26,11 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java index 32c9df8ad3bc..f24a894085bd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java @@ -20,9 +20,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.junit.BeforeClass; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java index c0b6a337da26..693713c572d5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java @@ -32,8 +32,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java index 395cddf1cd97..474f8153ca3d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java @@ -33,9 +33,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.RegionLocations; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java index 4d7cbdd2680a..e42af8bbed93 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java @@ -34,8 +34,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.LocalHBaseCluster; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.security.HBaseKerberosUtils; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java index 16c3829a1a7a..d09da4fefafd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java @@ -36,11 +36,11 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.PleaseHoldException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java index 4fda3f2b5677..6abc5e7d680d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java @@ -23,7 +23,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.testclassification.MasterTests; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java index df1e86fa4916..86cb2ba4af3d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java @@ -42,11 +42,11 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java index 14a2f83c21a3..46ff1a582235 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java @@ -29,9 +29,9 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.AsyncTable; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Put; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java index 29b55a9d46a4..253c5899c426 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java @@ -22,8 +22,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java index 12121dbd064b..df8dca55a09e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java @@ -22,10 +22,9 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.RegionLocator; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java index 91b9325c2831..bd3c9398a13b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java @@ -30,11 +30,10 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java index 4fa5761a29e9..6e37bf1be5c6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java @@ -28,10 +28,10 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java index 7d2c74d7ef3c..19e05e26f551 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java @@ -28,11 +28,11 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java index 7f8b4d9ed1a0..a6b11a569875 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java @@ -24,8 +24,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java index a679a36f6bb0..f20b06c7b003 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java @@ -21,7 +21,6 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.AsyncAdmin; import org.apache.hadoop.hbase.client.AsyncConnection; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java index e2e5a7fbe6c5..ce23d2342154 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java @@ -21,9 +21,9 @@ import java.util.concurrent.CountDownLatch; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ProcedureTestUtil; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java index 6794ebbbc24d..6941fe667e39 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java @@ -25,8 +25,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.ProcedureTestUtil; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.ProcedureTestUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AsyncAdmin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java index 6458e233977e..76935cea8968 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java @@ -37,8 +37,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.client.Admin; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java index 5e187a998f2a..e8a2f5202b25 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java @@ -29,9 +29,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.BalanceRequest; import org.apache.hadoop.hbase.master.HMaster; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java index 0d5c4a7888ac..8a1f7b5601cb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java @@ -29,8 +29,8 @@ import java.util.stream.Collectors; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java index f6f8fd1a7eb5..068b15fbd662 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java @@ -36,9 +36,9 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Admin; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java index 768854c32778..d179916cc6b7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java @@ -30,8 +30,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -93,8 +93,9 @@ public void testTableIsolation() throws Exception { admin.createTable(productTableDescriptor, BalancerConditionalsTestUtil.generateSplits(2 * NUM_SERVERS)); - Set tablesToBeSeparated = ImmutableSet. builder() - .add(MetaTableName.getInstance()).add(QuotaUtil.QUOTA_TABLE_NAME).add(productTableName).build(); + Set tablesToBeSeparated = + ImmutableSet. builder().add(MetaTableName.getInstance()) + .add(QuotaUtil.QUOTA_TABLE_NAME).add(productTableName).build(); // Pause the balancer admin.balancerSwitch(false, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java index 006a63625fc5..825fba807562 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java @@ -35,8 +35,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java index 75940a4fd42a..880a3ffbd812 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java @@ -26,7 +26,6 @@ import java.util.SortedSet; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.MetaTableName; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.testclassification.MasterTests; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java index 67537a96def7..62aea9177513 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java @@ -56,10 +56,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.RegionMetrics; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.Size; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.client.RegionInfo; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java index 39c34794bb2f..a9ebb47a6d84 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java @@ -34,10 +34,9 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; -import org.apache.hadoop.hbase.MetaTableName; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.Result; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java index 58507a063d2d..1fcf695da810 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java @@ -27,8 +27,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.YouAreDeadException; import org.apache.hadoop.hbase.client.Get; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java index 63dbfda6f2d1..801652080107 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java @@ -44,9 +44,9 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueTestUtil; import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; @@ -1003,7 +1003,8 @@ public void testShouldFlushMeta() throws Exception { TableDescriptors tds = new FSTableDescriptors(conf); FSTableDescriptors.tryUpdateMetaTableDescriptor(conf); HRegion meta = HRegion.createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, testDir, conf, - tds.get(MetaTableName.getInstance()), wFactory.getWAL(RegionInfoBuilder.FIRST_META_REGIONINFO)); + tds.get(MetaTableName.getInstance()), + wFactory.getWAL(RegionInfoBuilder.FIRST_META_REGIONINFO)); // parameterized tests add [#] suffix get rid of [ and ]. TableDescriptor desc = TableDescriptorBuilder .newBuilder(TableName.valueOf(name.getMethodName().replaceAll("[\\[\\]]", "_"))) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java index 7916101c425e..ddcb9cec5446 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java @@ -39,11 +39,11 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java index 8acd75c5e675..900adcc38757 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java @@ -32,9 +32,9 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java index f3f824ee5135..ebc7c4d13d8d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtil; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java index 3a31d1f7fd2d..fc0a2dd8a470 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java @@ -31,8 +31,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.RegionInfoDisplay; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java index 2ef40ffeb257..559a7e72832c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java @@ -34,8 +34,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TestMetaTableAccessor; import org.apache.hadoop.hbase.client.Consistency; import org.apache.hadoop.hbase.client.Get; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java index f5b3611fad29..bb2b2f0abd0b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java @@ -22,8 +22,8 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java index 96a8db5cabe6..e21606458761 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java @@ -23,10 +23,10 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionLocator; @@ -102,7 +102,8 @@ public static void stopMasterAndCacheMetaLocation(HBaseTestingUtil HTU) locator.getAllRegionLocations(); } } - try (RegionLocator locator = HTU.getConnection().getRegionLocator(MetaTableName.getInstance())) { + try ( + RegionLocator locator = HTU.getConnection().getRegionLocator(MetaTableName.getInstance())) { locator.getAllRegionLocations(); } // Stop master diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java index 177b2c7d7787..e42209ae78b5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java @@ -28,10 +28,10 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java index 37d9d5954d90..a5ad29af1ea8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java @@ -27,8 +27,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.YouAreDeadException; import org.apache.hadoop.hbase.client.Table; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java index 563f5f03dfcb..24e393fbe716 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java @@ -31,11 +31,11 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java index 511b58c9afd5..bc8ee0a836d6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java @@ -33,10 +33,10 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java index ad72bea40c89..3ba45bdf4915 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java @@ -35,8 +35,8 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java index ac62e0395325..635b8abcc7ab 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java @@ -29,9 +29,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.TableDescriptors; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java index 9e79200a0aa5..522bd5ac1367 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java @@ -34,8 +34,8 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.security.access.PermissionStorage; import org.apache.hadoop.hbase.security.visibility.VisibilityConstants; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java index e91fb0080982..c78c967563e2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java @@ -36,9 +36,9 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -101,8 +101,9 @@ public void before() throws Exception { // Enable hbase:meta replication. HBaseTestingUtil.setReplicas(HTU.getAdmin(), MetaTableName.getInstance(), numOfMetaReplica); - HTU.waitFor(30000, () -> HTU.getMiniHBaseCluster().getRegions(MetaTableName.getInstance()).size() - >= numOfMetaReplica); + HTU.waitFor(30000, + () -> HTU.getMiniHBaseCluster().getRegions(MetaTableName.getInstance()).size() + >= numOfMetaReplica); } @After @@ -119,12 +120,14 @@ public void testHBaseMetaReplicates() throws Exception { try (Table table = HTU.createTable(TableName.valueOf(this.name.getMethodName() + "_0"), HConstants.CATALOG_FAMILY, Arrays.copyOfRange(HBaseTestingUtil.KEYS, 1, HBaseTestingUtil.KEYS.length))) { - verifyReplication(MetaTableName.getInstance(), numOfMetaReplica, getMetaCells(table.getName())); + verifyReplication(MetaTableName.getInstance(), numOfMetaReplica, + getMetaCells(table.getName())); } try (Table table = HTU.createTable(TableName.valueOf(this.name.getMethodName() + "_1"), HConstants.CATALOG_FAMILY, Arrays.copyOfRange(HBaseTestingUtil.KEYS, 1, HBaseTestingUtil.KEYS.length))) { - verifyReplication(MetaTableName.getInstance(), numOfMetaReplica, getMetaCells(table.getName())); + verifyReplication(MetaTableName.getInstance(), numOfMetaReplica, + getMetaCells(table.getName())); // Try delete. HTU.deleteTableIfAny(table.getName()); verifyDeletedReplication(MetaTableName.getInstance(), numOfMetaReplica, table.getName()); @@ -427,7 +430,8 @@ public void testHBaseMetaReplicaGets() throws Exception { try (Table table = HTU.createTable(tn, HConstants.CATALOG_FAMILY, Arrays.copyOfRange(HBaseTestingUtil.KEYS, 1, HBaseTestingUtil.KEYS.length))) { - verifyReplication(MetaTableName.getInstance(), numOfMetaReplica, getMetaCells(table.getName())); + verifyReplication(MetaTableName.getInstance(), numOfMetaReplica, + getMetaCells(table.getName())); // load different values HTU.loadTable(table, new byte[][] { HConstants.CATALOG_FAMILY }, VALUE); for (int i = 0; i < NB_SERVERS; i++) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java index 9feb2581900e..c6d01d0e7a51 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java @@ -45,11 +45,11 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.regionserver.HRegionServer; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java index 40530a5ba39e..be93b44d8a62 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java @@ -36,9 +36,9 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsCPHookCalled.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsCPHookCalled.java index 4e82d3926c2b..9f6019974574 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsCPHookCalled.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsCPHookCalled.java @@ -20,10 +20,9 @@ import static org.junit.Assert.assertTrue; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RSGroupTests; import org.junit.After; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java index 4832252c4acd..5510ac230111 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java @@ -29,10 +29,10 @@ import java.util.Set; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Version; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java index aea5ac746f33..3a9ccca4be5e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java @@ -33,9 +33,9 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; @@ -260,8 +260,8 @@ public void testExecService() throws Exception { public void testExecProcedureWithRet() throws Exception { verifyAdminCheckForAction((admin) -> { // Using existing table instead of creating a new one. - admin.execProcedureWithReturn("flush-table-proc", MetaTableName.getInstance().getNameAsString(), - new HashMap<>()); + admin.execProcedureWithReturn("flush-table-proc", + MetaTableName.getInstance().getNameAsString(), new HashMap<>()); }); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java index 808930bbcf0f..e6b2775f2921 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java @@ -29,7 +29,6 @@ import java.util.Collection; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.AsyncConnection; import org.apache.hadoop.hbase.client.AsyncTable; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java index 69557a84128f..8a941ee65ce4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java @@ -33,8 +33,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java index bb91b5d0a759..0d0bd7b81b1c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java @@ -41,9 +41,9 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -199,7 +199,8 @@ protected void deleteRegion(Configuration conf, final TableDescriptor htd, byte[ } if (metaRow) { - try (Table meta = connection.getTable(MetaTableName.getInstance(), tableExecutorService)) { + try ( + Table meta = connection.getTable(MetaTableName.getInstance(), tableExecutorService)) { Delete delete = new Delete(deleteRow); meta.delete(delete); } @@ -526,8 +527,8 @@ protected void deleteMetaRegion(Configuration conf, boolean unassign, boolean hd LOG.info("deleting hdfs .regioninfo data: " + hri.toString() + hsa.toString()); Path rootDir = CommonFSUtils.getRootDir(conf); FileSystem fs = rootDir.getFileSystem(conf); - Path p = - new Path(rootDir + "/" + MetaTableName.getInstance().getNameAsString(), hri.getEncodedName()); + Path p = new Path(rootDir + "/" + MetaTableName.getInstance().getNameAsString(), + hri.getEncodedName()); Path hriPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE); fs.delete(hriPath, true); } @@ -536,8 +537,8 @@ protected void deleteMetaRegion(Configuration conf, boolean unassign, boolean hd LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString()); Path rootDir = CommonFSUtils.getRootDir(conf); FileSystem fs = rootDir.getFileSystem(conf); - Path p = - new Path(rootDir + "/" + MetaTableName.getInstance().getNameAsString(), hri.getEncodedName()); + Path p = new Path(rootDir + "/" + MetaTableName.getInstance().getNameAsString(), + hri.getEncodedName()); HBaseFsck.debugLsr(conf, p); boolean success = fs.delete(p, true); LOG.info("Deleted " + p + " sucessfully? " + success); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java index e506ab2512d8..9a25654f87de 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java @@ -36,9 +36,9 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; @@ -280,8 +280,8 @@ public void testGetAll() throws IOException, InterruptedException { TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i)).build()); } // add hbase:meta - htds - .createTableDescriptor(TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).build()); + htds.createTableDescriptor( + TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).build()); assertEquals("getAll() didn't return all TableDescriptors, expected: " + (count + 1) + " got: " + htds.getAll().size(), count + 1, htds.getAll().size()); } @@ -299,8 +299,8 @@ public void testParallelGetAll() throws IOException, InterruptedException { TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i)).build()); } // add hbase:meta - htds - .createTableDescriptor(TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).build()); + htds.createTableDescriptor( + TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).build()); int getTableDescriptorSize = htds.getAll().size(); assertEquals("getAll() didn't return all TableDescriptors, expected: " + (count + 1) + " got: " diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java index 2f73befff8d1..db5bc77a367c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java @@ -29,9 +29,9 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionInfo; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java index d28a533314df..04f7d74b950b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java @@ -27,8 +27,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.io.hfile.TestHFile; import org.apache.hadoop.hbase.master.assignment.AssignmentManager; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java index e6a79cec995c..508c2e63ab25 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java @@ -30,10 +30,10 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java index 809726759025..594141f7e802 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java @@ -26,9 +26,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java index b3a5b0384dd2..ef31c35dfca2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java @@ -59,9 +59,9 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination; diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java index d4f11144e7cb..19aa72487d5c 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java @@ -42,9 +42,9 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -1077,8 +1077,8 @@ public TRegionInfo getRegionInfo(ByteBuffer searchRow) throws IOError { getReverseScanResult(MetaTableName.getInstance().getName(), row, HConstants.CATALOG_FAMILY); if (startRowResult == null) { - throw new IOException( - "Cannot find row in " + MetaTableName.getInstance() + ", row=" + Bytes.toStringBinary(row)); + throw new IOException("Cannot find row in " + MetaTableName.getInstance() + ", row=" + + Bytes.toStringBinary(row)); } // find region start and end keys diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java index 9ab12c6daf18..a9fe61d1d4b8 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java @@ -19,9 +19,9 @@ import com.google.errorprone.annotations.RestrictedApi; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.master.RegionState; From e41984b9779ce08b3c7ac53706a30625e7977bc7 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Thu, 8 Jan 2026 11:24:19 -0500 Subject: [PATCH 07/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: Add dynamic meta table name discovery via ConnectionRegistry --- .../AbstractRpcBasedConnectionRegistry.java | 22 +++++++++++++ .../hadoop/hbase/client/AsyncConnection.java | 13 ++++++++ .../hbase/client/AsyncConnectionImpl.java | 14 ++++++-- .../hadoop/hbase/client/Connection.java | 13 ++++++++ .../hbase/client/ConnectionFactory.java | 33 +++++++++++++------ .../client/ConnectionOverAsyncConnection.java | 5 +++ .../hbase/client/ConnectionRegistry.java | 13 ++++++++ .../hbase/client/ZKConnectionRegistry.java | 10 ++++++ .../client/DoNothingConnectionRegistry.java | 6 ++++ .../client/TestAsyncAdminRpcPriority.java | 4 +-- .../client/TestAsyncConnectionTracing.java | 3 +- .../client/TestAsyncRegionLocatorTracing.java | 2 +- .../client/TestAsyncTableRpcPriority.java | 4 +-- .../hbase/client/TestAsyncTableTracing.java | 4 +-- .../hbase/client/TestMetricsConnection.java | 9 +++-- .../src/main/protobuf/server/Registry.proto | 13 ++++++++ .../hadoop/hbase/HBaseRpcServicesBase.java | 19 +++++++++++ .../apache/hadoop/hbase/HBaseServerBase.java | 8 +++++ .../client/AsyncClusterConnectionImpl.java | 5 +-- .../client/ClusterConnectionFactory.java | 4 ++- .../client/ConnectionRegistryEndpoint.java | 10 ++++++ .../hbase/client/SharedAsyncConnection.java | 5 +++ .../hadoop/hbase/client/SharedConnection.java | 5 +++ .../ShortCircuitConnectionRegistry.java | 5 +++ .../client/DummyAsyncClusterConnection.java | 5 +++ .../client/TestAsyncNonMetaRegionLocator.java | 2 +- ...ncNonMetaRegionLocatorConcurrenyLimit.java | 2 +- .../hbase/client/TestAsyncRegionLocator.java | 2 +- ...stAsyncSingleRequestRpcRetryingCaller.java | 5 +-- ...talogReplicaLoadBalanceSimpleSelector.java | 4 +-- .../hbase/client/TestFallbackToUseReplay.java | 4 +-- 31 files changed, 217 insertions(+), 36 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java index 62c6951b4535..c4778056c6c3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java @@ -34,8 +34,10 @@ import java.util.function.Predicate; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil; import org.apache.hadoop.hbase.exceptions.MasterRegistryFetchException; import org.apache.hadoop.hbase.ipc.HBaseRpcController; @@ -55,6 +57,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetClusterIdResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMetaRegionLocationsRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMetaRegionLocationsResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMetaTableNameRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMetaTableNameResponse; /** * Base class for rpc based connection registry implementation. @@ -250,6 +254,24 @@ public CompletableFuture getActiveMaster() { getClass().getSimpleName() + ".getActiveMaster"); } + @Override + public CompletableFuture getMetaTableName() { + return tracedFuture( + () -> this + . call( + (c, s, d) -> s.getMetaTableName(c, GetMetaTableNameRequest.getDefaultInstance(), d), + GetMetaTableNameResponse::hasTableName, "getMetaTableName()") + .thenApply(resp -> { + // If the response has a table name, parse it. Otherwise, return default. + if (resp.hasTableName() && !resp.getTableName().isEmpty()) { + return TableName.valueOf(resp.getTableName()); + } else { + return TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); + } + }), + getClass().getSimpleName() + ".getMetaTableName"); + } + @Override public void close() { trace(() -> { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java index 6e96918d1d9a..d858ffdc5aa6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java @@ -41,6 +41,19 @@ public interface AsyncConnection extends Closeable { */ Configuration getConfiguration(); + /** + * Returns the meta table name for this cluster. + *

+ * This value is fetched from the cluster during connection establishment and cached for the + * lifetime of this connection. For most clusters, this will be "hbase:meta". For read replica + * clusters or other specialized configurations, this may return a different table name. + *

+ * This method should be used instead of static references to meta table name to ensure + * compatibility with clusters that use custom meta table names. + * @return The meta table name for this cluster + */ + TableName getMetaTableName(); + /** * Retrieve a AsyncRegionLocator implementation to inspect region information on a table. The * returned AsyncRegionLocator is not thread-safe, so a new instance should be created for each diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java index 3f0e3e0b370e..cbe84222d953 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java @@ -92,6 +92,8 @@ public class AsyncConnectionImpl implements AsyncConnection { final ConnectionRegistry registry; + private final TableName metaTableName; + protected final int rpcTimeout; protected final RpcClient rpcClient; @@ -128,14 +130,16 @@ public class AsyncConnectionImpl implements AsyncConnection { private volatile ConnectionOverAsyncConnection conn; public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, String clusterId, - SocketAddress localAddress, User user) { - this(conf, registry, clusterId, localAddress, user, Collections.emptyMap()); + TableName metaTableName, SocketAddress localAddress, User user) { + this(conf, registry, clusterId, metaTableName, localAddress, user, Collections.emptyMap()); } public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, String clusterId, - SocketAddress localAddress, User user, Map connectionAttributes) { + TableName metaTableName, SocketAddress localAddress, User user, + Map connectionAttributes) { this.conf = conf; this.user = user; + this.metaTableName = metaTableName; this.metricsScope = MetricsConnection.getScope(conf, clusterId, this); if (user.isLoginFromKeytab()) { @@ -219,6 +223,10 @@ public Configuration getConfiguration() { return conf; } + public TableName getMetaTableName() { + return metaTableName; + } + @Override public boolean isClosed() { return closed.get(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java index 8220189d9b51..1737a787c8bc 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java @@ -62,6 +62,19 @@ public interface Connection extends Abortable, Closeable { /** Returns Configuration instance being used by this Connection instance. */ Configuration getConfiguration(); + /** + * Returns the meta table name for this cluster. + *

+ * This value is fetched from the cluster during connection establishment and cached for the + * lifetime of this connection. For most clusters, this will be "hbase:meta". For read replica + * clusters or other specialized configurations, this may return a different table name. + *

+ * This method should be used instead of static references to meta table name to ensure + * compatibility with clusters that use custom meta table names. + * @return The meta table name for this cluster + */ + TableName getMetaTableName(); + /** * Retrieve a Table implementation for accessing a table. The returned Table is not thread safe, a * new instance should be created for each using thread. This is a lightweight operation, pooling diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java index 144a790c406d..8e304fcbcac9 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java @@ -595,16 +595,29 @@ public static CompletableFuture createAsyncConnection(URI conne future.completeExceptionally(new IOException("clusterid came back null")); return; } - Class clazz = appliedConf.getClass( - HBASE_CLIENT_ASYNC_CONNECTION_IMPL, AsyncConnectionImpl.class, AsyncConnection.class); - try { - future.complete(user.runAs((PrivilegedExceptionAction< - ? extends AsyncConnection>) () -> ReflectionUtils.newInstance(clazz, appliedConf, - registry, clusterId, null, user, connectionAttributes))); - } catch (Exception e) { - registry.close(); - future.completeExceptionally(e); - } + // Fetch meta table name from registry + addListener(registry.getMetaTableName(), (metaTableName, metaError) -> { + if (metaError != null) { + registry.close(); + future.completeExceptionally(metaError); + return; + } + if (metaTableName == null) { + registry.close(); + future.completeExceptionally(new IOException("meta table name came back null")); + return; + } + Class clazz = appliedConf.getClass( + HBASE_CLIENT_ASYNC_CONNECTION_IMPL, AsyncConnectionImpl.class, AsyncConnection.class); + try { + future.complete(user.runAs((PrivilegedExceptionAction< + ? extends AsyncConnection>) () -> ReflectionUtils.newInstance(clazz, appliedConf, + registry, clusterId, metaTableName, null, user, connectionAttributes))); + } catch (Exception e) { + registry.close(); + future.completeExceptionally(e); + } + }); }); return future; }, "ConnectionFactory.createAsyncConnection"); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java index 471cfa874458..ad7483600ab6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionOverAsyncConnection.java @@ -88,6 +88,11 @@ public Configuration getConfiguration() { return conn.getConfiguration(); } + @Override + public TableName getMetaTableName() { + return conn.getMetaTableName(); + } + @Override public BufferedMutator getBufferedMutator(BufferedMutatorParams params) throws IOException { AsyncBufferedMutatorBuilder builder = conn.getBufferedMutatorBuilder(params.getTableName()); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java index 2ace3959ffa6..a15778fbcb3e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java @@ -21,6 +21,7 @@ import java.util.concurrent.CompletableFuture; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; import org.apache.yetus.audience.InterfaceAudience; /** @@ -48,6 +49,18 @@ public interface ConnectionRegistry extends Closeable { */ CompletableFuture getActiveMaster(); + /** + * Get the name of the meta table for this cluster. + *

+ * Should only be called once, similar to {@link #getClusterId()}. The upper layer should cache + * this value as it will not change during the connection lifetime. + *

+ * For most clusters, this will return the default "hbase:meta". For read replica clusters or + * other specialized configurations, this may return a different table name. + * @return CompletableFuture containing the meta table name + */ + CompletableFuture getMetaTableName(); + /** * Return the connection string associated with this registry instance. This value is * informational, used for annotating traces. Values returned may not be valid for establishing a diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java index ebb43723b8f8..36734ba3c97f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java @@ -35,8 +35,10 @@ import org.apache.hadoop.hbase.ClusterId; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.security.User; @@ -262,6 +264,14 @@ public CompletableFuture getActiveMaster() { "ZKConnectionRegistry.getActiveMaster"); } + @Override + public CompletableFuture getMetaTableName() { + return tracedFuture( + () -> CompletableFuture.completedFuture( + TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta")), + "ZKConnectionRegistry.getMetaTableName"); + } + @Override public String getConnectionString() { final String serverList = zk.getConnectString(); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java index 30d69d4b3f9e..7f87e7e29b72 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java @@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.security.User; import org.apache.yetus.audience.InterfaceAudience; @@ -48,6 +49,11 @@ public CompletableFuture getActiveMaster() { return CompletableFuture.completedFuture(null); } + @Override public CompletableFuture getMetaTableName() { + return CompletableFuture.completedFuture(null); + + } + @Override public String getConnectionString() { return "nothing"; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java index 2206a800767e..7fa6cf7fb108 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java @@ -144,8 +144,8 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(adminStub).stopServer(any(HBaseRpcController.class), any(StopServerRequest.class), any()); User user = UserProvider.instantiate(CONF).getCurrent(); - conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF, user), "test", null, - user) { + conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF, user), "test", + org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, user) { @Override CompletableFuture getMasterStub() { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java index e56fffbb2642..24e30098984f 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java @@ -72,7 +72,8 @@ public CompletableFuture getActiveMaster() { return CompletableFuture.completedFuture(masterServer); } }; - conn = new AsyncConnectionImpl(CONF, registry, "test", null, user); + conn = new AsyncConnectionImpl(CONF, registry, "test", + org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, user); } @After diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java index d519870080bf..787f0454e5dc 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java @@ -100,7 +100,7 @@ public void setUp() throws IOException { public CompletableFuture getMetaRegionLocations() { return CompletableFuture.completedFuture(locs); } - }, "test", null, user); + }, "test", org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, user); } @After diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java index 34e9ba201838..dcf51d038f35 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java @@ -163,8 +163,8 @@ public Void answer(InvocationOnMock invocation) throws Throwable { } }).when(stub).get(any(HBaseRpcController.class), any(GetRequest.class), any()); User user = UserProvider.instantiate(CONF).getCurrent(); - conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF, user), "test", null, - user) { + conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF, user), "test", + org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, user) { @Override AsyncRegionLocator getLocator() { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java index 2cecc974b6ef..6a1fc45737fd 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java @@ -209,8 +209,8 @@ public Void answer(InvocationOnMock invocation) throws Throwable { } }).when(stub).get(any(HBaseRpcController.class), any(GetRequest.class), any()); final User user = UserProvider.instantiate(CONF).getCurrent(); - conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF, user), "test", null, - user) { + conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF, user), "test", + org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, user) { @Override AsyncRegionLocator getLocator() { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java index e0d18f6bbb7e..40ec35649e4e 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java @@ -106,13 +106,15 @@ public void testMetricsConnectionScope() throws IOException { String scope = "testScope"; conf.setBoolean(MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY, true); - AsyncConnectionImpl impl = new AsyncConnectionImpl(conf, null, "foo", null, User.getCurrent()); + AsyncConnectionImpl impl = new AsyncConnectionImpl(conf, null, "foo", + org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()); Optional metrics = impl.getConnectionMetrics(); assertTrue("Metrics should be present", metrics.isPresent()); assertEquals(clusterId + "@" + Integer.toHexString(impl.hashCode()), metrics.get().getMetricScope()); conf.set(MetricsConnection.METRICS_SCOPE_KEY, scope); - impl = new AsyncConnectionImpl(conf, null, "foo", null, User.getCurrent()); + impl = new AsyncConnectionImpl(conf, null, "foo", + org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()); metrics = impl.getConnectionMetrics(); assertTrue("Metrics should be present", metrics.isPresent()); @@ -132,7 +134,8 @@ public void testMetricsWithMultiConnections() throws IOException { AsyncConnectionImpl impl; List connList = new ArrayList(); for (int i = 0; i < num; i++) { - impl = new AsyncConnectionImpl(conf, null, null, null, user); + impl = new AsyncConnectionImpl(conf, null, null, + org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, user); connList.add(impl); } diff --git a/hbase-protocol-shaded/src/main/protobuf/server/Registry.proto b/hbase-protocol-shaded/src/main/protobuf/server/Registry.proto index f55b892413b2..e6fc063ab0da 100644 --- a/hbase-protocol-shaded/src/main/protobuf/server/Registry.proto +++ b/hbase-protocol-shaded/src/main/protobuf/server/Registry.proto @@ -73,6 +73,14 @@ message GetBootstrapNodesResponse { repeated ServerName server_name = 1; } +/** Request and response to get the meta table name for this cluster */ +message GetMetaTableNameRequest { +} +message GetMetaTableNameResponse { + /** The name of the meta table. Defaults to "hbase:meta" if not set. */ + optional string table_name = 1; +} + /** * Implements all the RPCs needed by clients to look up cluster meta information needed for * connection establishment. @@ -105,6 +113,11 @@ service ClientMetaService { * Get nodes which could be used as ClientMetaService */ rpc GetBootstrapNodes(GetBootstrapNodesRequest) returns (GetBootstrapNodesResponse); + + /** + * Get the meta table name for this cluster. + */ + rpc GetMetaTableName(GetMetaTableNameRequest) returns(GetMetaTableNameResponse); } message GetConnectionRegistryRequest { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java index d6d277808838..6a6167272359 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java @@ -89,6 +89,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMastersResponseEntry; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMetaRegionLocationsRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMetaRegionLocationsResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMetaTableNameRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegistryProtos.GetMetaTableNameResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload; /** @@ -381,6 +383,23 @@ public final GetBootstrapNodesResponse getBootstrapNodes(RpcController controlle return builder.build(); } + @Override + public final GetMetaTableNameResponse getMetaTableName(RpcController controller, + GetMetaTableNameRequest request) throws ServiceException { + GetMetaTableNameResponse.Builder builder = GetMetaTableNameResponse.newBuilder(); + + try { + TableName metaTableName = server.getMetaTableName(); + if (metaTableName != null) { + builder.setTableName(metaTableName.getNameAsString()); + } + } catch (Exception e) { + throw new ServiceException(e); + } + + return builder.build(); + } + @Override @QosPriority(priority = HConstants.ADMIN_QOS) public UpdateConfigurationResponse updateConfiguration(RpcController controller, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java index c2f65edd4fda..9f2304f3844c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java @@ -675,6 +675,14 @@ public String toString() { return getServerName().toString(); } + @Override + public org.apache.hadoop.hbase.TableName getMetaTableName() { + // For now, always return the default meta table name. + // Future implementations may support custom meta table names from configuration or storage. + return org.apache.hadoop.hbase.TableName.valueOf( + org.apache.hadoop.hbase.NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); + } + protected abstract CoprocessorHost getCoprocessorHost(); protected abstract boolean canCreateBaseZNode(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java index 20b915288c61..9d232e34aa31 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java @@ -59,8 +59,9 @@ class AsyncClusterConnectionImpl extends AsyncConnectionImpl implements AsyncClusterConnection { public AsyncClusterConnectionImpl(Configuration conf, ConnectionRegistry registry, - String clusterId, SocketAddress localAddress, User user) { - super(conf, registry, clusterId, localAddress, user, Collections.emptyMap()); + String clusterId, org.apache.hadoop.hbase.TableName metaTableName, SocketAddress localAddress, + User user) { + super(conf, registry, clusterId, metaTableName, localAddress, user, Collections.emptyMap()); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java index 70a1e703c667..cd22d3ecd16d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java @@ -42,13 +42,15 @@ private ClusterConnectionFactory() { private static AsyncClusterConnection createAsyncClusterConnection(Configuration conf, ConnectionRegistry registry, SocketAddress localAddress, User user) throws IOException { String clusterId = FutureUtils.get(registry.getClusterId()); + org.apache.hadoop.hbase.TableName metaTableName = + FutureUtils.get(registry.getMetaTableName()); Class clazz = conf.getClass(HBASE_SERVER_CLUSTER_CONNECTION_IMPL, AsyncClusterConnectionImpl.class, AsyncClusterConnection.class); try { return user .runAs((PrivilegedExceptionAction) () -> ReflectionUtils - .newInstance(clazz, conf, registry, clusterId, localAddress, user)); + .newInstance(clazz, conf, registry, clusterId, metaTableName, localAddress, user)); } catch (Exception e) { throw new IOException(e); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryEndpoint.java index a75faf3db75b..9071a75d10f0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistryEndpoint.java @@ -22,6 +22,7 @@ import java.util.Optional; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; import org.apache.yetus.audience.InterfaceAudience; /** @@ -54,4 +55,13 @@ public interface ConnectionRegistryEndpoint { * Get the location of meta regions. */ List getMetaLocations(); + + /** + * Get the name of the meta table for this cluster. + *

+ * By default, this returns "hbase:meta". Future implementations may support custom meta table + * names for read replica clusters. + * @return The meta table name + */ + TableName getMetaTableName(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/SharedAsyncConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/SharedAsyncConnection.java index ebffc7ee5111..b19d86be19b2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/SharedAsyncConnection.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/SharedAsyncConnection.java @@ -52,6 +52,11 @@ public Configuration getConfiguration() { return conn.getConfiguration(); } + @Override + public TableName getMetaTableName() { + return conn.getMetaTableName(); + } + @Override public AsyncTableRegionLocator getRegionLocator(TableName tableName) { return conn.getRegionLocator(tableName); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/SharedConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/SharedConnection.java index ae52df266cfb..a9276dbaaf5a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/SharedConnection.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/SharedConnection.java @@ -61,6 +61,11 @@ public Configuration getConfiguration() { return this.conn.getConfiguration(); } + @Override + public TableName getMetaTableName() { + return this.conn.getMetaTableName(); + } + @Override public BufferedMutator getBufferedMutator(TableName tableName) throws IOException { return this.conn.getBufferedMutator(tableName); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitConnectionRegistry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitConnectionRegistry.java index 1eb4e2d08ea8..e257bd029607 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitConnectionRegistry.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitConnectionRegistry.java @@ -68,6 +68,11 @@ public CompletableFuture getActiveMaster() { return future; } + @Override + public CompletableFuture getMetaTableName() { + return CompletableFuture.completedFuture(endpoint.getMetaTableName()); + } + @Override public String getConnectionString() { return "short-circuit"; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncClusterConnection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncClusterConnection.java index cb54e6e72634..db3ab3496f97 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncClusterConnection.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncClusterConnection.java @@ -44,6 +44,11 @@ public Configuration getConfiguration() { return null; } + @Override + public TableName getMetaTableName() { + return TableName.META_TABLE_NAME; + } + @Override public AsyncTableRegionLocator getRegionLocator(TableName tableName) { return null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java index 4da4c87daba3..9f4cb8f3a4bb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java @@ -131,7 +131,7 @@ public void setUpBeforeTest() throws InterruptedException, ExecutionException, I ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); conn = - new AsyncConnectionImpl(c, registry, registry.getClusterId().get(), null, User.getCurrent()); + new AsyncConnectionImpl(c, registry, registry.getClusterId().get(), org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()); locator = new AsyncNonMetaRegionLocator(conn, AsyncConnectionImpl.RETRY_TIMER); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java index 4529c07dfd13..70073cae8ad5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java @@ -127,7 +127,7 @@ public static void setUp() throws Exception { ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, - registry.getClusterId().get(), null, User.getCurrent()); + registry.getClusterId().get(), org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()); LOCATOR = new AsyncNonMetaRegionLocator(CONN, AsyncConnectionImpl.RETRY_TIMER); SPLIT_KEYS = IntStream.range(1, 256).mapToObj(i -> Bytes.toBytes(String.format("%02x", i))) .toArray(byte[][]::new); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java index 5e858eb8d9d6..cce2c4fa735b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java @@ -102,7 +102,7 @@ public static void setUp() throws Exception { ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, - registry.getClusterId().get(), null, User.getCurrent()); + registry.getClusterId().get(), org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()); LOCATOR = CONN.getLocator(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java index baa4ee74ade9..c243901f4497 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java @@ -75,7 +75,7 @@ public static void setUpBeforeClass() throws Exception { ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, - registry.getClusterId().get(), null, User.getCurrent()); + registry.getClusterId().get(), org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()); } @AfterClass @@ -165,7 +165,8 @@ void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) { } }; try (AsyncConnectionImpl mockedConn = new AsyncConnectionImpl(CONN.getConfiguration(), - CONN.registry, CONN.registry.getClusterId().get(), null, User.getCurrent()) { + CONN.registry, CONN.registry.getClusterId().get(), + org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()) { @Override AsyncRegionLocator getLocator() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java index a392ff5c9a69..4512c060cee7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java @@ -78,8 +78,8 @@ public static void setUp() throws Exception { >= numOfMetaReplica); registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); - CONN = new AsyncConnectionImpl(conf, registry, registry.getClusterId().get(), null, - User.getCurrent()); + CONN = new AsyncConnectionImpl(conf, registry, registry.getClusterId().get(), + org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFallbackToUseReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFallbackToUseReplay.java index 44c096f0b8a9..b35bbc7ceba0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFallbackToUseReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFallbackToUseReplay.java @@ -98,8 +98,8 @@ public static void setUpBeforeClass() throws IOException { done.run(null); return null; }).when(stub).replay(any(), any(), any()); - CONN = new AsyncClusterConnectionImpl(CONF, mock(ConnectionRegistry.class), "test", null, - User.getCurrent()) { + CONN = new AsyncClusterConnectionImpl(CONF, mock(ConnectionRegistry.class), "test", + org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()) { @Override AsyncRegionLocator getLocator() { From be38a8c544a2a60be347dc89e9efba6ae6cd15f3 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Mon, 12 Jan 2026 12:39:48 -0500 Subject: [PATCH 08/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: Rename references --- .../backup/impl/IncrementalBackupManager.java | 2 +- .../favored/FavoredNodeAssignmentHelper.java | 2 +- .../SnapshotOfRegionAssignmentFromMeta.java | 6 +- ...rgeClusterBalancingMetaTableIsolation.java | 4 +- ...gTableIsolationAndReplicaDistribution.java | 4 +- .../client/AsyncNonMetaRegionLocator.java | 6 +- .../hbase/client/AsyncRegionLocator.java | 2 +- .../client/AsyncTableRegionLocatorImpl.java | 2 +- .../hbase/client/RawAsyncHBaseAdmin.java | 6 +- .../hbase/security/token/ClientTokenUtil.java | 4 +- .../client/TestAsyncAdminRpcPriority.java | 2 +- .../client/TestAsyncRegionLocatorTracing.java | 14 ++--- .../client/TestAsyncTableRpcPriority.java | 20 +++---- .../hbase/client/TestCompactFromClient.java | 2 +- .../hbase/client/TestRegionInfoBuilder.java | 2 +- .../client/TestTableDescriptorBuilder.java | 4 +- .../hadoop/hbase/TestCellComparator.java | 24 ++++---- .../org/apache/hadoop/hbase/TestKeyValue.java | 24 ++++---- .../hbase/TestClientClusterMetrics.java | 22 ++++---- .../util/ProcessBasedLocalHBaseCluster.java | 2 +- .../hadoop/hbase/util/RestartMetaTest.java | 2 +- .../hbase/IntegrationTestMetaReplicas.java | 2 +- .../hbase/mttr/IntegrationTestMTTR.java | 2 +- .../hbase/mapreduce/TestImportExport.java | 2 +- .../hadoop/hbase/rest/TestStatusResource.java | 2 +- .../model/TestStorageClusterStatusModel.java | 4 +- .../master/RegionPlacementMaintainer.java | 8 +-- .../hbase/master/TableStateManager.java | 4 +- .../master/assignment/AssignmentManager.java | 4 +- .../MergeTableRegionsProcedure.java | 2 +- .../master/assignment/RegionStateStore.java | 23 ++++---- .../assignment/SplitTableRegionProcedure.java | 2 +- .../hadoop/hbase/master/http/MetaBrowser.java | 2 +- .../hbase/master/janitor/CatalogJanitor.java | 8 ++- .../master/janitor/ReportMakingVisitor.java | 2 +- .../procedure/DeleteTableProcedure.java | 4 +- .../procedure/DisableTableProcedure.java | 4 +- .../procedure/HBCKServerCrashProcedure.java | 4 +- .../MigrateNamespaceTableProcedure.java | 4 +- .../procedure/ModifyTableProcedure.java | 2 +- .../hbase/regionserver/RSRpcServices.java | 2 +- .../apache/hadoop/hbase/util/HBaseFsck.java | 56 +++++++++---------- .../apache/hadoop/hbase/util/RegionMover.java | 4 +- .../hbase-webapps/master/catalogTables.jsp | 2 +- .../resources/hbase-webapps/master/table.jsp | 8 +-- .../hadoop/hbase/HBaseClusterInterface.java | 2 +- .../apache/hadoop/hbase/HBaseTestingUtil.java | 16 +++--- .../hadoop/hbase/TestHBaseMetaEdit.java | 18 +++--- .../apache/hadoop/hbase/TestInfoServers.java | 2 +- .../TestMetaUpdatesGoToPriorityQueue.java | 2 +- .../apache/hadoop/hbase/TestNamespace.java | 2 +- .../hbase/TestServerInternalsTracing.java | 2 +- .../client/AbstractTestRegionLocator.java | 18 +++--- .../client/MetaWithReplicasTestBase.java | 2 +- .../hbase/client/RegionReplicaTestHelper.java | 2 +- .../apache/hadoop/hbase/client/TestAdmin.java | 2 +- .../hadoop/hbase/client/TestAdmin2.java | 8 +-- .../TestAsyncAdminWithRegionReplicas.java | 8 +-- .../client/TestAsyncMetaRegionLocator.java | 4 +- .../client/TestAsyncNonMetaRegionLocator.java | 4 +- .../client/TestAsyncRegionAdminApi2.java | 14 ++--- .../hbase/client/TestAsyncTableAdminApi.java | 6 +- .../hbase/client/TestAsyncTableAdminApi2.java | 2 +- .../hbase/client/TestAsyncTableAdminApi3.java | 18 +++--- .../client/TestAsyncTableUseMetaReplicas.java | 4 +- ...talogReplicaLoadBalanceSimpleSelector.java | 16 +++--- .../hbase/client/TestCleanupMetaReplica.java | 2 +- .../client/TestClientScannerTimeouts.java | 2 +- .../client/TestClientSideRegionScanner.java | 8 +-- .../hadoop/hbase/client/TestEnableTable.java | 2 +- .../hbase/client/TestFromClientSide5.java | 2 +- .../TestIncreaseMetaReplicaThroughConfig.java | 2 +- .../hbase/client/TestMasterRegistry.java | 2 +- .../hadoop/hbase/client/TestMetaCache.java | 2 +- .../client/TestMetaRegionLocationCache.java | 2 +- .../client/TestMetaWithReplicasBasic.java | 2 +- .../TestMetaWithReplicasShutdownHandling.java | 4 +- .../TestMultiActionMetricsFromClient.java | 2 +- .../hbase/client/TestReplicaWithCluster.java | 2 +- .../client/TestRpcConnectionRegistry.java | 2 +- .../client/TestSeparateClientZKCluster.java | 8 +-- .../TestShutdownOfMetaReplicaHolder.java | 2 +- .../hbase/client/TestSnapshotFromClient.java | 2 +- .../client/TestZKConnectionRegistry.java | 4 +- .../hadoop/hbase/http/TestInfoServersACL.java | 4 +- .../hadoop/hbase/master/TestMaster.java | 2 +- ...MasterFileSystemWithStoreFileTracking.java | 2 +- ...TestMasterOperationsForRegionReplicas.java | 2 +- .../hbase/master/TestMasterRepairMode.java | 4 +- .../hbase/master/TestMasterTransitions.java | 2 +- .../TestMetaAssignmentWithStopMaster.java | 2 +- .../TestMigrateAndMirrorMetaLocations.java | 8 +-- .../master/TestMigrateNamespaceTable.java | 4 +- .../hbase/master/TestRecreateCluster.java | 4 +- .../TestRestartWithEmptyWALDirectory.java | 2 +- ...ServerCrashProcedureCarryingMetaStuck.java | 4 +- .../TestCloseRegionWhileRSCrash.java | 6 +- .../TestOpenRegionProcedureBackoff.java | 2 +- .../assignment/TestRegionStateStore.java | 2 +- .../master/assignment/TestRollbackSCP.java | 2 +- .../BalancerConditionalsTestUtil.java | 2 +- .../TestFavoredStochasticLoadBalancer.java | 2 +- ...MetaTableIsolationBalancerConditional.java | 4 +- .../TestReplicationBarrierCleaner.java | 12 ++-- .../TestCatalogJanitorInMemoryStates.java | 2 +- .../janitor/TestMetaFixerNoCluster.java | 16 +++--- .../TestSimpleRegionNormalizer.java | 2 +- .../hbase/master/procedure/TestHBCKSCP.java | 4 +- .../procedure/TestProcedurePriority.java | 2 +- ...TestTableProcedureWaitingQueueCleanup.java | 2 +- .../TestCompactionInDeadRegionServer.java | 2 +- .../regionserver/TestDefaultMemStore.java | 2 +- .../TestEndToEndSplitTransaction.java | 2 +- .../TestGetClosestAtOrBefore.java | 2 +- .../TestReadAndWriteRegionInfoFile.java | 4 +- .../hbase/regionserver/TestRegionInfo.java | 34 +++++------ .../regionserver/TestRegionReplicas.java | 2 +- .../TestRegionServerCrashDisableWAL.java | 2 +- .../TestRegionServerNoMaster.java | 4 +- .../TestRegionServerRejectDuringAbort.java | 2 +- .../TestShutdownWhileWALBroken.java | 2 +- .../wal/AbstractTestLogRolling.java | 2 +- .../regionserver/wal/TestLogRollAbort.java | 2 +- .../regionserver/wal/TestLogRolling.java | 2 +- .../wal/TestLogRollingNoCluster.java | 6 +- .../TestReplicationWALEntryFilters.java | 2 +- .../TestMetaRegionReplicaReplication.java | 26 ++++----- .../regionserver/TestReplicationSource.java | 2 +- .../TestSerialReplicationChecker.java | 6 +- .../rsgroup/TestRSGroupsCPHookCalled.java | 2 +- .../hbase/rsgroup/TestRSGroupsKillRS.java | 4 +- .../security/access/TestRpcAccessChecks.java | 4 +- .../token/TestGenerateDelegationToken.java | 2 +- .../snapshot/TestRegionSnapshotTask.java | 2 +- .../hadoop/hbase/util/BaseTestHBaseFsck.java | 8 +-- .../hbase/util/TestFSTableDescriptors.java | 6 +- ...TestHBaseFsckCleanReplicationBarriers.java | 2 +- .../hadoop/hbase/util/TestHBaseFsckMOB.java | 2 +- .../hadoop/hbase/util/TestRegionMover1.java | 2 +- .../TestRegionMoverWithRSGroupEnable.java | 2 +- .../apache/hadoop/hbase/wal/TestWALSplit.java | 4 +- .../hadoop/hbase/HBaseTestingUtility.java | 20 +++---- .../thrift2/client/ThriftConnection.java | 5 ++ 143 files changed, 399 insertions(+), 389 deletions(-) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java index ec5a04a63a05..f6454a737a3b 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java @@ -170,7 +170,7 @@ private List getLogFilesForNewBackup(Map olderTimestamps, LOG.debug("currentLogFile: " + log.getPath().toString()); if (AbstractFSWALProvider.isMetaFile(log.getPath())) { if (LOG.isDebugEnabled()) { - LOG.debug("Skip {} log file: {}", MetaTableName.getInstance(), log.getPath().getName()); + LOG.debug("Skip {} log file: {}", conn.getMetaTableName(), log.getPath().getName()); } continue; } diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java index d316075979a1..88d5ac88caed 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java @@ -131,7 +131,7 @@ public static void updateMetaWithFavoredNodesInfo( puts.add(put); } } - try (Table table = connection.getTable(MetaTableName.getInstance())) { + try (Table table = connection.getTable(connection.getMetaTableName())) { table.put(puts); } LOG.info("Added " + puts.size() + " region favored nodes in META"); diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java index e54ce7478e99..0a40e9dcdf8b 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java @@ -172,9 +172,9 @@ private void processMetaRecord(Result result) throws IOException { */ public void initialize() throws IOException { LOG.info("Start to scan {} for the current region assignment snapshot", - MetaTableName.getInstance()); + connection.getMetaTableName()); // Scan hbase:meta to pick up user regions - try (Table metaTable = connection.getTable(MetaTableName.getInstance()); + try (Table metaTable = connection.getTable(connection.getMetaTableName()); ResultScanner scanner = metaTable.getScanner(HConstants.CATALOG_FAMILY)) { for (;;) { Result result = scanner.next(); @@ -190,7 +190,7 @@ public void initialize() throws IOException { } } LOG.info("Finished scanning {} for the current region assignment snapshot", - MetaTableName.getInstance()); + connection.getMetaTableName()); } private void addRegion(RegionInfo regionInfo) { diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java index 1843cb7b895d..b8e144b4ac85 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java @@ -71,7 +71,7 @@ public static void setup() { // Create regions List allRegions = new ArrayList<>(); for (int i = 0; i < NUM_REGIONS; i++) { - TableName tableName = i < 3 ? MetaTableName.getInstance() : NON_META_TABLE_NAME; + TableName tableName = i < 3 ? connection.getMetaTableName() : NON_META_TABLE_NAME; byte[] startKey = new byte[1]; startKey[0] = (byte) i; byte[] endKey = new byte[1]; @@ -99,7 +99,7 @@ public void testMetaTableIsolation() { } private boolean isMetaTableIsolated(BalancerClusterState cluster) { - return isTableIsolated(cluster, MetaTableName.getInstance(), "Meta"); + return isTableIsolated(cluster, connection.getMetaTableName(), "Meta"); } } diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java index 4c8e9ffd5526..94f38e0d8cca 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java @@ -75,7 +75,7 @@ public static void setup() { for (int i = 0; i < NUM_REGIONS; i++) { TableName tableName; if (i < 1) { - tableName = MetaTableName.getInstance(); + tableName = connection.getMetaTableName(); } else if (i < 10) { tableName = SYSTEM_TABLE_NAME; } else { @@ -120,7 +120,7 @@ public void testTableIsolationAndReplicaDistribution() { * Validates whether all meta table regions are isolated. */ private boolean isMetaTableIsolated(BalancerClusterState cluster) { - return isTableIsolated(cluster, MetaTableName.getInstance(), "Meta"); + return isTableIsolated(cluster, connection.getMetaTableName(), "Meta"); } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java index 2ef315d2b4ac..9877810be5b9 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java @@ -238,7 +238,7 @@ private boolean tryComplete(LocateRequest req, CompletableFuture { + .createSelector(replicaSelectorClass, conn.getMetaTableName(), conn, () -> { int numOfReplicas = CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS; try { RegionLocations metaLocations = conn.registry.getMetaRegionLocations() @@ -246,7 +246,7 @@ private boolean tryComplete(LocateRequest req, CompletableFuture { LOG.debug("Clear meta cache for {}", tableName); - if (tableName.equals(MetaTableName.getInstance())) { + if (tableName.equals(conn.getMetaTableName())) { metaRegionLocator.clearCache(); } else { nonMetaRegionLocator.clearCache(tableName); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java index 32349a64651d..c0d297460330 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java @@ -64,7 +64,7 @@ public CompletableFuture> getAllRegionLocations() { .thenApply(locs -> Arrays.asList(locs.getRegionLocations())); } CompletableFuture> future = ClientMetaTableAccessor - .getTableHRegionLocations(conn.getTable(MetaTableName.getInstance()), tableName); + .getTableHRegionLocations(conn.getTable(conn.getMetaTableName()), tableName); addListener(future, (locs, error) -> locs.forEach(loc -> { // the cache assumes that all locations have a serverName. only add if that's true if (loc.getServerName() != null) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index 818d3be9913d..b09d6360a322 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -405,7 +405,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin { AsyncAdminBuilderBase builder) { this.connection = connection; this.retryTimer = retryTimer; - this.metaTable = connection.getTable(MetaTableName.getInstance()); + this.metaTable = connection.getTable(connection.getMetaTableName()); this.rpcTimeoutNs = builder.rpcTimeoutNs; this.operationTimeoutNs = builder.operationTimeoutNs; this.pauseNs = builder.pauseNs; @@ -1012,7 +1012,7 @@ List> adminCall(controller, stub, @Override public CompletableFuture> getRegions(TableName tableName) { - if (tableName.equals(MetaTableName.getInstance())) { + if (tableName.equals(connection.getMetaTableName())) { return connection.registry.getMetaRegionLocations() .thenApply(locs -> Stream.of(locs.getRegionLocations()).map(HRegionLocation::getRegion) .collect(Collectors.toList())); @@ -1303,7 +1303,7 @@ private CompletableFuture compactRegion(byte[] regionName, byte[] columnFa * List all region locations for the specific table. */ private CompletableFuture> getTableHRegionLocations(TableName tableName) { - if (MetaTableName.getInstance().equals(tableName)) { + if (connection.getMetaTableName().equals(tableName)) { CompletableFuture> future = new CompletableFuture<>(); addListener(connection.registry.getMetaRegionLocations(), (metaRegions, err) -> { if (err != null) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java index 8f0c11a03f6b..6c7fcb1b8feb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java @@ -73,7 +73,7 @@ private static void injectFault() throws ServiceException { future.completeExceptionally(ProtobufUtil.handleRemoteException(injectedException)); return future; } - AsyncTable table = conn.getTable(MetaTableName.getInstance()); + AsyncTable table = conn.getTable(conn.getMetaTableName()); table. coprocessorService( AuthenticationProtos.AuthenticationService::newStub, @@ -102,7 +102,7 @@ static Token obtainToken(Connection conn) throws try { injectFault(); - meta = conn.getTable(MetaTableName.getInstance()); + meta = conn.getTable(conn.getMetaTableName()); CoprocessorRpcChannel rpcChannel = meta.coprocessorService(HConstants.EMPTY_START_ROW); AuthenticationProtos.AuthenticationService.BlockingInterface service = AuthenticationProtos.AuthenticationService.newBlockingStub(rpcChannel); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java index 7fa6cf7fb108..c9eb597da204 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java @@ -196,7 +196,7 @@ public void testCreateSystemTable() { // that we pass the correct priority @Test public void testCreateMetaTable() { - conn.getAdmin().createTable(TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()) + conn.getAdmin().createTable(TableDescriptorBuilder.newBuilder(connection.getMetaTableName()) .setColumnFamily(ColumnFamilyDescriptorBuilder.of("cf")).build()).join(); verify(masterStub, times(1)).createTable(assertPriority(SYSTEMTABLE_QOS), any(CreateTableRequest.class), any()); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java index 787f0454e5dc..6215fea6853e 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java @@ -85,7 +85,7 @@ public class TestAsyncRegionLocatorTracing { @Before public void setUp() throws IOException { - RegionInfo metaRegionInfo = RegionInfoBuilder.newBuilder(MetaTableName.getInstance()).build(); + RegionInfo metaRegionInfo = RegionInfoBuilder.newBuilder(connection.getMetaTableName()).build(); locs = new RegionLocations( new HRegionLocation(metaRegionInfo, ServerName.valueOf("127.0.0.1", 12345, EnvironmentEdgeManager.currentTime())), @@ -147,30 +147,30 @@ public void testClearCacheServerName() { @Test public void testClearCacheTableName() { - conn.getLocator().clearCache(MetaTableName.getInstance()); + conn.getLocator().clearCache(connection.getMetaTableName()); SpanData span = waitSpan("AsyncRegionLocator.clearCache"); assertThat(span, allOf(hasStatusWithCode(StatusCode.OK), hasKind(SpanKind.INTERNAL), buildConnectionAttributesMatcher(conn), - buildTableAttributesMatcher(MetaTableName.getInstance()))); + buildTableAttributesMatcher(connection.getMetaTableName()))); } @Test public void testGetRegionLocation() { - conn.getLocator().getRegionLocation(MetaTableName.getInstance(), HConstants.EMPTY_START_ROW, + conn.getLocator().getRegionLocation(connection.getMetaTableName(), HConstants.EMPTY_START_ROW, RegionLocateType.CURRENT, TimeUnit.SECONDS.toNanos(1)).join(); SpanData span = waitSpan("AsyncRegionLocator.getRegionLocation"); assertThat(span, allOf(hasStatusWithCode(StatusCode.OK), hasKind(SpanKind.INTERNAL), buildConnectionAttributesMatcher(conn), - buildTableAttributesMatcher(MetaTableName.getInstance()), + buildTableAttributesMatcher(connection.getMetaTableName()), hasAttributes(containsEntryWithStringValuesOf("db.hbase.regions", locs.getDefaultRegionLocation().getRegion().getRegionNameAsString())))); } @Test public void testGetRegionLocations() { - conn.getLocator().getRegionLocations(MetaTableName.getInstance(), HConstants.EMPTY_START_ROW, + conn.getLocator().getRegionLocations(connection.getMetaTableName(), HConstants.EMPTY_START_ROW, RegionLocateType.CURRENT, false, TimeUnit.SECONDS.toNanos(1)).join(); SpanData span = waitSpan("AsyncRegionLocator.getRegionLocations"); String[] expectedRegions = @@ -178,7 +178,7 @@ public void testGetRegionLocations() { .map(RegionInfo::getRegionNameAsString).toArray(String[]::new); assertThat(span, allOf(hasStatusWithCode(StatusCode.OK), hasKind(SpanKind.INTERNAL), buildConnectionAttributesMatcher(conn), - buildTableAttributesMatcher(MetaTableName.getInstance()), hasAttributes( + buildTableAttributesMatcher(connection.getMetaTableName()), hasAttributes( containsEntryWithStringValuesOf("db.hbase.regions", containsInAnyOrder(expectedRegions))))); } } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java index dcf51d038f35..f3cfe1c2e7f7 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java @@ -238,7 +238,7 @@ public void testGetSystemTable() { @Test public void testGetMetaTable() { - conn.getTable(MetaTableName.getInstance()).get(new Get(Bytes.toBytes(0))).join(); + conn.getTable(conn.getMetaTableName()).get(new Get(Bytes.toBytes(0))).join(); verify(stub, times(1)).get(assertPriority(SYSTEMTABLE_QOS), any(GetRequest.class), any()); } @@ -269,7 +269,7 @@ public void testPutSystemTable() { @Test public void testPutMetaTable() { - conn.getTable(MetaTableName.getInstance()).put(new Put(Bytes.toBytes(0)) + conn.getTable(conn.getMetaTableName()).put(new Put(Bytes.toBytes(0)) .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v"))).join(); verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any()); } @@ -297,7 +297,7 @@ public void testDeleteSystemTable() { @Test public void testDeleteMetaTable() { - conn.getTable(MetaTableName.getInstance()).delete(new Delete(Bytes.toBytes(0))).join(); + conn.getTable(conn.getMetaTableName()).delete(new Delete(Bytes.toBytes(0))).join(); verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any()); } @@ -328,7 +328,7 @@ public void testAppendSystemTable() { @Test public void testAppendMetaTable() { - conn.getTable(MetaTableName.getInstance()).append(new Append(Bytes.toBytes(0)) + conn.getTable(conn.getMetaTableName()).append(new Append(Bytes.toBytes(0)) .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v"))).join(); verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any()); } @@ -356,7 +356,7 @@ public void testIncrementSystemTable() { @Test public void testIncrementMetaTable() { - conn.getTable(MetaTableName.getInstance()) + conn.getTable(conn.getMetaTableName()) .incrementColumnValue(Bytes.toBytes(0), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1).join(); verify(stub, times(1)).mutate(assertPriority(SYSTEMTABLE_QOS), any(MutateRequest.class), any()); } @@ -394,7 +394,7 @@ public void testCheckAndPutSystemTable() { @Test public void testCheckAndPutMetaTable() { - conn.getTable(MetaTableName.getInstance()).checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")) + conn.getTable(conn.getMetaTableName()).checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")) .qualifier(Bytes.toBytes("cq")).ifNotExists().thenPut(new Put(Bytes.toBytes(0)) .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v"))) .join(); @@ -427,7 +427,7 @@ public void testCheckAndDeleteSystemTable() { @Test public void testCheckAndDeleteMetaTable() { - conn.getTable(MetaTableName.getInstance()).checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")) + conn.getTable(conn.getMetaTableName()).checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")) .qualifier(Bytes.toBytes("cq")).ifNotExists().thenPut(new Put(Bytes.toBytes(0)) .addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v"))) .join(); @@ -468,7 +468,7 @@ public void testCheckAndMutateSystemTable() throws IOException { @Test public void testCheckAndMutateMetaTable() throws IOException { - conn.getTable(MetaTableName.getInstance()).checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")) + conn.getTable(conn.getMetaTableName()).checkAndMutate(Bytes.toBytes(0), Bytes.toBytes("cf")) .qualifier(Bytes.toBytes("cq")).ifEquals(Bytes.toBytes("v")) .thenMutate(new RowMutations(Bytes.toBytes(0)).add((Mutation) new Delete(Bytes.toBytes(0)))) .join(); @@ -556,7 +556,7 @@ public void testScanSystemTable() throws Exception { @Test public void testScanMetaTable() throws Exception { CompletableFuture renewFuture = mockScanReturnRenewFuture(SYSTEMTABLE_QOS); - testForTable(MetaTableName.getInstance(), renewFuture, Optional.empty()); + testForTable(connection.getMetaTableName(), renewFuture, Optional.empty()); } private void testForTable(TableName tableName, CompletableFuture renewFuture, @@ -599,7 +599,7 @@ public void testBatchSystemTable() { @Test public void testBatchMetaTable() { - conn.getTable(MetaTableName.getInstance()).batchAll(Arrays.asList(new Delete(Bytes.toBytes(0)))) + conn.getTable(conn.getMetaTableName()).batchAll(Arrays.asList(new Delete(Bytes.toBytes(0)))) .join(); verify(stub, times(1)).multi(assertPriority(SYSTEMTABLE_QOS), any(ClientProtos.MultiRequest.class), any()); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java index c8e6b2158ce1..3d4417947c52 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java @@ -94,7 +94,7 @@ public void testCompactTableWithNullLocations() throws Exception { mockedMeta.when(() -> ClientMetaTableAccessor.getTableHRegionLocations(any(AsyncTable.class), any(TableName.class))).thenReturn(nullLocationsFuture); AsyncTable metaTable = mock(AsyncTable.class); - when(connection.getTable(MetaTableName.getInstance())).thenReturn(metaTable); + when(connection.getTable(connection.getMetaTableName())).thenReturn(metaTable); HashedWheelTimer hashedWheelTimer = mock(HashedWheelTimer.class); AsyncAdminBuilderBase asyncAdminBuilderBase = mock(AsyncAdminBuilderBase.class); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java index e01b3b741dcc..8ed7b2739120 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java @@ -142,7 +142,7 @@ public void testContainsRange() { @Test public void testContainsRangeForMetaTable() { TableDescriptor tableDesc = - TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).build(); + TableDescriptorBuilder.newBuilder(connection.getMetaTableName()).build(); RegionInfo hri = RegionInfoBuilder.newBuilder(tableDesc.getTableName()).build(); byte[] startRow = HConstants.EMPTY_START_ROW; byte[] row1 = Bytes.toBytes("a,a,0"); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java index d09f7a225a6e..b94b73aedf59 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java @@ -60,7 +60,7 @@ public class TestTableDescriptorBuilder { @Test(expected = IOException.class) public void testAddCoprocessorTwice() throws IOException { String cpName = "a.b.c.d"; - TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).setCoprocessor(cpName) + TableDescriptorBuilder.newBuilder(connection.getMetaTableName()).setCoprocessor(cpName) .setCoprocessor(cpName).build(); } @@ -68,7 +68,7 @@ public void testAddCoprocessorTwice() throws IOException { public void testPb() throws DeserializationException, IOException { final int v = 123; TableDescriptor htd = - TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).setMaxFileSize(v) + TableDescriptorBuilder.newBuilder(connection.getMetaTableName()).setMaxFileSize(v) .setDurability(Durability.ASYNC_WAL).setReadOnly(true).setRegionReplication(2).build(); byte[] bytes = TableDescriptorBuilder.toByteArray(htd); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java index 5c143d8ee065..88b17cc051a5 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java @@ -197,36 +197,36 @@ public void testMetaComparisons2() { CellComparator c = MetaCellComparator.META_COMPARATOR; assertTrue(c.compare( createByteBufferKeyValueFromKeyValue(new KeyValue( - Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now)), + Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,1"), now)), createByteBufferKeyValueFromKeyValue(new KeyValue( - Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now))) == 0); + Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,1"), now))) == 0); Cell a = createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now)); + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,1"), now)); Cell b = createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,2"), now)); + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,2"), now)); assertTrue(c.compare(a, b) < 0); assertTrue(c.compare( createByteBufferKeyValueFromKeyValue(new KeyValue( - Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,2"), now)), + Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,2"), now)), createByteBufferKeyValueFromKeyValue(new KeyValue( - Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now))) > 0); + Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,1"), now))) > 0); assertTrue(c.compare( createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now)), + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,1"), now)), createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now))) + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,1"), now))) == 0); assertTrue(c.compare( createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now)), + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,1"), now)), createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,2"), now))) + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,2"), now))) < 0); assertTrue(c.compare( createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,2"), now)), + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,2"), now)), createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now))) + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,1"), now))) > 0); } diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java index 1e65b75a9777..24e751215149 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java @@ -198,32 +198,32 @@ public void testKeyValueBorderCases() { private void metacomparisons(final CellComparatorImpl c) { long now = EnvironmentEdgeManager.currentTime(); assertTrue(c.compare( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now), - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now)) + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,1"), now), + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,1"), now)) == 0); KeyValue a = - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now); + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,1"), now); KeyValue b = - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,2"), now); + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,2"), now); assertTrue(c.compare(a, b) < 0); assertTrue(c.compare( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,2"), now), - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",a,,0,1"), now)) + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,2"), now), + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,1"), now)) > 0); } private void comparisons(final CellComparatorImpl c) { long now = EnvironmentEdgeManager.currentTime(); assertTrue(c.compare( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now), - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now)) + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,1"), now), + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,1"), now)) == 0); assertTrue(c.compare( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now), - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,2"), now)) < 0); + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,1"), now), + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,2"), now)) < 0); assertTrue(c.compare( - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,2"), now), - new KeyValue(Bytes.toBytes(MetaTableName.getInstance().getNameAsString() + ",,1"), now)) > 0); + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,2"), now), + new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,1"), now)) > 0); } @Test diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java index 54ceeecfec21..6daf0f94be17 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java @@ -222,16 +222,16 @@ public void testRegionStatesCount() throws Exception { ClusterMetrics metrics = ADMIN.getClusterMetrics(); Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getRegionsInTransition(), + metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getRegionsInTransition(), 0); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getOpenRegions(), 1); + metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getOpenRegions(), 1); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getTotalRegions(), 1); + metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getTotalRegions(), 1); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getClosedRegions(), 0); + metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getClosedRegions(), 0); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getSplitRegions(), 0); + metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getSplitRegions(), 0); Assert.assertEquals( metrics.getTableRegionStatesCount().get(TABLE_NAME).getRegionsInTransition(), 0); Assert.assertEquals(metrics.getTableRegionStatesCount().get(TABLE_NAME).getOpenRegions(), 1); @@ -253,12 +253,12 @@ public void testRegionStatesWithSplit() throws Exception { ClusterMetrics metrics = ADMIN.getClusterMetrics(); Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getRegionsInTransition(), + metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getRegionsInTransition(), 0); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getOpenRegions(), 1); + metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getOpenRegions(), 1); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getTotalRegions(), 1); + metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getTotalRegions(), 1); Assert.assertEquals( metrics.getTableRegionStatesCount().get(TABLE_NAME).getRegionsInTransition(), 0); Assert.assertEquals(metrics.getTableRegionStatesCount().get(TABLE_NAME).getOpenRegions(), 1); @@ -273,12 +273,12 @@ public void testRegionStatesWithSplit() throws Exception { metrics = ADMIN.getClusterMetrics(); Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getRegionsInTransition(), + metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getRegionsInTransition(), 0); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getOpenRegions(), 1); + metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getOpenRegions(), 1); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(MetaTableName.getInstance()).getTotalRegions(), 1); + metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getTotalRegions(), 1); Assert.assertEquals( metrics.getTableRegionStatesCount().get(TABLE_NAME).getRegionsInTransition(), 0); Assert.assertEquals(metrics.getTableRegionStatesCount().get(TABLE_NAME).getOpenRegions(), 2); diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java index 267b78dade13..ea0d3cf16236 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java @@ -165,7 +165,7 @@ public void startHBase() throws IOException { int attemptsLeft = 10; while (attemptsLeft-- > 0) { try { - testUtil.getConnection().getTable(MetaTableName.getInstance()); + testUtil.getConnection().getTable(connection.getMetaTableName()); } catch (Exception e) { LOG.info("Waiting for HBase to startup. Retries left: " + attemptsLeft, e); Threads.sleep(1000); diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java index dc7d025796bd..4b56ed4c00c5 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java @@ -125,7 +125,7 @@ protected int doWork() throws Exception { LOG.debug("Trying to scan meta"); - Table metaTable = connection.getTable(MetaTableName.getInstance()); + Table metaTable = connection.getTable(connection.getMetaTableName()); ResultScanner scanner = metaTable.getScanner(new Scan()); Result result; while ((result = scanner.next()) != null) { diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java index fd07d7e1dc6a..57a635a02c14 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java @@ -56,7 +56,7 @@ public static void setUp() throws Exception { 1000); // Make sure there are three servers. util.initializeCluster(3); - HBaseTestingUtil.setReplicas(util.getAdmin(), MetaTableName.getInstance(), 3); + HBaseTestingUtil.setReplicas(util.getAdmin(), connection.getMetaTableName(), 3); ZKWatcher zkw = util.getZooKeeperWatcher(); Configuration conf = util.getConfiguration(); String baseZNode = diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java index fbc98f006393..3fe10281264d 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java @@ -185,7 +185,7 @@ private static void setupActions() throws IOException { // Set up the action that will move the regions of meta. moveMetaRegionsAction = new MoveRegionsOfTableAction(sleepTime, - MonkeyConstants.DEFAULT_MOVE_REGIONS_MAX_TIME, MetaTableName.getInstance()); + MonkeyConstants.DEFAULT_MOVE_REGIONS_MAX_TIME, connection.getMetaTableName()); // Set up the action that will move the regions of our table. moveRegionAction = new MoveRegionsOfTableAction(sleepTime, diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java index b861c29a9bcc..0e5d7ec4c572 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java @@ -257,7 +257,7 @@ public void testSimpleCase() throws Throwable { @Test public void testMetaExport() throws Throwable { String[] args = - new String[] { MetaTableName.getInstance().getNameAsString(), FQ_OUTPUT_DIR, "1", "0", "0" }; + new String[] { connection.getMetaTableName().getNameAsString(), FQ_OUTPUT_DIR, "1", "0", "0" }; assertTrue(runExport(args)); } diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java index 5ddc23d36554..c429bdb48f99 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java @@ -56,7 +56,7 @@ public class TestStatusResource { private static final Logger LOG = LoggerFactory.getLogger(TestStatusResource.class); - private static final byte[] META_REGION_NAME = Bytes.toBytes(MetaTableName.getInstance() + ",,1"); + private static final byte[] META_REGION_NAME = Bytes.toBytes(connection.getMetaTableName() + ",,1"); private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); private static final HBaseRESTTestingUtility REST_TEST_UTIL = new HBaseRESTTestingUtility(); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java index 1db8c371593c..8be9a2b18f1a 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java @@ -89,7 +89,7 @@ protected StorageClusterStatusModel buildTestModel() { model.addLiveNode("test1", 1245219839331L, 128, 1024).addRegion(Bytes.toBytes("hbase:root,,0"), 1, 1, 0, 0, 0, 1, 1, 2, 1, 1, 1, 1, 1); model.addLiveNode("test2", 1245239331198L, 512, 1024).addRegion( - Bytes.toBytes(MetaTableName.getInstance() + ",,1246000043724"), 1, 1, 0, 0, 0, 1, 1, 2, 1, 1, + Bytes.toBytes(connection.getMetaTableName() + ",,1246000043724"), 1, 1, 0, 0, 0, 1, 1, 2, 1, 1, 1, 1, 1); return model; } @@ -128,7 +128,7 @@ protected void checkModel(StorageClusterStatusModel model) { assertEquals(1024, node.getMaxHeapSizeMB()); regions = node.getRegions().iterator(); region = regions.next(); - assertEquals(Bytes.toString(region.getName()), MetaTableName.getInstance() + ",,1246000043724"); + assertEquals(Bytes.toString(region.getName()), connection.getMetaTableName() + ",,1246000043724"); assertEquals(1, region.getStores()); assertEquals(1, region.getStorefiles()); assertEquals(0, region.getStorefileSizeMB()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java index 3edfc1eb67a0..3b16014ead55 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java @@ -606,7 +606,7 @@ public static void printAssignmentPlan(FavoredNodesPlan plan) { */ public void updateAssignmentPlanToMeta(FavoredNodesPlan plan) throws IOException { try { - LOG.info("Started updating {} with the new assignment plan", MetaTableName.getInstance()); + LOG.info("Started updating {} with the new assignment plan", connection.getMetaTableName()); Map> assignmentMap = plan.getAssignmentMap(); Map> planToUpdate = new HashMap<>(assignmentMap.size()); Map regionToRegionInfoMap = @@ -620,7 +620,7 @@ public void updateAssignmentPlanToMeta(FavoredNodesPlan plan) throws IOException } catch (Exception e) { LOG.error( "Failed to update hbase:meta with the new assignment" + "plan because " + e.getMessage()); - LOG.info("Updated {} with the new assignment plan", MetaTableName.getInstance()); + LOG.info("Updated {} with the new assignment plan", connection.getMetaTableName()); } } @@ -693,13 +693,13 @@ private void updateAssignmentPlanToRegionServers(FavoredNodesPlan plan) throws I public void updateAssignmentPlan(FavoredNodesPlan plan) throws IOException { LOG.info("Started updating the new assignment plan for {} and the region servers", - MetaTableName.getInstance()); + connection.getMetaTableName()); // Update the new assignment plan to META updateAssignmentPlanToMeta(plan); // Update the new assignment plan to Region Servers updateAssignmentPlanToRegionServers(plan); LOG.info("Finished updating the new assignment plan for {} and the region servers", - MetaTableName.getInstance()); + connection.getMetaTableName()); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java index cdd54d616bee..bfe9a9451c89 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java @@ -87,7 +87,7 @@ public boolean isTableState(TableName tableName, TableState.State... states) { } public void setDeletedTable(TableName tableName) throws IOException { - if (tableName.equals(MetaTableName.getInstance())) { + if (tableName.equals(master.getConnection().getMetaTableName())) { // Can't delete the hbase:meta table. return; } @@ -148,7 +148,7 @@ public TableState getTableState(TableName tableName) throws IOException { } private void updateMetaState(TableName tableName, TableState.State newState) throws IOException { - if (tableName.equals(MetaTableName.getInstance())) { + if (tableName.equals(master.getConnection().getMetaTableName())) { if ( TableState.State.DISABLING.equals(newState) || TableState.State.DISABLED.equals(newState) ) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java index 3a50d74e8a64..58dac734823c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java @@ -355,7 +355,7 @@ public void start() throws IOException, KeeperException { if (RegionReplicaUtil.isDefaultReplica(regionInfo.getReplicaId())) { setMetaAssigned(regionInfo, state == State.OPEN); } - LOG.debug("Loaded {} {}", MetaTableName.getInstance(), regionNode); + LOG.debug("Loaded {} {}", master.getConnection().getMetaTableName(), regionNode); }, result); } } @@ -1963,7 +1963,7 @@ private void checkMetaLoaded(RegionInfo hri, long procId) throws PleaseHoldExcep boolean meta = isMetaRegion(hri); boolean metaLoaded = isMetaLoaded(); if (!meta && !metaLoaded) { - throw new PleaseHoldException("Master not fully online; " + MetaTableName.getInstance() + "=" + throw new PleaseHoldException("Master not fully online; " + master.getConnection().getMetaTableName() + "=" + meta + ", metaLoaded=" + metaLoaded); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java index e6891d08f075..c08ed70dd538 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java @@ -722,7 +722,7 @@ private void preMergeRegionsCommit(final MasterProcedureEnv env) throws IOExcept LOG.error( "Row key of mutation from coprocessor is not parsable as region name. " + "Mutations from coprocessor should only be for {} table.", - MetaTableName.getInstance(), e); + env.getMasterServices().getConnection().getMetaTableName(), e); throw e; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java index 4d42ad619255..afe7a18ae433 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java @@ -172,8 +172,8 @@ public static void visitMetaEntry(final RegionStateVisitor visitor, final Result LOG.debug( "Load {} entry region={}, regionState={}, lastHost={}, " + "regionLocation={}, openSeqNum={}", - MetaTableName.getInstance(), regionInfo.getEncodedName(), state, lastHost, regionLocation, - openSeqNum); + MetaTableName.getInstance(), regionInfo.getEncodedName(), state, lastHost, + regionLocation, openSeqNum); visitor.visitRegionState(result, regionInfo, state, regionLocation, lastHost, openSeqNum); } } @@ -191,9 +191,9 @@ private Put generateUpdateRegionLocationPut(RegionStateNode regionStateNode) thr final int replicaId = regionInfo.getReplicaId(); final Put put = new Put(CatalogFamilyFormat.getMetaKeyForRegion(regionInfo), time); MetaTableAccessor.addRegionInfo(put, regionInfo); - final StringBuilder info = - new StringBuilder("pid=").append(pid).append(" updating ").append(MetaTableName.getInstance()) - .append(" row=").append(regionInfo.getEncodedName()).append(", regionState=").append(state); + final StringBuilder info = new StringBuilder("pid=").append(pid).append(" updating ") + .append(MetaTableName.getInstance()).append(" row=") + .append(regionInfo.getEncodedName()).append(", regionState=").append(state); if (openSeqNum >= 0) { Preconditions.checkArgument(state == State.OPEN && regionLocation != null, "Open region should be on a server"); @@ -285,7 +285,8 @@ private CompletableFuture updateRegionLocation(RegionInfo regionInfo, Stat future = FutureUtils.failedFuture(e); } } else { - AsyncTable table = master.getAsyncConnection().getTable(MetaTableName.getInstance()); + AsyncTable table = + master.getAsyncConnection().getTable(MetaTableName.getInstance()); future = table.put(put); } FutureUtils.addListener(future, (r, e) -> { @@ -331,8 +332,8 @@ private void multiMutate(RegionInfo ri, List mutations) throws IOExcep } } MutateRowsRequest request = builder.build(); - AsyncTable table = - master.getConnection().toAsyncConnection().getTable(MetaTableName.getInstance()); + AsyncTable table = master.getConnection().toAsyncConnection() + .getTable(MetaTableName.getInstance()); CompletableFuture future = table. coprocessorService(MultiRowMutationService::newStub, (stub, controller, done) -> stub.mutateRows(controller, request, done), row); @@ -506,7 +507,8 @@ public void deleteMergeQualifiers(RegionInfo mergeRegion) throws IOException { + " in meta table, they are cleaned up already, Skip."); return; } - try (Table table = master.getConnection().getTable(MetaTableName.getInstance())) { + try (Table table = + master.getConnection().getTable(MetaTableName.getInstance())) { table.delete(delete); } LOG.info( @@ -696,7 +698,8 @@ public static State getRegionState(final Result r, RegionInfo regionInfo) { return State.valueOf(state); } catch (IllegalArgumentException e) { LOG.warn( - "BAD value {} in " + MetaTableName.getInstance() + " info:state column for region {} , " + "BAD value {} in " + MetaTableName.getInstance() + + " info:state column for region {} , " + "Consider using HBCK2 setRegionState ENCODED_REGION_NAME STATE", state, regionInfo.getEncodedName()); return null; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java index 55ac3fa2a0bb..950e4cf13aeb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java @@ -907,7 +907,7 @@ private void preSplitRegionBeforeMETA(final MasterProcedureEnv env) LOG.error( "pid={} row key of mutation from coprocessor not parsable as region name. " + "Mutations from coprocessor should only be for {} table.", - getProcId(), MetaTableName.getInstance()); + getProcId(), env.getMasterServices().getConnection().getMetaTableName()); throw e; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/http/MetaBrowser.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/http/MetaBrowser.java index f9fa67da83ae..b6c1daade0ed 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/http/MetaBrowser.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/http/MetaBrowser.java @@ -157,7 +157,7 @@ public TableName getScanTable() { public Results getResults() { final AsyncTable asyncTable = - connection.getTable(MetaTableName.getInstance()); + connection.getTable(connection.getMetaTableName()); return new Results(asyncTable.getScanner(buildScan())); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java index a6cec33c3efb..0af596e26a41 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java @@ -106,7 +106,8 @@ protected boolean initialChore() { scan(); } } catch (IOException e) { - LOG.warn("Failed initial janitorial scan of {} table", MetaTableName.getInstance(), e); + LOG.warn("Failed initial janitorial scan of {} table", + services.getConnection().getMetaTableName(), e); return false; } return true; @@ -146,7 +147,8 @@ protected void chore() { + this.services.getServerManager().isClusterShutdown()); } } catch (IOException e) { - LOG.warn("Failed janitorial scan of {} table", MetaTableName.getInstance(), e); + LOG.warn("Failed janitorial scan of {} table", + services.getConnection().getMetaTableName(), e); } } @@ -485,7 +487,7 @@ public static void main(String[] args) throws IOException { */ Get g = new Get(Bytes.toBytes("t2,40,1564119846424.1db8c57d64e0733e0f027aaeae7a0bf0.")); g.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); - try (Table t = connection.getTable(MetaTableName.getInstance())) { + try (Table t = connection.getTable(connection.getMetaTableName())) { Result r = t.get(g); byte[] row = g.getRow(); row[row.length - 2] <<= row[row.length - 2]; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java index c74be9de50d9..c915db1f0fe9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java @@ -140,7 +140,7 @@ private RegionInfo metaTableConsistencyCheck(Result metaTableRow) { "INCONSISTENCY: Row name is not equal to serialized info:regioninfo content; " + "row={} {}; See if RegionInfo is referenced in another {} row? Delete?", Bytes.toStringBinary(metaTableRow.getRow()), ri.getRegionNameAsString(), - MetaTableName.getInstance()); + services.getConnection().getMetaTableName()); return null; } // Skip split parent region diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java index 98f43871238e..59cf23be8a44 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java @@ -396,7 +396,7 @@ private static void cleanRegionsInMeta(final MasterProcedureEnv env, final Table List deletes = new ArrayList<>(); try ( Table metaTable = - env.getMasterServices().getConnection().getTable(MetaTableName.getInstance()); + env.getMasterServices().getConnection().getTable(env.getMasterServices().getConnection().getMetaTableName()); ResultScanner scanner = metaTable.getScanner(tableScan)) { for (;;) { Result result = scanner.next(); @@ -407,7 +407,7 @@ private static void cleanRegionsInMeta(final MasterProcedureEnv env, final Table } if (!deletes.isEmpty()) { LOG.warn("Deleting some vestigial " + deletes.size() + " rows of " + tableName + " from " - + MetaTableName.getInstance()); + + env.getMasterServices().getConnection().getMetaTableName()); metaTable.delete(deletes); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java index 8ce33c1574ca..5cb9aca6962b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java @@ -112,7 +112,7 @@ protected Flow executeFromState(final MasterProcedureEnv env, final DisableTable ) { MasterFileSystem fs = env.getMasterFileSystem(); try (BufferedMutator mutator = env.getMasterServices().getConnection() - .getBufferedMutator(MetaTableName.getInstance())) { + .getBufferedMutator(env.getMasterServices().getConnection().getMetaTableName())) { for (RegionInfo region : env.getAssignmentManager().getRegionStates() .getRegionsOfTable(tableName)) { long maxSequenceId = WALSplitUtil.getMaxRegionSequenceId( @@ -231,7 +231,7 @@ public TableOperationType getTableOperationType() { */ private boolean prepareDisable(final MasterProcedureEnv env) throws IOException { boolean canTableBeDisabled = true; - if (tableName.equals(MetaTableName.getInstance())) { + if (tableName.equals(env.getMasterServices().getConnection().getMetaTableName())) { setFailure("master-disable-table", new ConstraintException("Cannot disable " + this.tableName)); canTableBeDisabled = false; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java index d7a4ce95c4ff..88eaf7efcfb0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java @@ -103,13 +103,13 @@ List getRegionsOnCrashedServer(MasterProcedureEnv env) { MetaTableAccessor.scanMetaForTableRegions(env.getMasterServices().getConnection(), visitor, null); } catch (IOException ioe) { - LOG.warn("Failed scan of {} for 'Unknown Servers'", MetaTableName.getInstance(), ioe); + LOG.warn("Failed scan of {} for 'Unknown Servers'", env.getMasterServices().getConnection().getMetaTableName(), ioe); return ris; } // create the server state node too env.getAssignmentManager().getRegionStates().createServer(getServerName()); LOG.info("Found {} mentions of {} in {} of OPEN/OPENING Regions: {}", - visitor.getReassigns().size(), getServerName(), MetaTableName.getInstance(), visitor + visitor.getReassigns().size(), getServerName(), env.getMasterServices().getConnection().getMetaTableName(), visitor .getReassigns().stream().map(RegionInfo::getEncodedName).collect(Collectors.joining(","))); return visitor.getReassigns(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java index 30a120143ade..909d6862e636 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java @@ -65,7 +65,7 @@ private void migrate(MasterProcedureEnv env) throws IOException { try (Table nsTable = conn.getTable(TableName.NAMESPACE_TABLE_NAME); ResultScanner scanner = nsTable.getScanner( new Scan().addFamily(TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES).readAllVersions()); - BufferedMutator mutator = conn.getBufferedMutator(MetaTableName.getInstance())) { + BufferedMutator mutator = conn.getBufferedMutator(env.getMasterServices().getConnection().getMetaTableName())) { for (Result result;;) { result = scanner.next(); if (result == null) { @@ -89,7 +89,7 @@ protected Flow executeFromState(MasterProcedureEnv env, MigrateNamespaceTablePro switch (state) { case MIGRATE_NAMESPACE_TABLE_ADD_FAMILY: TableDescriptor metaTableDesc = - env.getMasterServices().getTableDescriptors().get(MetaTableName.getInstance()); + env.getMasterServices().getTableDescriptors().get(env.getMasterServices().getConnection().getMetaTableName()); if (!metaTableDesc.hasColumnFamily(HConstants.NAMESPACE_FAMILY)) { TableDescriptor newMetaTableDesc = TableDescriptorBuilder.newBuilder(metaTableDesc) .setColumnFamily( diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java index c9e2d67b43df..74603c226a61 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java @@ -110,7 +110,7 @@ protected void preflightChecks(MasterProcedureEnv env, Boolean enabled) throws H Set cfs = this.modifiedTableDescriptor.getColumnFamilyNames(); for (byte[] family : UNDELETABLE_META_COLUMNFAMILIES) { if (!cfs.contains(family)) { - throw new HBaseIOException("Delete of " + MetaTableName.getInstance() + " column family " + throw new HBaseIOException("Delete of " + env.getMasterServices().getConnection().getMetaTableName() + " column family " + Bytes.toString(family)); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 4b7b7787f79d..957650564e20 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -1929,7 +1929,7 @@ public OpenRegionResponse openRegion(final RpcController controller, tableName = ProtobufUtil.toTableName(ri.getTableName()); } } - if (!MetaTableName.getInstance().equals(tableName)) { + if (!server.getConnection().getMetaTableName().equals(tableName)) { throw new ServiceException(ie); } // We are assigning meta, wait a little for regionserver to finish initialization. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index 9eff10a0b160..b94511c9c264 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -541,7 +541,7 @@ public void run() { connection = ConnectionFactory.createConnection(getConf()); admin = connection.getAdmin(); - meta = connection.getTable(MetaTableName.getInstance()); + meta = connection.getTable(connection.getMetaTableName()); status = admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS, Option.DEAD_SERVERS, Option.MASTER, Option.BACKUP_MASTERS, Option.REGIONS_IN_TRANSITION, Option.HBASE_VERSION)); } @@ -661,19 +661,19 @@ public int onlineConsistencyRepair() throws IOException, KeeperException, Interr reportUnknownServers(); // Check if hbase:meta is found only once and in the right place if (!checkMetaRegion()) { - String errorMsg = MetaTableName.getInstance() + " table is not consistent. "; + String errorMsg = connection.getMetaTableName() + " table is not consistent. "; if (shouldFixAssignments()) { - errorMsg += "HBCK will try fixing it. Rerun once " + MetaTableName.getInstance() + errorMsg += "HBCK will try fixing it. Rerun once " + connection.getMetaTableName() + " is back " + "to consistent state."; } else { - errorMsg += "Run HBCK with proper fix options to fix " + MetaTableName.getInstance() + errorMsg += "Run HBCK with proper fix options to fix " + connection.getMetaTableName() + " inconsistency."; } errors.reportError(errorMsg + " Exiting..."); return -2; } // Not going with further consistency check for tables when hbase:meta itself is not consistent. - LOG.info("Loading regionsinfo from the {} table", MetaTableName.getInstance()); + LOG.info("Loading regionsinfo from the {} table", connection.getMetaTableName()); boolean success = loadMetaEntries(); if (!success) return -1; @@ -1222,7 +1222,7 @@ private boolean sidelineFile(FileSystem fs, Path hbaseRoot, Path path) throws IO * TODO -- need to add tests for this. */ private void reportEmptyMetaCells() { - errors.print("Number of empty REGIONINFO_QUALIFIER rows in " + MetaTableName.getInstance() + errors.print("Number of empty REGIONINFO_QUALIFIER rows in " + connection.getMetaTableName() + ": " + emptyRegionInfoQualifiers.size()); if (details) { for (Result r : emptyRegionInfoQualifiers) { @@ -1374,7 +1374,7 @@ private boolean fabricateTableInfo(FSTableDescriptors fstd, TableName tableName, */ public void fixEmptyMetaCells() throws IOException { if (shouldFixEmptyMetaCells() && !emptyRegionInfoQualifiers.isEmpty()) { - LOG.info("Trying to fix empty REGIONINFO_QUALIFIER {} rows.", MetaTableName.getInstance()); + LOG.info("Trying to fix empty REGIONINFO_QUALIFIER {} rows.", connection.getMetaTableName()); for (Result region : emptyRegionInfoQualifiers) { deleteMetaRegion(region.getRow()); errors.getErrorList().remove(ERROR_CODE.EMPTY_META_CELL); @@ -1577,8 +1577,8 @@ private void loadTableStates() throws IOException { // Add hbase:meta so this tool keeps working. In hbase2, meta is always enabled though it // has no entry in the table states. HBCK doesn't work right w/ hbase2 but just do this in // meantime. - this.tableStates.put(MetaTableName.getInstance(), - new TableState(MetaTableName.getInstance(), TableState.State.ENABLED)); + this.tableStates.put(connection.getMetaTableName(), + new TableState(connection.getMetaTableName(), TableState.State.ENABLED)); } /** @@ -1607,7 +1607,7 @@ public void loadHdfsRegionDirs() throws IOException, InterruptedException { TableName tableName = CommonFSUtils.getTableName(path); if ( (!checkMetaOnly && isTableIncluded(tableName)) - || tableName.equals(MetaTableName.getInstance()) + || tableName.equals(connection.getMetaTableName()) ) { tableDirs.add(fs.getFileStatus(path)); } @@ -1652,7 +1652,7 @@ public void loadHdfsRegionDirs() throws IOException, InterruptedException { */ private boolean recordMetaRegion() throws IOException { List locs; - try (RegionLocator locator = connection.getRegionLocator(MetaTableName.getInstance())) { + try (RegionLocator locator = connection.getRegionLocator(connection.getMetaTableName())) { locs = locator.getRegionLocations(HConstants.EMPTY_START_ROW, true); } if (locs == null || locs.isEmpty()) { @@ -2026,7 +2026,7 @@ void closeRegion(HbckRegionInfo hi) throws IOException, InterruptedException { "Unable to close region " + hi.getRegionNameAsString() + " because {} had invalid or missing " + HConstants.CATALOG_FAMILY_STR + ":" + Bytes.toString(HConstants.REGIONINFO_QUALIFIER) + " qualifier value.", - MetaTableName.getInstance()); + connection.getMetaTableName()); continue; } // close the region -- close files and remove assignment @@ -2146,7 +2146,7 @@ else if (!inMeta && !inHdfs && !isDeployed) { } else if (!inMeta && !inHdfs && isDeployed) { errors.reportError(ERROR_CODE.NOT_IN_META_HDFS, "Region " + descriptiveName + ", key=" + key + ", not on HDFS or in " - + MetaTableName.getInstance() + " but " + "deployed on " + + connection.getMetaTableName() + " but " + "deployed on " + Joiner.on(", ").join(hbi.getDeployedOn())); if (shouldFixAssignments()) { undeployRegions(hbi); @@ -2162,7 +2162,7 @@ else if (!inMeta && !inHdfs && !isDeployed) { return; } errors.reportError(ERROR_CODE.NOT_IN_META_OR_DEPLOYED, - "Region " + descriptiveName + " on HDFS, but not listed in " + MetaTableName.getInstance() + "Region " + descriptiveName + " on HDFS, but not listed in " + connection.getMetaTableName() + " or deployed on any region server"); // restore region consistency of an adopted orphan if (shouldFixMeta()) { @@ -2203,7 +2203,7 @@ else if (!inMeta && !inHdfs && !isDeployed) { } } } - LOG.info("Patching {} with .regioninfo: " + hbi.getHdfsHRI(), MetaTableName.getInstance()); + LOG.info("Patching {} with .regioninfo: " + hbi.getHdfsHRI(), connection.getMetaTableName()); int numReplicas = admin.getDescriptor(hbi.getTableName()).getRegionReplication(); HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(), admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet(), @@ -2232,7 +2232,7 @@ else if (!inMeta && !inHdfs && !isDeployed) { } LOG.info("Patching {} with with .regioninfo: " + hbi.getHdfsHRI(), - MetaTableName.getInstance()); + connection.getMetaTableName()); int numReplicas = admin.getDescriptor(hbi.getTableName()).getRegionReplication(); HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(), admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet(), @@ -2309,7 +2309,7 @@ else if (!inMeta && !inHdfs && !isDeployed) { } } else if (inMeta && inHdfs && isMultiplyDeployed) { errors.reportError(ERROR_CODE.MULTI_DEPLOYED, - "Region " + descriptiveName + " is listed in " + MetaTableName.getInstance() + "Region " + descriptiveName + " is listed in " + connection.getMetaTableName() + " on region server " + hbi.getMetaEntry().regionServer + " but is multiply assigned" + " to region servers " + Joiner.on(", ").join(hbi.getDeployedOn())); // If we are trying to fix the errors @@ -2321,7 +2321,7 @@ else if (!inMeta && !inHdfs && !isDeployed) { } } else if (inMeta && inHdfs && isDeployed && !deploymentMatchesMeta) { errors.reportError(ERROR_CODE.SERVER_DOES_NOT_MATCH_META, - "Region " + descriptiveName + " listed in " + MetaTableName.getInstance() + "Region " + descriptiveName + " listed in " + connection.getMetaTableName() + " on region server " + hbi.getMetaEntry().regionServer + " but found on region server " + hbi.getDeployedOn().get(0)); // If we are trying to fix the errors @@ -2607,7 +2607,7 @@ boolean checkMetaRegion() throws IOException, KeeperException, InterruptedExcept metaRegions.put(value.getReplicaId(), value); } } - int metaReplication = admin.getDescriptor(MetaTableName.getInstance()).getRegionReplication(); + int metaReplication = admin.getDescriptor(connection.getMetaTableName()).getRegionReplication(); boolean noProblem = true; // There will be always entries in regionInfoMap corresponding to hbase:meta & its replicas // Check the deployed servers. It should be exactly one server for each replica. @@ -2623,10 +2623,10 @@ boolean checkMetaRegion() throws IOException, KeeperException, InterruptedExcept assignMetaReplica(i); } else if (servers.size() > 1) { errors.reportError(ERROR_CODE.MULTI_META_REGION, - MetaTableName.getInstance() + ", replicaId " + metaHbckRegionInfo.getReplicaId() + connection.getMetaTableName() + ", replicaId " + metaHbckRegionInfo.getReplicaId() + " is found on more than one region."); if (shouldFixAssignments()) { - errors.print("Trying to fix a problem with " + MetaTableName.getInstance() + errors.print("Trying to fix a problem with " + connection.getMetaTableName() + ", replicaId " + metaHbckRegionInfo.getReplicaId() + ".."); setShouldRerun(); // try fix it (treat is a dupe assignment) @@ -2640,11 +2640,11 @@ boolean checkMetaRegion() throws IOException, KeeperException, InterruptedExcept for (Map.Entry entry : metaRegions.entrySet()) { noProblem = false; errors.reportError(ERROR_CODE.SHOULD_NOT_BE_DEPLOYED, - MetaTableName.getInstance() + " replicas are deployed in excess. Configured " + connection.getMetaTableName() + " replicas are deployed in excess. Configured " + metaReplication + ", deployed " + metaRegions.size()); if (shouldFixAssignments()) { errors.print("Trying to undeploy excess replica, replicaId: " + entry.getKey() + " of " - + MetaTableName.getInstance() + ".."); + + connection.getMetaTableName() + ".."); setShouldRerun(); unassignMetaReplica(entry.getValue()); } @@ -2664,9 +2664,9 @@ private void unassignMetaReplica(HbckRegionInfo hi) private void assignMetaReplica(int replicaId) throws IOException, KeeperException, InterruptedException { errors.reportError(ERROR_CODE.NO_META_REGION, - MetaTableName.getInstance() + ", replicaId " + replicaId + " is not found on any region."); + connection.getMetaTableName() + ", replicaId " + replicaId + " is not found on any region."); if (shouldFixAssignments()) { - errors.print("Trying to fix a problem with " + MetaTableName.getInstance() + ".."); + errors.print("Trying to fix a problem with " + connection.getMetaTableName() + ".."); setShouldRerun(); // try to fix it (treat it as unassigned region) RegionInfo h = RegionReplicaUtil @@ -2702,7 +2702,7 @@ public boolean visit(Result result) throws IOException { if (rl == null) { emptyRegionInfoQualifiers.add(result); errors.reportError(ERROR_CODE.EMPTY_META_CELL, - "Empty REGIONINFO_QUALIFIER found in " + MetaTableName.getInstance()); + "Empty REGIONINFO_QUALIFIER found in " + connection.getMetaTableName()); return true; } ServerName sn = null; @@ -2712,7 +2712,7 @@ public boolean visit(Result result) throws IOException { ) { emptyRegionInfoQualifiers.add(result); errors.reportError(ERROR_CODE.EMPTY_META_CELL, - "Empty REGIONINFO_QUALIFIER found in " + MetaTableName.getInstance()); + "Empty REGIONINFO_QUALIFIER found in " + connection.getMetaTableName()); return true; } RegionInfo hri = rl.getRegionLocation(RegionInfo.DEFAULT_REPLICA_ID).getRegion(); @@ -2741,7 +2741,7 @@ public boolean visit(Result result) throws IOException { previous.setMetaEntry(m); } else { throw new IOException( - "Two entries in " + MetaTableName.getInstance() + " are same " + previous); + "Two entries in " + connection.getMetaTableName() + " are same " + previous); } } List mergeParents = CatalogFamilyFormat.getMergeRegions(result.rawCells()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java index 7321d7da178d..11d295435f73 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java @@ -587,12 +587,12 @@ private void unloadRegions(ServerName server, List regionServers, // For isolating hbase:meta, it should move explicitly in Ack mode, // hence the forceMoveRegionByAck = true. if (!metaSeverName.equals(server)) { - LOG.info("Region of {} {} is on server {} moving to {}", MetaTableName.getInstance(), + LOG.info("Region of {} {} is on server {} moving to {}", conn.getMetaTableName(), metaRegionInfo.getEncodedName(), metaSeverName, server); submitRegionMovesWhileUnloading(metaSeverName, Collections.singletonList(server), movedRegions, Collections.singletonList(metaRegionInfo), true); } else { - LOG.info("Region of {} {} already exists on server: {}", MetaTableName.getInstance(), + LOG.info("Region of {} {} already exists on server: {}", conn.getMetaTableName(), metaRegionInfo.getEncodedName(), server); } isolateRegionInfoList.add(RegionInfoBuilder.FIRST_META_REGIONINFO); diff --git a/hbase-server/src/main/resources/hbase-webapps/master/catalogTables.jsp b/hbase-server/src/main/resources/hbase-webapps/master/catalogTables.jsp index f157f327103e..be5b63cff9ea 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/catalogTables.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/catalogTables.jsp @@ -57,7 +57,7 @@ <%= frags.get(tableName.getNameAsString()) != null ? frags.get(tableName.getNameAsString()) + "%" : "n/a" %> <% } %> <% String description = ""; - if (tableName.equals(TableName.META_TABLE_NAME)){ + if (tableName.equals(master.getConnection().getMetaTableName())){ description = "The hbase:meta table holds references to all User Table regions."; } else if (tableName.equals(CanaryTool.DEFAULT_WRITE_TABLE_NAME)){ description = "The hbase:canary table is used to sniff the write availability of" diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp index d838cd3c50bd..c94b32e5d651 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp @@ -196,8 +196,8 @@ Table table = master.getConnection().getTable(TableName.valueOf(fqtn)); boolean showFragmentation = conf.getBoolean("hbase.master.ui.fragmentation.enabled", false); boolean readOnly = !InfoServer.canUserModifyUI(request, getServletContext(), conf); - int numMetaReplicas = - master.getTableDescriptors().get(MetaTableName.getInstance()).getRegionReplication(); + int numMetaReplicas = master.getTableDescriptors() + .get(master.getConnection().getMetaTableName()).getRegionReplication(); Map frags = null; if (showFragmentation) { frags = FSUtils.getTableFragmentation(master); @@ -318,7 +318,7 @@

<% //Meta table. - if(fqtn.equals(MetaTableName.getInstance().getNameAsString())) { %> + if(fqtn.equals(master.getConnection().getMetaTableName().getNameAsString())) { %>

Table Regions

@@ -654,7 +654,7 @@
- +
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseClusterInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseClusterInterface.java index 018d4e1182f1..686ba6204b01 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseClusterInterface.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseClusterInterface.java @@ -388,7 +388,7 @@ public boolean restoreClusterMetrics(ClusterMetrics desiredStatus) throws IOExce * Get the ServerName of region server serving the first hbase:meta region */ public ServerName getServerHoldingMeta() throws IOException { - return getServerHoldingRegion(MetaTableName.getInstance(), + return getServerHoldingRegion(connection.getMetaTableName(), RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java index 78b03ee66826..28a74b0f79e1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java @@ -902,7 +902,7 @@ public SingleProcessHBaseCluster startMiniHBaseCluster(StartTestingClusterOption // Populate the master address configuration from mini cluster configuration. conf.set(HConstants.MASTER_ADDRS_KEY, MasterRegistry.getMasterAddr(c)); // Don't leave here till we've done a successful scan of the hbase:meta - try (Table t = getConnection().getTable(MetaTableName.getInstance()); + try (Table t = getConnection().getTable(connection.getMetaTableName()); ResultScanner s = t.getScanner(new Scan())) { for (;;) { if (s.next() == null) { @@ -1025,7 +1025,7 @@ public void restartHBaseCluster(StartTestingClusterOption option) option.getMasterClass(), option.getRsClass()); // Don't leave here till we've done a successful scan of the hbase:meta Connection conn = ConnectionFactory.createConnection(this.conf); - Table t = conn.getTable(MetaTableName.getInstance()); + Table t = conn.getTable(conn.getMetaTableName()); ResultScanner s = t.getScanner(new Scan()); while (s.next() != null) { // do nothing @@ -2169,7 +2169,7 @@ public String checksumRows(final Table table) throws Exception { */ public List createMultiRegionsInMeta(final Configuration conf, final TableDescriptor htd, byte[][] startKeys) throws IOException { - try (Table meta = getConnection().getTable(MetaTableName.getInstance())) { + try (Table meta = getConnection().getTable(connection.getMetaTableName())) { Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR); List newRegions = new ArrayList<>(startKeys.length); MetaTableAccessor.updateTableState(getConnection(), htd.getTableName(), @@ -2522,7 +2522,7 @@ public void process(WatchedEvent watchedEvent) { monitor.close(); if (checkStatus) { - getConnection().getTable(MetaTableName.getInstance()).close(); + getConnection().getTable(connection.getMetaTableName()).close(); } } @@ -3047,7 +3047,7 @@ public void waitUntilAllRegionsAssigned(final TableName tableName) throws IOExce * Waith until all system table's regions get assigned */ public void waitUntilAllSystemRegionsAssigned() throws IOException { - waitUntilAllRegionsAssigned(MetaTableName.getInstance()); + waitUntilAllRegionsAssigned(connection.getMetaTableName()); } /** @@ -3060,7 +3060,7 @@ public void waitUntilAllSystemRegionsAssigned() throws IOException { public void waitUntilAllRegionsAssigned(final TableName tableName, final long timeout) throws IOException { if (!TableName.isMetaTableName(tableName)) { - try (final Table meta = getConnection().getTable(MetaTableName.getInstance())) { + try (final Table meta = getConnection().getTable(connection.getMetaTableName())) { LOG.debug("Waiting until all regions of table " + tableName + " get assigned. Timeout = " + timeout + "ms"); waitFor(timeout, 200, true, new ExplainingPredicate() { @@ -3280,7 +3280,7 @@ public Table createRandomTable(TableName tableName, final Collection fam Bytes.toBytes(String.format(keyFormat, splitEndKey)), numRegions); if (hbaseCluster != null) { - getMiniHBaseCluster().flushcache(MetaTableName.getInstance()); + getMiniHBaseCluster().flushcache(connection.getMetaTableName()); } BufferedMutator mutator = getConnection().getBufferedMutator(tableName); @@ -3361,7 +3361,7 @@ public static void waitForHostPort(String host, int port) throws IOException { } public static int getMetaRSPort(Connection connection) throws IOException { - try (RegionLocator locator = connection.getRegionLocator(MetaTableName.getInstance())) { + try (RegionLocator locator = connection.getRegionLocator(connection.getMetaTableName())) { return locator.getRegionLocation(Bytes.toBytes("")).getPort(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java index b90831e32c9f..ae53760d7924 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java @@ -68,11 +68,11 @@ public void after() throws Exception { // make sure that with every possible way, we get the same meta table descriptor. private TableDescriptor getMetaDescriptor() throws TableNotFoundException, IOException { Admin admin = UTIL.getAdmin(); - TableDescriptor get = admin.getDescriptor(MetaTableName.getInstance()); + TableDescriptor get = admin.getDescriptor(connection.getMetaTableName()); TableDescriptor list = admin.listTableDescriptors(true).stream().filter(td -> td.isMetaTable()).findAny().get(); TableDescriptor listByName = - admin.listTableDescriptors(Collections.singletonList(MetaTableName.getInstance())).get(0); + admin.listTableDescriptors(Collections.singletonList(connection.getMetaTableName())).get(0); TableDescriptor listByNs = admin.listTableDescriptorsByNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME).stream() .filter(td -> td.isMetaTable()).findAny().get(); @@ -91,7 +91,7 @@ private TableDescriptor getMetaDescriptor() throws TableNotFoundException, IOExc @Test public void testEditMeta() throws IOException { Admin admin = UTIL.getAdmin(); - admin.tableExists(MetaTableName.getInstance()); + admin.tableExists(connection.getMetaTableName()); TableDescriptor originalDescriptor = getMetaDescriptor(); ColumnFamilyDescriptor cfd = originalDescriptor.getColumnFamily(HConstants.CATALOG_FAMILY); int oldVersions = cfd.getMaxVersions(); @@ -100,11 +100,11 @@ public void testEditMeta() throws IOException { .setConfiguration(ColumnFamilyDescriptorBuilder.DATA_BLOCK_ENCODING, DataBlockEncoding.ROW_INDEX_V1.toString()) .build(); - admin.modifyColumnFamily(MetaTableName.getInstance(), cfd); + admin.modifyColumnFamily(connection.getMetaTableName(), cfd); byte[] extraColumnFamilyName = Bytes.toBytes("xtra"); ColumnFamilyDescriptor newCfd = ColumnFamilyDescriptorBuilder.newBuilder(extraColumnFamilyName).build(); - admin.addColumnFamily(MetaTableName.getInstance(), newCfd); + admin.addColumnFamily(connection.getMetaTableName(), newCfd); TableDescriptor descriptor = getMetaDescriptor(); // Assert new max versions is == old versions plus 1. assertEquals(oldVersions + 1, @@ -126,11 +126,11 @@ public void testEditMeta() throws IOException { assertEquals(encoding, DataBlockEncoding.ROW_INDEX_V1.toString()); assertTrue(r.getStore(extraColumnFamilyName) != null); // Assert we can't drop critical hbase:meta column family but we can drop any other. - admin.deleteColumnFamily(MetaTableName.getInstance(), newCfd.getName()); + admin.deleteColumnFamily(connection.getMetaTableName(), newCfd.getName()); descriptor = getMetaDescriptor(); assertTrue(descriptor.getColumnFamily(newCfd.getName()) == null); try { - admin.deleteColumnFamily(MetaTableName.getInstance(), HConstants.CATALOG_FAMILY); + admin.deleteColumnFamily(connection.getMetaTableName(), HConstants.CATALOG_FAMILY); fail("Should not reach here"); } catch (HBaseIOException hioe) { assertTrue(hioe.getMessage().contains("Delete of hbase:meta")); @@ -144,7 +144,7 @@ public void testEditMeta() throws IOException { @Test public void testAlterMetaWithReadOnly() throws IOException { Admin admin = UTIL.getAdmin(); - TableDescriptor origMetaTableDesc = admin.getDescriptor(MetaTableName.getInstance()); + TableDescriptor origMetaTableDesc = admin.getDescriptor(connection.getMetaTableName()); assertFalse(origMetaTableDesc.isReadOnly()); TableDescriptor newTD = TableDescriptorBuilder.newBuilder(origMetaTableDesc).setReadOnly(true).build(); @@ -152,7 +152,7 @@ public void testAlterMetaWithReadOnly() throws IOException { admin.modifyTable(newTD); fail("Meta table can't be set as read only"); } catch (Exception e) { - assertFalse(admin.getDescriptor(MetaTableName.getInstance()).isReadOnly()); + assertFalse(admin.getDescriptor(connection.getMetaTableName()).isReadOnly()); } // Create a table to check region assignment & meta operation diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java index cf0ae4f7eb80..c14a3cbd1a71 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java @@ -91,7 +91,7 @@ public void testGetMasterInfoPort() throws Exception { @Test public void testInfoServersRedirect() throws Exception { // give the cluster time to start up - UTIL.getConnection().getTable(MetaTableName.getInstance()).close(); + UTIL.getConnection().getTable(connection.getMetaTableName()).close(); int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort(); assertContainsContent(new URL("http://localhost:" + port + "/index.html"), "master.jsp"); assertContainsContent(new URL("http://localhost:" + port + "/master-status"), "master.jsp"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java index e1c76567be7f..01e18b23024c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java @@ -87,7 +87,7 @@ private void multiMutate(byte[] row, List mutations) throws IOExceptio } } MutateRowsRequest request = builder.build(); - AsyncTable table = UTIL.getAsyncConnection().getTable(MetaTableName.getInstance()); + AsyncTable table = UTIL.getAsyncConnection().getTable(connection.getMetaTableName()); CompletableFuture future = table. coprocessorService(MultiRowMutationService::newStub, (stub, controller, done) -> stub.mutateRows(controller, request, done), row); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java index 67110aca8181..3508dd1b0d92 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java @@ -117,7 +117,7 @@ public void verifyReservedNS() throws IOException { assertEquals(2, admin.listNamespaceDescriptors().length); // verify existence of system tables - Set systemTables = Sets.newHashSet(MetaTableName.getInstance()); + Set systemTables = Sets.newHashSet(connection.getMetaTableName()); List descs = admin.listTableDescriptorsByNamespace( Bytes.toBytes(NamespaceDescriptor.SYSTEM_NAMESPACE.getName())); assertEquals(systemTables.size(), descs.size()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerInternalsTracing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerInternalsTracing.java index 16ede9b98c60..a05e54897070 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerInternalsTracing.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerInternalsTracing.java @@ -77,7 +77,7 @@ public Setup(final Supplier testingUtilSupplier) { @Override protected void before() throws Throwable { final HBaseTestingUtil testingUtil = testingUtilSupplier.get(); - testingUtil.waitTableAvailable(MetaTableName.getInstance()); + testingUtil.waitTableAvailable(connection.getMetaTableName()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java index 77fc747178fa..99e2920eccaf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java @@ -50,7 +50,7 @@ public abstract class AbstractTestRegionLocator { protected static void startClusterAndCreateTable() throws Exception { UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), MetaTableName.getInstance(), REGION_REPLICATION); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), connection.getMetaTableName(), REGION_REPLICATION); TableDescriptor td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRegionReplication(REGION_REPLICATION) .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build(); @@ -70,7 +70,7 @@ protected static void startClusterAndCreateTable() throws Exception { @After public void tearDownAfterTest() throws IOException { clearCache(TABLE_NAME); - clearCache(MetaTableName.getInstance()); + clearCache(connection.getMetaTableName()); } private byte[] getStartKey(int index) { @@ -172,7 +172,7 @@ private void assertMetaRegionLocation(HRegionLocation loc, int replicaId) { assertArrayEquals(HConstants.EMPTY_END_ROW, region.getEndKey()); assertEquals(replicaId, region.getReplicaId()); ServerName expected = - findRegionLocation(MetaTableName.getInstance(), region.getStartKey(), replicaId); + findRegionLocation(connection.getMetaTableName(), region.getStartKey(), replicaId); assertEquals(expected, loc.getServerName()); } @@ -185,19 +185,19 @@ private void assertMetaRegionLocations(List locs) { @Test public void testMeta() throws IOException { - assertMetaStartOrEndKeys(getStartKeys(MetaTableName.getInstance())); - assertMetaStartOrEndKeys(getEndKeys(MetaTableName.getInstance())); - Pair startEndKeys = getStartEndKeys(MetaTableName.getInstance()); + assertMetaStartOrEndKeys(getStartKeys(connection.getMetaTableName())); + assertMetaStartOrEndKeys(getEndKeys(connection.getMetaTableName())); + Pair startEndKeys = getStartEndKeys(connection.getMetaTableName()); assertMetaStartOrEndKeys(startEndKeys.getFirst()); assertMetaStartOrEndKeys(startEndKeys.getSecond()); for (int replicaId = 0; replicaId < REGION_REPLICATION; replicaId++) { assertMetaRegionLocation( - getRegionLocation(MetaTableName.getInstance(), HConstants.EMPTY_START_ROW, replicaId), + getRegionLocation(connection.getMetaTableName(), HConstants.EMPTY_START_ROW, replicaId), replicaId); } assertMetaRegionLocations( - getRegionLocations(MetaTableName.getInstance(), HConstants.EMPTY_START_ROW)); - assertMetaRegionLocations(getAllRegionLocations(MetaTableName.getInstance())); + getRegionLocations(connection.getMetaTableName(), HConstants.EMPTY_START_ROW)); + assertMetaRegionLocations(getAllRegionLocations(connection.getMetaTableName())); } protected abstract byte[][] getStartKeys(TableName tableName) throws IOException; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java index 7b2aa41a2202..c1cad1a911dd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java @@ -58,7 +58,7 @@ protected static void startCluster() throws Exception { .numAlwaysStandByMasters(1).numMasters(1).numRegionServers(REGIONSERVERS_COUNT).build(); TEST_UTIL.startMiniCluster(option); Admin admin = TEST_UTIL.getAdmin(); - HBaseTestingUtil.setReplicas(admin, MetaTableName.getInstance(), 3); + HBaseTestingUtil.setReplicas(admin, connection.getMetaTableName(), 3); AssignmentManager am = TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager(); Set sns = new HashSet(); ServerName hbaseMetaServerName = am.getRegionStates() diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java index 25953227d8df..459deb9f3157 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java @@ -48,7 +48,7 @@ static void waitUntilAllMetaReplicasAreReady(HBaseTestingUtil util, ConnectionRe throws IOException { Configuration conf = util.getConfiguration(); int regionReplicaCount = - util.getAdmin().getDescriptor(MetaTableName.getInstance()).getRegionReplication(); + util.getAdmin().getDescriptor(connection.getMetaTableName()).getRegionReplication(); Waiter.waitFor(conf, conf.getLong("hbase.client.sync.wait.timeout.msec", 60000), 200, true, new ExplainingPredicate() { @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java index 986701050453..c1623b2bbc96 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java @@ -65,7 +65,7 @@ public class TestAdmin extends TestAdminBase { @Test public void testListTableDescriptors() throws IOException { TableDescriptor metaTableDescriptor = - TEST_UTIL.getAdmin().getDescriptor(MetaTableName.getInstance()); + TEST_UTIL.getAdmin().getDescriptor(connection.getMetaTableName()); List tableDescriptors = TEST_UTIL.getAdmin().listTableDescriptors(true); assertTrue(tableDescriptors.contains(metaTableDescriptor)); tableDescriptors = TEST_UTIL.getAdmin().listTableDescriptors(false); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index a6dd60781bd0..67bee384cee6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -85,13 +85,13 @@ public class TestAdmin2 extends TestAdminBase { public void testCreateBadTables() throws IOException { String msg = null; try { - ADMIN.createTable(TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).build()); + ADMIN.createTable(TableDescriptorBuilder.newBuilder(connection.getMetaTableName()).build()); } catch (TableExistsException e) { msg = e.toString(); } assertTrue("Unexcepted exception message " + msg, msg != null && msg.startsWith(TableExistsException.class.getName()) - && msg.contains(MetaTableName.getInstance().getNameAsString())); + && msg.contains(connection.getMetaTableName().getNameAsString())); // Now try and do concurrent creation with a bunch of threads. TableDescriptor tableDescriptor = @@ -457,7 +457,7 @@ private void setUpforLogRolling() { private HRegionServer startAndWriteData(TableName tableName, byte[] value) throws IOException, InterruptedException { // When the hbase:meta table can be opened, the region servers are running - TEST_UTIL.getConnection().getTable(MetaTableName.getInstance()).close(); + TEST_UTIL.getConnection().getTable(connection.getMetaTableName()).close(); // Create the test table and open it TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName) @@ -487,7 +487,7 @@ private HRegionServer startAndWriteData(TableName tableName, byte[] value) @Test public void testDisableCatalogTable() throws Exception { try { - ADMIN.disableTable(MetaTableName.getInstance()); + ADMIN.disableTable(connection.getMetaTableName()); fail("Expected to throw ConstraintException"); } catch (ConstraintException e) { } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java index 66ac07cdf88f..61d9e774ca32 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java @@ -55,7 +55,7 @@ public class TestAsyncAdminWithRegionReplicas extends TestAsyncAdminBase { @BeforeClass public static void setUpBeforeClass() throws Exception { TestAsyncAdminBase.setUpBeforeClass(); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), MetaTableName.getInstance(), 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), connection.getMetaTableName(), 3); try (ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent())) { RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, registry); @@ -81,7 +81,7 @@ public void testMoveNonDefaultReplica() throws InterruptedException, ExecutionException, IOException { createTableWithDefaultConf(tableName, 3); testMoveNonDefaultReplica(tableName); - testMoveNonDefaultReplica(MetaTableName.getInstance()); + testMoveNonDefaultReplica(connection.getMetaTableName()); } @Test @@ -139,11 +139,11 @@ public void testCloneTableSchema() throws IOException, InterruptedException, Exe @Test public void testGetTableRegions() throws InterruptedException, ExecutionException, IOException { - List metaRegions = admin.getRegions(MetaTableName.getInstance()).get(); + List metaRegions = admin.getRegions(connection.getMetaTableName()).get(); assertEquals(3, metaRegions.size()); for (int i = 0; i < 3; i++) { RegionInfo metaRegion = metaRegions.get(i); - assertEquals(MetaTableName.getInstance(), metaRegion.getTable()); + assertEquals(connection.getMetaTableName(), metaRegion.getTable()); assertEquals(i, metaRegion.getReplicaId()); } createTableWithDefaultConf(tableName, 3); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java index 871590d732e7..a9811f9243fb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java @@ -106,7 +106,7 @@ private void assertInitialized() { protected void before() throws Throwable { final AsyncAdmin admin = connectionRule.getAsyncConnection().getAdmin(); testUtil = miniClusterRule.getTestingUtility(); - HBaseTestingUtil.setReplicas(admin, MetaTableName.getInstance(), 3); + HBaseTestingUtil.setReplicas(admin, connection.getMetaTableName(), 3); testUtil.waitUntilNoRegionsInTransition(); registry = ConnectionRegistryFactory.create(testUtil.getConfiguration(), User.getCurrent()); RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(testUtil, registry); @@ -164,7 +164,7 @@ public void test() throws Exception { TraceUtil.trace(() -> { try { - testLocator(miniClusterRule.getTestingUtility(), MetaTableName.getInstance(), + testLocator(miniClusterRule.getTestingUtility(), connection.getMetaTableName(), new Locator() { @Override public void updateCachedLocationOnError(HRegionLocation loc, Throwable error) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java index 9f4cb8f3a4bb..e0164c65bf08 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java @@ -107,9 +107,9 @@ public static void setUp() throws Exception { admin.balancerSwitch(false, true); // Enable hbase:meta replication. - HBaseTestingUtil.setReplicas(admin, MetaTableName.getInstance(), NUM_OF_META_REPLICA); + HBaseTestingUtil.setReplicas(admin, connection.getMetaTableName(), NUM_OF_META_REPLICA); TEST_UTIL.waitFor(30000, - () -> TEST_UTIL.getMiniHBaseCluster().getRegions(MetaTableName.getInstance()).size() + () -> TEST_UTIL.getMiniHBaseCluster().getRegions(connection.getMetaTableName()).size() >= NUM_OF_META_REPLICA); SPLIT_KEYS = new byte[8][]; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java index 4a51b6d26465..374da4811657 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java @@ -87,7 +87,7 @@ public void testSplitSwitch() throws Exception { TestAsyncRegionAdminApi.loadData(tableName, families, rows); AsyncTable metaTable = - ASYNC_CONN.getTable(MetaTableName.getInstance()); + ASYNC_CONN.getTable(connection.getMetaTableName()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); int originalCount = regionLocations.size(); @@ -119,7 +119,7 @@ public void testMergeSwitch() throws Exception { TestAsyncRegionAdminApi.loadData(tableName, families, 1000); AsyncTable metaTable = - ASYNC_CONN.getTable(MetaTableName.getInstance()); + ASYNC_CONN.getTable(connection.getMetaTableName()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); int originalCount = regionLocations.size(); @@ -165,7 +165,7 @@ public void testMergeRegions() throws Exception { createTableWithDefaultConf(tableName, splitRows); AsyncTable metaTable = - ASYNC_CONN.getTable(MetaTableName.getInstance()); + ASYNC_CONN.getTable(connection.getMetaTableName()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); RegionInfo regionA; @@ -246,7 +246,7 @@ private void splitTest(TableName tableName, int rowCount, boolean isSplitRegion, createTableWithDefaultConf(tableName); AsyncTable metaTable = - ASYNC_CONN.getTable(MetaTableName.getInstance()); + ASYNC_CONN.getTable(connection.getMetaTableName()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); assertEquals(1, regionLocations.size()); @@ -304,7 +304,7 @@ public void testTruncateRegion() throws Exception { createTableWithDefaultConf(tableName, splitKeys, bFamilies); AsyncTable metaTable = - ASYNC_CONN.getTable(MetaTableName.getInstance()); + ASYNC_CONN.getTable(connection.getMetaTableName()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); RegionInfo regionToBeTruncated = regionLocations.get(0).getRegion(); @@ -339,7 +339,7 @@ public void testTruncateReplicaRegionNotAllowed() throws Exception { createTableWithDefaultConf(tableName, 2, splitKeys, bFamilies); AsyncTable metaTable = - ASYNC_CONN.getTable(MetaTableName.getInstance()); + ASYNC_CONN.getTable(connection.getMetaTableName()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); RegionInfo primaryRegion = regionLocations.get(0).getRegion(); @@ -360,7 +360,7 @@ public void testTruncateReplicaRegionNotAllowed() throws Exception { @Test public void testTruncateRegionsMetaTableRegionsNotAllowed() throws Exception { - AsyncTableRegionLocator locator = ASYNC_CONN.getRegionLocator(MetaTableName.getInstance()); + AsyncTableRegionLocator locator = ASYNC_CONN.getRegionLocator(connection.getMetaTableName()); List regionLocations = locator.getAllRegionLocations().get(); HRegionLocation regionToBeTruncated = regionLocations.get(0); // 1 diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java index f6732c611194..7b0b0c93a95d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java @@ -73,7 +73,7 @@ public void testCreateTable() throws Exception { static TableState.State getStateFromMeta(TableName table) throws Exception { Optional state = ClientMetaTableAccessor - .getTableState(ASYNC_CONN.getTable(MetaTableName.getInstance()), table).get(); + .getTableState(ASYNC_CONN.getTable(connection.getMetaTableName()), table).get(); assertTrue(state.isPresent()); return state.get().getState(); } @@ -81,7 +81,7 @@ static TableState.State getStateFromMeta(TableName table) throws Exception { @Test public void testCreateTableNumberOfRegions() throws Exception { AsyncTable metaTable = - ASYNC_CONN.getTable(MetaTableName.getInstance()); + ASYNC_CONN.getTable(connection.getMetaTableName()); createTableWithDefaultConf(tableName); List regionLocations = @@ -130,7 +130,7 @@ public void testCreateTableWithRegions() throws Exception { assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable); AsyncTable metaTable = - ASYNC_CONN.getTable(MetaTableName.getInstance()); + ASYNC_CONN.getTable(connection.getMetaTableName()); List regions = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); Iterator hris = regions.iterator(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java index 1b2ef352515c..08ea9689c889 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java @@ -58,7 +58,7 @@ public class TestAsyncTableAdminApi2 extends TestAsyncAdminBase { @Test public void testDisableCatalogTable() throws Exception { try { - this.admin.disableTable(MetaTableName.getInstance()).join(); + this.admin.disableTable(connection.getMetaTableName()).join(); fail("Expected to throw ConstraintException"); } catch (Exception e) { } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java index 93e684453f51..c6081bd8a29b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java @@ -66,10 +66,10 @@ public void testTableExist() throws Exception { TEST_UTIL.createTable(tableName, FAMILY); exist = admin.tableExists(tableName).get(); assertTrue(exist); - exist = admin.tableExists(MetaTableName.getInstance()).get(); + exist = admin.tableExists(connection.getMetaTableName()).get(); assertTrue(exist); // meta table already exists - exist = admin.tableExists(MetaTableName.getInstance()).get(); + exist = admin.tableExists(connection.getMetaTableName()).get(); assertTrue(exist); } @@ -118,7 +118,7 @@ public void testListTables() throws Exception { assertEquals(0, size); Collections.addAll(tableNames, tables); - tableNames.add(MetaTableName.getInstance()); + tableNames.add(connection.getMetaTableName()); tableDescs = admin.listTableDescriptors(tableNames).get(); size = tableDescs.size(); assertEquals(tables.length + 1, size); @@ -126,7 +126,7 @@ public void testListTables() throws Exception { assertTrue("tableName should be equal in order", tableDescs.get(j).getTableName().equals(tables[i])); } - assertTrue(tableDescs.get(size - 1).getTableName().equals(MetaTableName.getInstance())); + assertTrue(tableDescs.get(size - 1).getTableName().equals(connection.getMetaTableName())); for (int i = 0; i < tables.length; i++) { admin.disableTable(tables[i]).join(); @@ -205,7 +205,7 @@ public void testDisableAndEnableTable() throws Exception { assertTrue(ok); // meta table can not be disabled. try { - admin.disableTable(MetaTableName.getInstance()).get(); + admin.disableTable(connection.getMetaTableName()).get(); fail("meta table can not be disabled"); } catch (ExecutionException e) { Throwable cause = e.getCause(); @@ -286,7 +286,7 @@ public void testEnableTableRetainAssignment() throws Exception { createTableWithDefaultConf(tableName, splitKeys); AsyncTable metaTable = - ASYNC_CONN.getTable(MetaTableName.getInstance()); + ASYNC_CONN.getTable(connection.getMetaTableName()); List regions = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); assertEquals( @@ -315,8 +315,8 @@ public void testIsTableEnabledAndDisabled() throws Exception { assertTrue(admin.isTableDisabled(tableName).get()); // meta table is always enabled - assertTrue(admin.isTableEnabled(MetaTableName.getInstance()).get()); - assertFalse(admin.isTableDisabled(MetaTableName.getInstance()).get()); + assertTrue(admin.isTableEnabled(connection.getMetaTableName()).get()); + assertFalse(admin.isTableDisabled(connection.getMetaTableName()).get()); } @Test @@ -324,6 +324,6 @@ public void testIsTableAvailable() throws Exception { createTableWithDefaultConf(tableName); TEST_UTIL.waitTableAvailable(tableName); assertTrue(admin.isTableAvailable(tableName).get()); - assertTrue(admin.isTableAvailable(MetaTableName.getInstance()).get()); + assertTrue(admin.isTableAvailable(connection.getMetaTableName()).get()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java index db794ff014ff..d7670fd4e872 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java @@ -95,14 +95,14 @@ public static void setUp() throws Exception { conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, FailPrimaryMetaScanCp.class.getName()); UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), MetaTableName.getInstance(), 3); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), connection.getMetaTableName(), 3); try (ConnectionRegistry registry = ConnectionRegistryFactory.create(conf, User.getCurrent())) { RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(UTIL, registry); } try (Table table = UTIL.createTable(TABLE_NAME, FAMILY)) { table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)); } - UTIL.flush(MetaTableName.getInstance()); + UTIL.flush(connection.getMetaTableName()); // wait for the store file refresh so we can read the region location from secondary meta // replicas Thread.sleep(2000); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java index 4512c060cee7..2dd1baa7ac24 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java @@ -72,9 +72,9 @@ public static void setUp() throws Exception { admin.balancerSwitch(false, true); // Enable hbase:meta replication. - HBaseTestingUtil.setReplicas(admin, MetaTableName.getInstance(), numOfMetaReplica); + HBaseTestingUtil.setReplicas(admin, connection.getMetaTableName(), numOfMetaReplica); TEST_UTIL.waitFor(30000, - () -> TEST_UTIL.getMiniHBaseCluster().getRegions(MetaTableName.getInstance()).size() + () -> TEST_UTIL.getMiniHBaseCluster().getRegions(connection.getMetaTableName()).size() >= numOfMetaReplica); registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); @@ -95,14 +95,14 @@ public void testMetaChangeFromReplicaNoReplica() throws IOException, Interrupted CatalogReplicaLoadBalanceSimpleSelector.class.getName()); CatalogReplicaLoadBalanceSelector metaSelector = CatalogReplicaLoadBalanceSelectorFactory - .createSelector(replicaSelectorClass, MetaTableName.getInstance(), CONN, () -> { + .createSelector(replicaSelectorClass, connection.getMetaTableName(), CONN, () -> { int numOfReplicas = CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS; try { RegionLocations metaLocations = CONN.registry.getMetaRegionLocations() .get(CONN.connConf.getMetaReadRpcTimeoutNs(), TimeUnit.NANOSECONDS); numOfReplicas = metaLocations.size(); } catch (Exception e) { - LOG.error("Failed to get table {}'s region replication, ", MetaTableName.getInstance(), + LOG.error("Failed to get table {}'s region replication, ", connection.getMetaTableName(), e); } return numOfReplicas; @@ -117,20 +117,20 @@ public void testMetaChangeFromReplicaNoReplica() throws IOException, Interrupted IntStream.range(0, numOfMetaReplica).forEach(i -> assertNotEquals(replicaIdCount[i], 0)); // Change to No meta replica - HBaseTestingUtil.setReplicas(admin, MetaTableName.getInstance(), 1); + HBaseTestingUtil.setReplicas(admin, connection.getMetaTableName(), 1); TEST_UTIL.waitFor(30000, - () -> TEST_UTIL.getMiniHBaseCluster().getRegions(MetaTableName.getInstance()).size() == 1); + () -> TEST_UTIL.getMiniHBaseCluster().getRegions(connection.getMetaTableName()).size() == 1); CatalogReplicaLoadBalanceSelector metaSelectorWithNoReplica = CatalogReplicaLoadBalanceSelectorFactory.createSelector(replicaSelectorClass, - MetaTableName.getInstance(), CONN, () -> { + connection.getMetaTableName(), CONN, () -> { int numOfReplicas = CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS; try { RegionLocations metaLocations = CONN.registry.getMetaRegionLocations() .get(CONN.connConf.getMetaReadRpcTimeoutNs(), TimeUnit.NANOSECONDS); numOfReplicas = metaLocations.size(); } catch (Exception e) { - LOG.error("Failed to get table {}'s region replication, ", MetaTableName.getInstance(), + LOG.error("Failed to get table {}'s region replication, ", connection.getMetaTableName(), e); } return numOfReplicas; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java index 3b5a1496e403..6f3121e9ea46 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java @@ -48,7 +48,7 @@ public void testReplicaCleanup() throws Exception { ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher(); List metaReplicaZnodes = zkw.getMetaReplicaNodes(); assertEquals(3, metaReplicaZnodes.size()); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), MetaTableName.getInstance(), 1); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), connection.getMetaTableName(), 1); metaReplicaZnodes = zkw.getMetaReplicaNodes(); assertEquals(1, metaReplicaZnodes.size()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java index 55bf792f568f..66de0abf673f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java @@ -454,7 +454,7 @@ public ScanResponse scan(final RpcController controller, final ScanRequest reque } else { ScanResponse scanRes = super.scan(controller, request); String regionName = Bytes.toString(request.getRegion().getValue().toByteArray()); - if (!regionName.contains(MetaTableName.getInstance().getNameAsString())) { + if (!regionName.contains(connection.getMetaTableName().getNameAsString())) { tableScannerId = scanRes.getScannerId(); if (sleepOnOpen) { try { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java index e40208aa3b90..ee75ad008406 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java @@ -90,8 +90,8 @@ public void setup() throws IOException { conf = TEST_UTIL.getConfiguration(); rootDir = TEST_UTIL.getDefaultRootDirPath(); fs = TEST_UTIL.getTestFileSystem(); - htd = TEST_UTIL.getAdmin().getDescriptor(MetaTableName.getInstance()); - hri = TEST_UTIL.getAdmin().getRegions(MetaTableName.getInstance()).get(0); + htd = TEST_UTIL.getAdmin().getDescriptor(connection.getMetaTableName()); + hri = TEST_UTIL.getAdmin().getRegions(connection.getMetaTableName()).get(0); scan = new Scan(); } @@ -201,7 +201,7 @@ private void testScanMetricsWithScanMetricsByRegionDisabled(ScanMetrics scanMetr Configuration copyConf = new Configuration(conf); Scan scan = new Scan(); scan.setScanMetricsEnabled(true); - TEST_UTIL.getAdmin().flush(MetaTableName.getInstance()); + TEST_UTIL.getAdmin().flush(connection.getMetaTableName()); try (ClientSideRegionScanner clientSideRegionScanner = new ClientSideRegionScanner(copyConf, fs, rootDir, htd, hri, scan, scanMetrics)) { clientSideRegionScanner.next(); @@ -230,7 +230,7 @@ private void testScanMetricByRegion(ScanMetrics scanMetrics) throws IOException Configuration copyConf = new Configuration(conf); Scan scan = new Scan(); scan.setEnableScanMetricsByRegion(true); - TEST_UTIL.getAdmin().flush(MetaTableName.getInstance()); + TEST_UTIL.getAdmin().flush(connection.getMetaTableName()); try (ClientSideRegionScanner clientSideRegionScanner = new ClientSideRegionScanner(copyConf, fs, rootDir, htd, hri, scan, scanMetrics)) { clientSideRegionScanner.next(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java index 225c30046fde..d5c4993db711 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java @@ -93,7 +93,7 @@ public void testDeleteForSureClearsAllTableRowsFromMeta() } // Now I have a nice table, mangle it by removing the HConstants.REGIONINFO_QUALIFIER_STR // content from a few of the rows. - try (Table metaTable = TEST_UTIL.getConnection().getTable(MetaTableName.getInstance())) { + try (Table metaTable = TEST_UTIL.getConnection().getTable(connection.getMetaTableName())) { try (ResultScanner scanner = metaTable.getScanner( MetaTableAccessor.getScanForTableName(TEST_UTIL.getConfiguration(), tableName))) { for (Result result : scanner) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java index d224ad41a277..7af27446a028 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java @@ -2533,7 +2533,7 @@ public void testFilterAllRecords() throws IOException { scan.setCaching(1); // Filter out any records scan.setFilter(new FilterList(new FirstKeyOnlyFilter(), new InclusiveStopFilter(new byte[0]))); - try (Table table = TEST_UTIL.getConnection().getTable(MetaTableName.getInstance())) { + try (Table table = TEST_UTIL.getConnection().getTable(connection.getMetaTableName())) { try (ResultScanner s = table.getScanner(scan)) { assertNull(s.next()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java index e14944a14e03..cf2ea4190a7e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java @@ -50,7 +50,7 @@ public static void setUp() throws Exception { public void testUpgradeAndIncreaseReplicaCount() throws Exception { HMaster oldMaster = TEST_UTIL.getMiniHBaseCluster().getMaster(); TableDescriptors oldTds = oldMaster.getTableDescriptors(); - TableDescriptor oldMetaTd = oldTds.get(MetaTableName.getInstance()); + TableDescriptor oldMetaTd = oldTds.get(connection.getMetaTableName()); assertEquals(3, oldMetaTd.getRegionReplication()); // force update the replica count to 1 and then kill the master, to simulate that hen upgrading, // we have no region replication in meta table descriptor but we actually have meta region diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java index 365e371bf13d..1ce4df6c5de8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java @@ -61,7 +61,7 @@ public static void setUp() throws Exception { StartTestingClusterOption.Builder builder = StartTestingClusterOption.builder(); builder.numMasters(3).numRegionServers(3); TEST_UTIL.startMiniCluster(builder.build()); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), MetaTableName.getInstance(), 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), connection.getMetaTableName(), 3); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java index 267593311fdc..8994ba58b0c2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java @@ -88,7 +88,7 @@ public static void setUpBeforeClass() throws Exception { conf.setStrings(HConstants.REGION_SERVER_IMPL, RegionServerWithFakeRpcServices.class.getName()); TEST_UTIL.startMiniCluster(1); TEST_UTIL.getHBaseCluster().waitForActiveAndReadyMaster(); - TEST_UTIL.waitUntilAllRegionsAssigned(MetaTableName.getInstance()); + TEST_UTIL.waitUntilAllRegionsAssigned(connection.getMetaTableName()); badRS = TEST_UTIL.getHBaseCluster().getRegionServer(0); assertTrue(badRS.getRSRpcServices() instanceof FakeRSRpcServices); TableDescriptor desc = TableDescriptorBuilder.newBuilder(TABLE_NAME) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java index cfe193f60fb8..d910778cffe6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java @@ -63,7 +63,7 @@ public class TestMetaRegionLocationCache { @BeforeClass public static void setUp() throws Exception { TEST_UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), MetaTableName.getInstance(), 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), connection.getMetaTableName(), 3); REGISTRY = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, REGISTRY); TEST_UTIL.getAdmin().balancerSwitch(false, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java index 1fe06bcd3c56..836a3dc451b4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java @@ -54,7 +54,7 @@ public static void setUp() throws Exception { @Test public void testMetaHTDReplicaCount() throws Exception { assertEquals(3, - TEST_UTIL.getAdmin().getDescriptor(MetaTableName.getInstance()).getRegionReplication()); + TEST_UTIL.getAdmin().getDescriptor(connection.getMetaTableName()).getRegionReplication()); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java index 240e60099edb..26faaa431a43 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java @@ -98,7 +98,7 @@ public static void shutdownMetaAndDoValidations(HBaseTestingUtil util) throws Ex ServerName master = null; try (Connection c = ConnectionFactory.createConnection(util.getConfiguration())) { try (Table htable = util.createTable(TABLE, FAMILIES)) { - util.getAdmin().flush(MetaTableName.getInstance()); + util.getAdmin().flush(connection.getMetaTableName()); Thread.sleep( conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 30000) * 6); List regions = MetaTableAccessor.getTableRegions(c, TABLE); @@ -115,7 +115,7 @@ public static void shutdownMetaAndDoValidations(HBaseTestingUtil util) throws Ex Thread.sleep(10); hrl = MetaTableAccessor.getRegionLocation(c, regions.get(0)); } while (primary.equals(hrl.getServerName())); - util.getAdmin().flush(MetaTableName.getInstance()); + util.getAdmin().flush(connection.getMetaTableName()); Thread.sleep( conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 30000) * 3); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java index 5e2a11a13dc8..6d774cd366f6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java @@ -49,7 +49,7 @@ public class TestMultiActionMetricsFromClient { public static void setUpBeforeClass() throws Exception { TEST_UTIL.startMiniCluster(1); TEST_UTIL.getHBaseCluster().waitForActiveAndReadyMaster(); - TEST_UTIL.waitUntilAllRegionsAssigned(MetaTableName.getInstance()); + TEST_UTIL.waitUntilAllRegionsAssigned(connection.getMetaTableName()); TEST_UTIL.createTable(TABLE_NAME, FAMILY); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java index a453646e4247..67107441cae8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java @@ -262,7 +262,7 @@ public static void beforeClass() throws Exception { HTU.startMiniCluster(NB_SERVERS); // Enable meta replica at server side - HBaseTestingUtil.setReplicas(HTU.getAdmin(), MetaTableName.getInstance(), 2); + HBaseTestingUtil.setReplicas(HTU.getAdmin(), connection.getMetaTableName(), 2); HTU.getHBaseCluster().startMaster(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java index 4c24dcc10949..8870fcc26171 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java @@ -72,7 +72,7 @@ public static void setUpBeforeClass() throws Exception { UTIL.getConfiguration().setLong(RpcConnectionRegistry.MIN_SECS_BETWEEN_REFRESHES, 0); UTIL.getConfiguration().setLong(BootstrapNodeManager.REQUEST_MASTER_MIN_INTERVAL_SECS, 1); UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), MetaTableName.getInstance(), 3); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), connection.getMetaTableName(), 3); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java index c196dded6618..eb36db653ab6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java @@ -281,13 +281,13 @@ public void testAsyncTable() throws Exception { public void testChangeMetaReplicaCount() throws Exception { Admin admin = TEST_UTIL.getAdmin(); try (RegionLocator locator = - TEST_UTIL.getConnection().getRegionLocator(MetaTableName.getInstance())) { + TEST_UTIL.getConnection().getRegionLocator(connection.getMetaTableName())) { assertEquals(1, locator.getAllRegionLocations().size()); - HBaseTestingUtil.setReplicas(admin, MetaTableName.getInstance(), 3); + HBaseTestingUtil.setReplicas(admin, connection.getMetaTableName(), 3); TEST_UTIL.waitFor(30000, () -> locator.getAllRegionLocations().size() == 3); - HBaseTestingUtil.setReplicas(admin, MetaTableName.getInstance(), 2); + HBaseTestingUtil.setReplicas(admin, connection.getMetaTableName(), 2); TEST_UTIL.waitFor(30000, () -> locator.getAllRegionLocations().size() == 2); - HBaseTestingUtil.setReplicas(admin, MetaTableName.getInstance(), 1); + HBaseTestingUtil.setReplicas(admin, connection.getMetaTableName(), 1); TEST_UTIL.waitFor(30000, () -> locator.getAllRegionLocations().size() == 1); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java index f24a894085bd..6fa2167d9a2b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java @@ -50,7 +50,7 @@ public void testShutdownOfReplicaHolder() throws Exception { // checks that the when the server holding meta replica is shut down, the meta replica // can be recovered try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); - RegionLocator locator = conn.getRegionLocator(MetaTableName.getInstance())) { + RegionLocator locator = conn.getRegionLocator(connection.getMetaTableName())) { HRegionLocation hrl = locator.getRegionLocations(HConstants.EMPTY_START_ROW, true).get(1); ServerName oldServer = hrl.getServerName(); TEST_UTIL.getHBaseClusterInterface().killRegionServer(oldServer); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java index 693713c572d5..45af730962ab 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java @@ -164,7 +164,7 @@ public static void cleanupTest() throws Exception { */ @Test(expected = IllegalArgumentException.class) public void testMetaTablesSnapshot() throws Exception { - UTIL.getAdmin().snapshot("metaSnapshot", MetaTableName.getInstance()); + UTIL.getAdmin().snapshot("metaSnapshot", connection.getMetaTableName()); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java index 474f8153ca3d..63bfacf4e43b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java @@ -64,7 +64,7 @@ public class TestZKConnectionRegistry { @BeforeClass public static void setUp() throws Exception { TEST_UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), MetaTableName.getInstance(), 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), connection.getMetaTableName(), 3); REGISTRY = new ZKConnectionRegistry(TEST_UTIL.getConfiguration(), null); } @@ -89,7 +89,7 @@ public void test() throws InterruptedException, ExecutionException, IOException IntStream.range(0, 3).forEach(i -> { HRegionLocation loc = locs.getRegionLocation(i); assertNotNull("Replica " + i + " doesn't have location", loc); - assertEquals(MetaTableName.getInstance(), loc.getRegion().getTable()); + assertEquals(connection.getMetaTableName(), loc.getRegion().getTable()); assertEquals(i, loc.getRegion().getReplicaId()); }); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java index e42af8bbed93..c96b29ff7a39 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java @@ -201,7 +201,7 @@ public void testTableActionsAvailableForAdmins() throws Exception { @Override public Void run() throws Exception { // Check the expected content is present in the http response - Pair pair = getTablePage(MetaTableName.getInstance()); + Pair pair = getTablePage(connection.getMetaTableName()); assertEquals(HttpURLConnection.HTTP_OK, pair.getFirst().intValue()); assertTrue("expected=" + expectedAuthorizedContent + ", content=" + pair.getSecond(), pair.getSecond().contains(expectedAuthorizedContent)); @@ -214,7 +214,7 @@ public Void run() throws Exception { nonAdmin.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - Pair pair = getTablePage(MetaTableName.getInstance()); + Pair pair = getTablePage(connection.getMetaTableName()); assertEquals(HttpURLConnection.HTTP_OK, pair.getFirst().intValue()); assertFalse( "should not find=" + expectedAuthorizedContent + ", content=" + pair.getSecond(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java index d09da4fefafd..14f4ed153709 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java @@ -244,7 +244,7 @@ public void testFlushedSequenceIdPersistLoad() throws Exception { .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf"))).build(); Table table = TEST_UTIL.createTable(tableDescriptor, null); // flush META region - TEST_UTIL.flush(MetaTableName.getInstance()); + TEST_UTIL.flush(connection.getMetaTableName()); // wait for regionserver report Threads.sleep(msgInterval * 2); // record flush seqid before cluster shutdown diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java index 6abc5e7d680d..9f981b015da6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java @@ -64,7 +64,7 @@ public static void teardownTest() throws Exception { @Test public void tesMetaDescriptorHasSFTConfig() throws Exception { - TableDescriptor descriptor = UTIL.getAdmin().getDescriptor(MetaTableName.getInstance()); + TableDescriptor descriptor = UTIL.getAdmin().getDescriptor(connection.getMetaTableName()); assertEquals(FILE.name(), descriptor.getValue(TRACKER_IMPL)); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java index 86cb2ba4af3d..eeabc0e48818 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java @@ -308,7 +308,7 @@ public void testIncompleteMetaTableReplicaInformation() throws Exception { ADMIN.disableTable(tableName); // now delete one replica info from all the rows // this is to make the meta appear to be only partially updated - Table metaTable = ADMIN.getConnection().getTable(MetaTableName.getInstance()); + Table metaTable = ADMIN.getConnection().getTable(connection.getMetaTableName()); for (byte[] row : tableRows) { Delete deleteOneReplicaLocation = new Delete(row); deleteOneReplicaLocation.addColumns(HConstants.CATALOG_FAMILY, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java index 46ff1a582235..87bda4fbaf61 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java @@ -94,7 +94,7 @@ public void testNewCluster() throws Exception { Connection conn = TEST_UTIL.getConnection(); assertTrue(conn.getAdmin().isMasterInMaintenanceMode()); - try (Table table = conn.getTable(MetaTableName.getInstance()); + try (Table table = conn.getTable(connection.getMetaTableName()); ResultScanner scanner = table.getScanner(new Scan())) { assertNotNull("Could not read meta.", scanner.next()); } @@ -121,7 +121,7 @@ public void testExistingCluster() throws Exception { Connection conn = TEST_UTIL.getConnection(); assertTrue(conn.getAdmin().isMasterInMaintenanceMode()); - try (Table table = conn.getTable(MetaTableName.getInstance()); + try (Table table = conn.getTable(connection.getMetaTableName()); ResultScanner scanner = table.getScanner(HConstants.TABLE_FAMILY); Stream results = StreamSupport.stream(scanner.spliterator(), false)) { assertTrue("Did not find user table records while reading hbase:meta", diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java index 253c5899c426..5b1fb7a0518d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java @@ -302,7 +302,7 @@ public void testKillRSWithOpeningRegion2482() throws Exception { */ private static int addToEachStartKey(final int expected) throws IOException { Table t = TEST_UTIL.getConnection().getTable(TABLENAME); - Table meta = TEST_UTIL.getConnection().getTable(MetaTableName.getInstance()); + Table meta = TEST_UTIL.getConnection().getTable(connection.getMetaTableName()); int rows = 0; Scan scan = new Scan(); scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java index df8dca55a09e..d98a57eea38f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java @@ -66,7 +66,7 @@ public static void tearDownAfterClass() throws Exception { @Test public void testStopActiveMaster() throws Exception { try (Connection conn = ConnectionFactory.createConnection(UTIL.getConfiguration()); - RegionLocator locator = conn.getRegionLocator(MetaTableName.getInstance())) { + RegionLocator locator = conn.getRegionLocator(connection.getMetaTableName())) { ServerName oldMetaServer = locator.getAllRegionLocations().get(0).getServerName(); ServerName oldMaster = UTIL.getMiniHBaseCluster().getMaster().getServerName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java index bd3c9398a13b..443ce15e8545 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java @@ -68,7 +68,7 @@ public class TestMigrateAndMirrorMetaLocations { @BeforeClass public static void setUp() throws Exception { UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), MetaTableName.getInstance(), 2); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), connection.getMetaTableName(), 2); } @AfterClass @@ -143,20 +143,20 @@ public void test() throws Exception { } // wait until all meta regions have been assigned UTIL.waitFor(30000, - () -> UTIL.getMiniHBaseCluster().getRegions(MetaTableName.getInstance()).size() == 2); + () -> UTIL.getMiniHBaseCluster().getRegions(connection.getMetaTableName()).size() == 2); // make sure all the SCPs are finished waitUntilNoSCP(); checkMirrorLocation(2); // increase replica count to 3 - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), MetaTableName.getInstance(), 3); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), connection.getMetaTableName(), 3); checkMirrorLocation(3); byte[] replica2Data = ZKUtil.getData(UTIL.getZooKeeperWatcher(), UTIL.getZooKeeperWatcher().getZNodePaths().getZNodeForReplica(2)); // decrease replica count to 1 - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), MetaTableName.getInstance(), 1); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), connection.getMetaTableName(), 1); checkMirrorLocation(1); // restart the whole cluster, put an extra replica znode on zookeeper, to see if we will remove diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java index 6e37bf1be5c6..72b9be888eda 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java @@ -78,7 +78,7 @@ public static final class SuspendProcedure extends Procedure @Override public TableName getTableName() { - return MetaTableName.getInstance(); + return connection.getMetaTableName(); } @Override @@ -155,7 +155,7 @@ public static void tearDown() throws Exception { private void removeNamespaceFamily() throws IOException { FileSystem fs = UTIL.getTestFileSystem(); Path rootDir = CommonFSUtils.getRootDir(UTIL.getConfiguration()); - Path tableDir = CommonFSUtils.getTableDir(rootDir, MetaTableName.getInstance()); + Path tableDir = CommonFSUtils.getTableDir(rootDir, connection.getMetaTableName()); TableDescriptor metaTableDesc = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir); TableDescriptor noNsMetaTableDesc = TableDescriptorBuilder.newBuilder(metaTableDesc) .removeColumnFamily(HConstants.NAMESPACE_FAMILY).build(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java index 19e05e26f551..1d0d6f85cc6d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java @@ -127,7 +127,7 @@ private void validateRecreateClusterWithUserTableEnabled(boolean cleanupWALs, private void restartHBaseCluster(boolean cleanUpWALs, boolean cleanUpZnodes) throws Exception { // flush cache so that everything is on disk - TEST_UTIL.getMiniHBaseCluster().flushcache(MetaTableName.getInstance()); + TEST_UTIL.getMiniHBaseCluster().flushcache(connection.getMetaTableName()); TEST_UTIL.getMiniHBaseCluster().flushcache(); List oldServers = @@ -178,7 +178,7 @@ private void prepareDataBeforeRecreate(HBaseTestingUtil testUtil, TableName tabl put.addColumn(Bytes.toBytes("f"), Bytes.toBytes("c"), Bytes.toBytes("v")); table.put(put); - ensureTableNotColocatedWithSystemTable(tableName, MetaTableName.getInstance()); + ensureTableNotColocatedWithSystemTable(tableName, connection.getMetaTableName()); } private void ensureTableNotColocatedWithSystemTable(TableName userTable, TableName systemTable) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java index a6b11a569875..2c8ef58a7774 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java @@ -81,7 +81,7 @@ public void testRestart() throws IOException, InterruptedException { table.put(new Put(row).addColumn(FAMILY, QUALIFIER, row)); } // flush all in memory data - UTIL.flush(MetaTableName.getInstance()); + UTIL.flush(connection.getMetaTableName()); UTIL.flush(NAME); // stop master first, so when stopping region server, we will not schedule a SCP. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java index f20b06c7b003..cabc9bba5904 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java @@ -64,13 +64,13 @@ public static void tearDown() throws Exception { public void test() throws Exception { RegionServerThread rsThread = null; for (RegionServerThread t : UTIL.getMiniHBaseCluster().getRegionServerThreads()) { - if (!t.getRegionServer().getRegions(MetaTableName.getInstance()).isEmpty()) { + if (!t.getRegionServer().getRegions(connection.getMetaTableName()).isEmpty()) { rsThread = t; break; } } HRegionServer rs = rsThread.getRegionServer(); - RegionInfo hri = rs.getRegions(MetaTableName.getInstance()).get(0).getRegionInfo(); + RegionInfo hri = rs.getRegions(connection.getMetaTableName()).get(0).getRegionInfo(); HMaster master = UTIL.getMiniHBaseCluster().getMaster(); ProcedureExecutor executor = master.getMasterProcedureExecutor(); DummyRegionProcedure proc = new DummyRegionProcedure(executor.getEnvironment(), hri); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java index ce23d2342154..d52a7458743c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java @@ -151,11 +151,11 @@ public static void setUp() throws Exception { UTIL.createTable(TABLE_NAME, CF); UTIL.getAdmin().balancerSwitch(false, true); HRegionServer srcRs = UTIL.getRSForFirstRegionInTable(TABLE_NAME); - if (!srcRs.getRegions(MetaTableName.getInstance()).isEmpty()) { - RegionInfo metaRegion = srcRs.getRegions(MetaTableName.getInstance()).get(0).getRegionInfo(); + if (!srcRs.getRegions(connection.getMetaTableName()).isEmpty()) { + RegionInfo metaRegion = srcRs.getRegions(connection.getMetaTableName()).get(0).getRegionInfo(); HRegionServer dstRs = UTIL.getOtherRegionServer(srcRs); UTIL.getAdmin().move(metaRegion.getEncodedNameAsBytes(), dstRs.getServerName()); - UTIL.waitFor(30000, () -> !dstRs.getRegions(MetaTableName.getInstance()).isEmpty()); + UTIL.waitFor(30000, () -> !dstRs.getRegions(connection.getMetaTableName()).isEmpty()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java index 6941fe667e39..0a21143bb2e0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java @@ -95,7 +95,7 @@ public static void setUp() throws Exception { Configuration conf = UTIL.getConfiguration(); conf.setClass(HConstants.MASTER_IMPL, HMasterForTest.class, HMaster.class); UTIL.startMiniCluster(1); - UTIL.waitTableAvailable(MetaTableName.getInstance()); + UTIL.waitTableAvailable(connection.getMetaTableName()); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java index 76935cea8968..7af43fd41336 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java @@ -122,7 +122,7 @@ public void testVisitMetaForBadRegionState() throws Exception { put.addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER, Bytes.toBytes("BAD_STATE")); - try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { + try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { table.put(put); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java index e8a2f5202b25..7f9f63883445 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java @@ -168,7 +168,7 @@ public void describeTo(Description description) { @Test public void testFailAndRollback() throws Exception { - HRegionServer rsWithMeta = UTIL.getRSForFirstRegionInTable(MetaTableName.getInstance()); + HRegionServer rsWithMeta = UTIL.getRSForFirstRegionInTable(connection.getMetaTableName()); UTIL.getMiniHBaseCluster().killRegionServer(rsWithMeta.getServerName()); UTIL.waitFor(15000, () -> getSCPForServer(rsWithMeta.getServerName()) != null); ServerCrashProcedure scp = getSCPForServer(rsWithMeta.getServerName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java index 8a1f7b5601cb..2311d13c17f8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java @@ -147,7 +147,7 @@ static void validateReplicaDistribution(Connection connection, TableName tableNa static void validateRegionLocations(Map> tableToServers, TableName productTableName, boolean shouldBeBalanced) { ServerName metaServer = - tableToServers.get(MetaTableName.getInstance()).stream().findFirst().orElseThrow(); + tableToServers.get(connection.getMetaTableName()).stream().findFirst().orElseThrow(); ServerName quotaServer = tableToServers.get(QuotaUtil.QUOTA_TABLE_NAME).stream().findFirst().orElseThrow(); Set productServers = tableToServers.get(productTableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java index 068b15fbd662..e2f063cdbec0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java @@ -162,7 +162,7 @@ public void testRoundRobinAssignment() throws Exception { LoadBalancer balancer = master.getLoadBalancer(); List regions = admin.getRegions(tableName); - regions.addAll(admin.getRegions(MetaTableName.getInstance())); + regions.addAll(admin.getRegions(connection.getMetaTableName())); List servers = Lists.newArrayList( admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet()); Map> map = balancer.roundRobinAssignment(regions, servers); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java index d179916cc6b7..c0833525c091 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java @@ -94,7 +94,7 @@ public void testTableIsolation() throws Exception { BalancerConditionalsTestUtil.generateSplits(2 * NUM_SERVERS)); Set tablesToBeSeparated = - ImmutableSet. builder().add(MetaTableName.getInstance()) + ImmutableSet. builder().add(connection.getMetaTableName()) .add(QuotaUtil.QUOTA_TABLE_NAME).add(productTableName).build(); // Pause the balancer @@ -149,7 +149,7 @@ private static void validateRegionLocations(Map> tabl TableName productTableName, boolean shouldBeBalanced) { // Validate that the region assignments ServerName metaServer = - tableToServers.get(MetaTableName.getInstance()).stream().findFirst().orElseThrow(); + tableToServers.get(connection.getMetaTableName()).stream().findFirst().orElseThrow(); ServerName quotaServer = tableToServers.get(QuotaUtil.QUOTA_TABLE_NAME).stream().findFirst().orElseThrow(); Set productServers = tableToServers.get(productTableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java index 825fba807562..7824114cecce 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java @@ -95,7 +95,7 @@ public static void tearDownAfterClass() throws Exception { @After public void tearDown() throws IOException { - try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance()); + try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName()); ResultScanner scanner = table.getScanner(new Scan().addFamily(HConstants.CATALOG_FAMILY) .addFamily(HConstants.REPLICATION_BARRIER_FAMILY).setFilter(new FirstKeyOnlyFilter()))) { for (;;) { @@ -149,20 +149,20 @@ private void addBarrier(RegionInfo region, long... barriers) throws IOException put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER, put.getTimestamp() - barriers.length + i, Bytes.toBytes(barriers[i])); } - try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { + try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { table.put(put); } } private void fillCatalogFamily(RegionInfo region) throws IOException { - try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { + try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { table.put(new Put(region.getRegionName()).addColumn(HConstants.CATALOG_FAMILY, Bytes.toBytes("whatever"), Bytes.toBytes("whatever"))); } } private void clearCatalogFamily(RegionInfo region) throws IOException { - try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { + try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { table.delete(new Delete(region.getRegionName()).addFamily(HConstants.CATALOG_FAMILY)); } } @@ -282,7 +282,7 @@ public void testDeleteRowForDeletedRegion() throws IOException, ReplicationExcep // No catalog family, then we should remove the whole row clearCatalogFamily(region); cleaner.chore(); - try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { + try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { assertFalse(table .exists(new Get(region.getRegionName()).addFamily(HConstants.REPLICATION_BARRIER_FAMILY))); } @@ -304,7 +304,7 @@ public void testDeleteRowForDeletedRegionNoPeers() throws IOException { // There are no peers, and no catalog family for this region either, so we should remove the // barriers. And since there is no catalog family, after we delete the barrier family, the whole // row is deleted. - try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { + try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { assertFalse(table.exists(new Get(region.getRegionName()))); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java index b610c2750041..0bc050343e7e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java @@ -183,7 +183,7 @@ private PairOfSameType waitOnDaughters(final RegionInfo r) throws IO long start = EnvironmentEdgeManager.currentTime(); PairOfSameType pair = null; try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); - Table metaTable = conn.getTable(MetaTableName.getInstance())) { + Table metaTable = conn.getTable(connection.getMetaTableName())) { Result result = null; RegionInfo region = null; while ((EnvironmentEdgeManager.currentTime() - start) < 60000) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java index 880a3ffbd812..8fc1d7cb6987 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java @@ -50,21 +50,21 @@ public class TestMetaFixerNoCluster { private static byte[] D = Bytes.toBytes("d"); private static RegionInfo ALL = RegionInfoBuilder.FIRST_META_REGIONINFO; private static RegionInfo _ARI = - RegionInfoBuilder.newBuilder(MetaTableName.getInstance()).setEndKey(A).build(); + RegionInfoBuilder.newBuilder(connection.getMetaTableName()).setEndKey(A).build(); private static RegionInfo _BRI = - RegionInfoBuilder.newBuilder(MetaTableName.getInstance()).setEndKey(B).build(); + RegionInfoBuilder.newBuilder(connection.getMetaTableName()).setEndKey(B).build(); private static RegionInfo ABRI = - RegionInfoBuilder.newBuilder(MetaTableName.getInstance()).setStartKey(A).setEndKey(B).build(); + RegionInfoBuilder.newBuilder(connection.getMetaTableName()).setStartKey(A).setEndKey(B).build(); private static RegionInfo ACRI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(MetaTableName.getInstance()).setStartKey(A).setEndKey(C).build(); + .newBuilder(connection.getMetaTableName()).setStartKey(A).setEndKey(C).build(); private static RegionInfo CDRI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(MetaTableName.getInstance()).setStartKey(C).setEndKey(D).build(); + .newBuilder(connection.getMetaTableName()).setStartKey(C).setEndKey(D).build(); private static RegionInfo ADRI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(MetaTableName.getInstance()).setStartKey(A).setEndKey(D).build(); + .newBuilder(connection.getMetaTableName()).setStartKey(A).setEndKey(D).build(); private static RegionInfo D_RI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(MetaTableName.getInstance()).setStartKey(D).build(); + .newBuilder(connection.getMetaTableName()).setStartKey(D).build(); private static RegionInfo C_RI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(MetaTableName.getInstance()).setStartKey(C).build(); + .newBuilder(connection.getMetaTableName()).setStartKey(C).build(); @Test public void testGetRegionInfoWithLargestEndKey() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java index 62aea9177513..eb97f2f2a1f3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java @@ -105,7 +105,7 @@ public void before() { @Test public void testNoNormalizationForMetaTable() { - TableName testTable = MetaTableName.getInstance(); + TableName testTable = connection.getMetaTableName(); TableDescriptor testMetaTd = TableDescriptorBuilder.newBuilder(testTable).build(); List RegionInfo = new ArrayList<>(); Map regionSizes = new HashMap<>(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java index a9ebb47a6d84..73d467cd8ecc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java @@ -262,7 +262,7 @@ public String toString() { private static class PrimaryNotMetaRegionSelector extends RegionSelector { @Override boolean regionFilter(final RegionInfo info) { - return !Objects.equals(MetaTableName.getInstance(), info.getTable()) + return !Objects.equals(connection.getMetaTableName(), info.getTable()) && Objects.equals(RegionInfo.DEFAULT_REPLICA_ID, info.getReplicaId()); } @@ -278,7 +278,7 @@ Exception regionFilterFailure() { private static class ReplicaNonMetaRegionSelector extends RegionSelector { @Override boolean regionFilter(RegionInfo info) { - return !Objects.equals(MetaTableName.getInstance(), info.getTable()) + return !Objects.equals(connection.getMetaTableName(), info.getTable()) && !Objects.equals(RegionInfo.DEFAULT_REPLICA_ID, info.getReplicaId()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java index c5e99471a7f6..314ff36368a1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java @@ -148,7 +148,7 @@ public static void tearDown() throws Exception { @Test public void test() throws Exception { RegionServerThread rsWithMetaThread = UTIL.getMiniHBaseCluster().getRegionServerThreads() - .stream().filter(t -> !t.getRegionServer().getRegions(MetaTableName.getInstance()).isEmpty()) + .stream().filter(t -> !t.getRegionServer().getRegions(connection.getMetaTableName()).isEmpty()) .findAny().get(); HRegionServer rsNoMeta = UTIL.getOtherRegionServer(rsWithMetaThread.getRegionServer()); FAIL = true; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java index cf58f01b7843..13a30cf559de 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java @@ -123,7 +123,7 @@ public static class MetaTableProcedure extends Procedure @Override public TableName getTableName() { - return MetaTableName.getInstance(); + return connection.getMetaTableName(); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java index 1fcf695da810..e2518790e4ae 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java @@ -132,7 +132,7 @@ public void test() throws Exception { HRegionServer regionSvr = UTIL.getRSForFirstRegionInTable(TABLE_NAME); HRegion region = regionSvr.getRegions(TABLE_NAME).get(0); String regName = region.getRegionInfo().getEncodedName(); - List metaRegs = regionSvr.getRegions(MetaTableName.getInstance()); + List metaRegs = regionSvr.getRegions(connection.getMetaTableName()); if (metaRegs != null && !metaRegs.isEmpty()) { LOG.info("meta is on the same server: " + regionSvr); // when region is on same server as hbase:meta, reassigning meta would abort the server diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java index 801652080107..e639cd3a1ed0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java @@ -1003,7 +1003,7 @@ public void testShouldFlushMeta() throws Exception { TableDescriptors tds = new FSTableDescriptors(conf); FSTableDescriptors.tryUpdateMetaTableDescriptor(conf); HRegion meta = HRegion.createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, testDir, conf, - tds.get(MetaTableName.getInstance()), + tds.get(connection.getMetaTableName()), wFactory.getWAL(RegionInfoBuilder.FIRST_META_REGIONINFO)); // parameterized tests add [#] suffix get rid of [ and ]. TableDescriptor desc = TableDescriptorBuilder diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java index ddcb9cec5446..89f33e8d4b22 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java @@ -430,7 +430,7 @@ public static void blockUntilRegionSplit(Configuration conf, long timeout, log("blocking until region is split:" + Bytes.toStringBinary(regionName)); RegionInfo daughterA = null, daughterB = null; try (Connection conn = ConnectionFactory.createConnection(conf); - Table metaTable = conn.getTable(MetaTableName.getInstance())) { + Table metaTable = conn.getTable(connection.getMetaTableName())) { Result result = null; RegionInfo region = null; while ((EnvironmentEdgeManager.currentTime() - start) < timeout) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java index 900adcc38757..ed110896c03f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java @@ -93,7 +93,7 @@ public void testUsingMetaAndBinary() throws IOException { // Up flush size else we bind up when we use default catalog flush of 16k. TableDescriptors tds = new FSTableDescriptors(UTIL.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(UTIL.getConfiguration()); - TableDescriptor td = tds.get(MetaTableName.getInstance()); + TableDescriptor td = tds.get(connection.getMetaTableName()); td = TableDescriptorBuilder.newBuilder(td).setMemStoreFlushSize(64 * 1024 * 1024).build(); HRegion mr = HBaseTestingUtil.createRegionAndWAL(RegionInfoBuilder.FIRST_META_REGIONINFO, rootdir, conf, td); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java index ebc7c4d13d8d..907b666b74fd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java @@ -73,12 +73,12 @@ public void testReadAndWriteRegionInfoFile() throws IOException, InterruptedExce FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(FS, ROOT_DIR); FSTableDescriptors.tryUpdateAndGetMetaTableDescriptor(CONF, FS, ROOT_DIR); HRegion r = HBaseTestingUtil.createRegionAndWAL(ri, ROOT_DIR, CONF, - fsTableDescriptors.get(MetaTableName.getInstance())); + fsTableDescriptors.get(connection.getMetaTableName())); // Get modtime on the file. long modtime = getModTime(r); HBaseTestingUtil.closeRegionAndWAL(r); Thread.sleep(1001); - r = HRegion.openHRegion(ROOT_DIR, ri, fsTableDescriptors.get(MetaTableName.getInstance()), null, + r = HRegion.openHRegion(ROOT_DIR, ri, fsTableDescriptors.get(connection.getMetaTableName()), null, CONF); // Ensure the file is not written for a second time. long modtime2 = getModTime(r); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java index fc0a2dd8a470..563d8c3ba7ed 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java @@ -72,7 +72,7 @@ public class TestRegionInfo { public void testIsStart() { assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isFirst()); org.apache.hadoop.hbase.client.RegionInfo ri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(MetaTableName.getInstance()).setStartKey(Bytes.toBytes("not_start")).build(); + .newBuilder(connection.getMetaTableName()).setStartKey(Bytes.toBytes("not_start")).build(); assertFalse(ri.isFirst()); } @@ -80,7 +80,7 @@ public void testIsStart() { public void testIsEnd() { assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isFirst()); org.apache.hadoop.hbase.client.RegionInfo ri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(MetaTableName.getInstance()).setEndKey(Bytes.toBytes("not_end")).build(); + .newBuilder(connection.getMetaTableName()).setEndKey(Bytes.toBytes("not_end")).build(); assertFalse(ri.isLast()); } @@ -88,9 +88,9 @@ public void testIsEnd() { public void testIsNext() { byte[] bytes = Bytes.toBytes("row"); org.apache.hadoop.hbase.client.RegionInfo ri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(MetaTableName.getInstance()).setEndKey(bytes).build(); + .newBuilder(connection.getMetaTableName()).setEndKey(bytes).build(); org.apache.hadoop.hbase.client.RegionInfo ri2 = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(MetaTableName.getInstance()).setStartKey(bytes).build(); + .newBuilder(connection.getMetaTableName()).setStartKey(bytes).build(); assertFalse(ri.isNext(RegionInfoBuilder.FIRST_META_REGIONINFO)); assertTrue(ri.isNext(ri2)); } @@ -103,18 +103,18 @@ public void testIsOverlap() { byte[] d = Bytes.toBytes("d"); org.apache.hadoop.hbase.client.RegionInfo all = RegionInfoBuilder.FIRST_META_REGIONINFO; org.apache.hadoop.hbase.client.RegionInfo ari = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(MetaTableName.getInstance()).setEndKey(a).build(); + .newBuilder(connection.getMetaTableName()).setEndKey(a).build(); org.apache.hadoop.hbase.client.RegionInfo abri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(MetaTableName.getInstance()) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(connection.getMetaTableName()) .setStartKey(a).setEndKey(b).build(); org.apache.hadoop.hbase.client.RegionInfo adri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(MetaTableName.getInstance()) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(connection.getMetaTableName()) .setStartKey(a).setEndKey(d).build(); org.apache.hadoop.hbase.client.RegionInfo cdri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(MetaTableName.getInstance()) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(connection.getMetaTableName()) .setStartKey(c).setEndKey(d).build(); org.apache.hadoop.hbase.client.RegionInfo dri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(MetaTableName.getInstance()).setStartKey(d).build(); + .newBuilder(connection.getMetaTableName()).setStartKey(d).build(); assertTrue(all.isOverlap(all)); assertTrue(all.isOverlap(abri)); assertFalse(abri.isOverlap(cdri)); @@ -141,17 +141,17 @@ public void testIsOverlaps() { byte[] e = Bytes.toBytes("e"); byte[] f = Bytes.toBytes("f"); org.apache.hadoop.hbase.client.RegionInfo ari = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(MetaTableName.getInstance()).setEndKey(a).build(); + .newBuilder(connection.getMetaTableName()).setEndKey(a).build(); org.apache.hadoop.hbase.client.RegionInfo abri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(MetaTableName.getInstance()) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(connection.getMetaTableName()) .setStartKey(a).setEndKey(b).build(); org.apache.hadoop.hbase.client.RegionInfo eri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(MetaTableName.getInstance()).setEndKey(e).build(); + .newBuilder(connection.getMetaTableName()).setEndKey(e).build(); org.apache.hadoop.hbase.client.RegionInfo cdri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(MetaTableName.getInstance()) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(connection.getMetaTableName()) .setStartKey(c).setEndKey(d).build(); org.apache.hadoop.hbase.client.RegionInfo efri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(MetaTableName.getInstance()) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(connection.getMetaTableName()) .setStartKey(e).setEndKey(f).build(); assertFalse(ari.isOverlap(abri)); assertTrue(abri.isOverlap(eri)); @@ -176,12 +176,12 @@ public void testReadAndWriteHRegionInfoFile() throws IOException, InterruptedExc FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(htu.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(htu.getConfiguration()); HRegion r = HBaseTestingUtil.createRegionAndWAL(hri, basedir, htu.getConfiguration(), - fsTableDescriptors.get(MetaTableName.getInstance())); + fsTableDescriptors.get(connection.getMetaTableName())); // Get modtime on the file. long modtime = getModTime(r); HBaseTestingUtil.closeRegionAndWAL(r); Thread.sleep(1001); - r = HRegion.openHRegion(basedir, hri, fsTableDescriptors.get(MetaTableName.getInstance()), null, + r = HRegion.openHRegion(basedir, hri, fsTableDescriptors.get(connection.getMetaTableName()), null, htu.getConfiguration()); // Ensure the file is not written for a second time. long modtime2 = getModTime(r); @@ -255,7 +255,7 @@ public void testContainsRange() { @Test public void testContainsRangeForMetaTable() { TableDescriptor tableDesc = - TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).build(); + TableDescriptorBuilder.newBuilder(connection.getMetaTableName()).build(); RegionInfo hri = RegionInfoBuilder.newBuilder(tableDesc.getTableName()).build(); byte[] startRow = HConstants.EMPTY_START_ROW; byte[] row1 = Bytes.toBytes("a,a,0"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java index 559a7e72832c..c3f6f5bc16f9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java @@ -148,7 +148,7 @@ public void testRegionReplicaUpdatesMetaLocation() throws Exception { openRegion(HTU, getRS(), hriSecondary); Table meta = null; try { - meta = HTU.getConnection().getTable(MetaTableName.getInstance()); + meta = HTU.getConnection().getTable(connection.getMetaTableName()); TestMetaTableAccessor.assertMetaLocation(meta, hriPrimary.getRegionName(), getRS().getServerName(), -1, 1, false); } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java index bb2b2f0abd0b..cd1bb0866036 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java @@ -66,7 +66,7 @@ public static void setUp() throws Exception { UTIL.createTable(TABLE_NAME, CF); UTIL.waitTableAvailable(TABLE_NAME); HRegionServer rs = UTIL.getRSForFirstRegionInTable(TABLE_NAME); - if (!rs.getRegions(MetaTableName.getInstance()).isEmpty()) { + if (!rs.getRegions(connection.getMetaTableName()).isEmpty()) { HRegionServer rs1 = UTIL.getOtherRegionServer(rs); UTIL.moveRegionAndWait( UTIL.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0).getRegionInfo(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java index e21606458761..8da716fe6a9a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java @@ -98,12 +98,12 @@ public static void stopMasterAndCacheMetaLocation(HBaseTestingUtil HTU) // cache meta location, so we will not go to master to lookup meta region location for (JVMClusterUtil.RegionServerThread t : HTU.getMiniHBaseCluster().getRegionServerThreads()) { try (RegionLocator locator = - t.getRegionServer().getConnection().getRegionLocator(MetaTableName.getInstance())) { + t.getRegionServer().getConnection().getRegionLocator(connection.getMetaTableName())) { locator.getAllRegionLocations(); } } try ( - RegionLocator locator = HTU.getConnection().getRegionLocator(MetaTableName.getInstance())) { + RegionLocator locator = HTU.getConnection().getRegionLocator(connection.getMetaTableName())) { locator.getAllRegionLocations(); } // Stop master diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java index e42209ae78b5..f3be3bcd9192 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java @@ -136,7 +136,7 @@ public void testRejectRequestsOnAbort() throws Exception { .getRegionServerThreads()) { HRegionServer regionServer = regionServerThread.getRegionServer(); if ( - regionServer.getRegions(MetaTableName.getInstance()).isEmpty() + regionServer.getRegions(connection.getMetaTableName()).isEmpty() && !regionServer.getRegions(TABLE_NAME).isEmpty() ) { serverWithoutMeta = regionServer; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java index a5ad29af1ea8..49037259d09a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java @@ -136,7 +136,7 @@ public void test() throws Exception { RegionServerThread rst1 = UTIL.getMiniHBaseCluster().getRegionServerThreads().get(1); HRegionServer liveRS; RegionServerThread toKillRSThread; - if (rst1.getRegionServer().getRegions(MetaTableName.getInstance()).isEmpty()) { + if (rst1.getRegionServer().getRegions(connection.getMetaTableName()).isEmpty()) { liveRS = rst0.getRegionServer(); toKillRSThread = rst1; } else { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java index 24e393fbe716..859708769050 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java @@ -339,7 +339,7 @@ void validateData(Table table, int rownum) throws IOException { public void testCompactionRecordDoesntBlockRolling() throws Exception { // When the hbase:meta table can be opened, the region servers are running - try (Table t = TEST_UTIL.getConnection().getTable(MetaTableName.getInstance()); + try (Table t = TEST_UTIL.getConnection().getTable(connection.getMetaTableName()); Table table = createTestTable(getName())) { server = TEST_UTIL.getRSForFirstRegionInTable(table.getName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java index bc8ee0a836d6..c1c2585774d3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java @@ -163,7 +163,7 @@ public void testRSAbortWithUnflushedEdits() throws Exception { LOG.info("Starting testRSAbortWithUnflushedEdits()"); // When the hbase:meta table can be opened, the region servers are running - TEST_UTIL.getConnection().getTable(MetaTableName.getInstance()).close(); + TEST_UTIL.getConnection().getTable(connection.getMetaTableName()).close(); // Create the test table and open it TableName tableName = TableName.valueOf(this.getClass().getSimpleName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java index 3ba45bdf4915..97c3354f35ee 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java @@ -323,7 +323,7 @@ public void testLogRollOnPipelineRestart() throws Exception { fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) > 1); LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS())); // When the hbase:meta table can be opened, the region servers are running - Table t = TEST_UTIL.getConnection().getTable(MetaTableName.getInstance()); + Table t = TEST_UTIL.getConnection().getTable(connection.getMetaTableName()); try { this.server = cluster.getRegionServer(0); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java index 635b8abcc7ab..c849b57378a0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java @@ -95,7 +95,7 @@ public void testContendedLogRolling() throws Exception { CommonFSUtils.setRootDir(conf, dir); FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(TEST_UTIL.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration()); - TableDescriptor metaTableDescriptor = fsTableDescriptors.get(MetaTableName.getInstance()); + TableDescriptor metaTableDescriptor = fsTableDescriptors.get(connection.getMetaTableName()); conf.set(FSHLogProvider.WRITER_IMPL, HighLatencySyncWriter.class.getName()); final WALFactory wals = new WALFactory(conf, TestLogRollingNoCluster.class.getName()); final WAL wal = wals.getWAL(null); @@ -159,7 +159,7 @@ public void run() { try { TableDescriptors tds = new FSTableDescriptors(TEST_UTIL.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration()); - TableDescriptor htd = tds.get(MetaTableName.getInstance()); + TableDescriptor htd = tds.get(connection.getMetaTableName()); for (int i = 0; i < this.count; i++) { long now = EnvironmentEdgeManager.currentTime(); // Roll every ten edits @@ -176,7 +176,7 @@ public void run() { scopes.put(fam, 0); } final long txid = wal.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), - MetaTableName.getInstance(), now, mvcc, scopes), edit); + connection.getMetaTableName(), now, mvcc, scopes), edit); Threads.sleep(ThreadLocalRandom.current().nextInt(5)); wal.sync(txid); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java index 522bd5ac1367..fe09c1c6aa83 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java @@ -73,7 +73,7 @@ public void testSystemTableWALEntryFilter() { // meta WALKeyImpl key1 = new WALKeyImpl(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(), - MetaTableName.getInstance(), EnvironmentEdgeManager.currentTime()); + connection.getMetaTableName(), EnvironmentEdgeManager.currentTime()); Entry metaEntry = new Entry(key1, null); assertNull(filter.filter(metaEntry)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java index c78c967563e2..07247363689e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java @@ -99,10 +99,10 @@ public void before() throws Exception { // conf.setInt(HConstants.META_REPLICAS_NUM, numOfMetaReplica); HTU.startMiniCluster(NB_SERVERS); // Enable hbase:meta replication. - HBaseTestingUtil.setReplicas(HTU.getAdmin(), MetaTableName.getInstance(), numOfMetaReplica); + HBaseTestingUtil.setReplicas(HTU.getAdmin(), connection.getMetaTableName(), numOfMetaReplica); HTU.waitFor(30000, - () -> HTU.getMiniHBaseCluster().getRegions(MetaTableName.getInstance()).size() + () -> HTU.getMiniHBaseCluster().getRegions(connection.getMetaTableName()).size() >= numOfMetaReplica); } @@ -120,37 +120,37 @@ public void testHBaseMetaReplicates() throws Exception { try (Table table = HTU.createTable(TableName.valueOf(this.name.getMethodName() + "_0"), HConstants.CATALOG_FAMILY, Arrays.copyOfRange(HBaseTestingUtil.KEYS, 1, HBaseTestingUtil.KEYS.length))) { - verifyReplication(MetaTableName.getInstance(), numOfMetaReplica, + verifyReplication(connection.getMetaTableName(), numOfMetaReplica, getMetaCells(table.getName())); } try (Table table = HTU.createTable(TableName.valueOf(this.name.getMethodName() + "_1"), HConstants.CATALOG_FAMILY, Arrays.copyOfRange(HBaseTestingUtil.KEYS, 1, HBaseTestingUtil.KEYS.length))) { - verifyReplication(MetaTableName.getInstance(), numOfMetaReplica, + verifyReplication(connection.getMetaTableName(), numOfMetaReplica, getMetaCells(table.getName())); // Try delete. HTU.deleteTableIfAny(table.getName()); - verifyDeletedReplication(MetaTableName.getInstance(), numOfMetaReplica, table.getName()); + verifyDeletedReplication(connection.getMetaTableName(), numOfMetaReplica, table.getName()); } } @Test public void testCatalogReplicaReplicationWithFlushAndCompaction() throws Exception { try (Connection connection = ConnectionFactory.createConnection(HTU.getConfiguration()); - Table table = connection.getTable(MetaTableName.getInstance())) { + Table table = connection.getTable(connection.getMetaTableName())) { // load the data to the table for (int i = 0; i < 5; i++) { LOG.info("Writing data from " + i * 1000 + " to " + (i * 1000 + 1000)); HTU.loadNumericRows(table, HConstants.CATALOG_FAMILY, i * 1000, i * 1000 + 1000); LOG.info("flushing table"); - HTU.flush(MetaTableName.getInstance()); + HTU.flush(connection.getMetaTableName()); LOG.info("compacting table"); if (i < 4) { - HTU.compact(MetaTableName.getInstance(), false); + HTU.compact(connection.getMetaTableName(), false); } } - verifyReplication(MetaTableName.getInstance(), numOfMetaReplica, 0, 5000, + verifyReplication(connection.getMetaTableName(), numOfMetaReplica, 0, 5000, HConstants.CATALOG_FAMILY); } } @@ -185,7 +185,7 @@ public void testCatalogReplicaReplicationWithReplicaMoved() throws Exception { } } try (Connection connection = ConnectionFactory.createConnection(HTU.getConfiguration()); - Table table = connection.getTable(MetaTableName.getInstance())) { + Table table = connection.getTable(connection.getMetaTableName())) { // load the data to the table for (int i = 0; i < 5; i++) { LOG.info("Writing data from " + i * 1000 + " to " + (i * 1000 + 1000)); @@ -195,7 +195,7 @@ public void testCatalogReplicaReplicationWithReplicaMoved() throws Exception { } } - verifyReplication(MetaTableName.getInstance(), numOfMetaReplica, 0, 5000, + verifyReplication(connection.getMetaTableName(), numOfMetaReplica, 0, 5000, HConstants.CATALOG_FAMILY); } } @@ -417,7 +417,7 @@ private void getMetaReplicaReadRequests(final Region[] metaRegions, final long[] @Test public void testHBaseMetaReplicaGets() throws Exception { TableName tn = TableName.valueOf(this.name.getMethodName()); - final Region[] metaRegions = getAllRegions(MetaTableName.getInstance(), numOfMetaReplica); + final Region[] metaRegions = getAllRegions(connection.getMetaTableName(), numOfMetaReplica); long[] readReqsForMetaReplicas = new long[numOfMetaReplica]; long[] readReqsForMetaReplicasAfterGet = new long[numOfMetaReplica]; long[] readReqsForMetaReplicasAfterGetAllLocations = new long[numOfMetaReplica]; @@ -430,7 +430,7 @@ public void testHBaseMetaReplicaGets() throws Exception { try (Table table = HTU.createTable(tn, HConstants.CATALOG_FAMILY, Arrays.copyOfRange(HBaseTestingUtil.KEYS, 1, HBaseTestingUtil.KEYS.length))) { - verifyReplication(MetaTableName.getInstance(), numOfMetaReplica, + verifyReplication(connection.getMetaTableName(), numOfMetaReplica, getMetaCells(table.getName())); // load different values HTU.loadTable(table, new byte[][] { HConstants.CATALOG_FAMILY }, VALUE); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java index c6d01d0e7a51..cfd430fdbc76 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java @@ -197,7 +197,7 @@ public void testWALEntryFilter() throws IOException { assertTrue(wef.filter(e) == e); // Test system WAL edit. e = new WAL.Entry( - new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, MetaTableName.getInstance(), -1, -1, uuid), we); + new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, connection.getMetaTableName(), -1, -1, uuid), we); assertNull(wef.filter(e)); } finally { rs.terminate("Done"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java index be93b44d8a62..65edfcc07dda 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java @@ -163,7 +163,7 @@ private void addStateAndBarrier(RegionInfo region, RegionState.State state, long put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER, put.getTimestamp() - barriers.length + i, Bytes.toBytes(barriers[i])); } - try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { + try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { table.put(put); } } @@ -172,7 +172,7 @@ private void setState(RegionInfo region, RegionState.State state) throws IOExcep Put put = new Put(region.getRegionName(), EnvironmentEdgeManager.currentTime()); put.addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER, Bytes.toBytes(state.name())); - try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { + try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { table.put(put); } } @@ -189,7 +189,7 @@ private void addParents(RegionInfo region, List parents) throws IOEx put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, ReplicationBarrierFamilyFormat.REPLICATION_PARENT_QUALIFIER, ReplicationBarrierFamilyFormat.getParentsBytes(parents)); - try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { + try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { table.put(put); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsCPHookCalled.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsCPHookCalled.java index 9f6019974574..8288784c522a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsCPHookCalled.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsCPHookCalled.java @@ -69,7 +69,7 @@ public void testGetRSGroupInfoCPHookCalled() throws Exception { @Test public void testGetRSGroupInfoOfTableCPHookCalled() throws Exception { - ADMIN.getRSGroup(MetaTableName.getInstance()); + ADMIN.getRSGroup(connection.getMetaTableName()); assertTrue(OBSERVER.preGetRSGroupInfoOfTableCalled); assertTrue(OBSERVER.postGetRSGroupInfoOfTableCalled); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java index 5510ac230111..883fb8d3a843 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java @@ -247,10 +247,10 @@ public void testLowerMetaGroupVersion() throws Exception { // move hbase:meta to meta_group Set toAddTables = new HashSet<>(); - toAddTables.add(MetaTableName.getInstance()); + toAddTables.add(connection.getMetaTableName()); ADMIN.setRSGroup(toAddTables, groupName); assertTrue(ADMIN.getConfiguredNamespacesAndTablesInRSGroup(groupName).getSecond() - .contains(MetaTableName.getInstance())); + .contains(connection.getMetaTableName())); // restart the regionserver in meta_group, and lower its version String originVersion = ""; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java index 3a9ccca4be5e..fb1ec46740f3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java @@ -237,7 +237,7 @@ public void testRunCleanerChore() throws Exception { public void testExecProcedure() throws Exception { verifyAdminCheckForAction((admin) -> { // Using existing table instead of creating a new one. - admin.execProcedure("flush-table-proc", MetaTableName.getInstance().getNameAsString(), + admin.execProcedure("flush-table-proc", connection.getMetaTableName().getNameAsString(), new HashMap<>()); }); } @@ -261,7 +261,7 @@ public void testExecProcedureWithRet() throws Exception { verifyAdminCheckForAction((admin) -> { // Using existing table instead of creating a new one. admin.execProcedureWithReturn("flush-table-proc", - MetaTableName.getInstance().getNameAsString(), new HashMap<>()); + connection.getMetaTableName().getNameAsString(), new HashMap<>()); }); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java index e6b2775f2921..6f973320a1eb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java @@ -97,7 +97,7 @@ public void setUpBeforeMethod() { private void testToken() throws Exception { try (AsyncConnection conn = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get()) { - AsyncTable table = conn.getTable(MetaTableName.getInstance()); + AsyncTable table = conn.getTable(conn.getMetaTableName()); WhoAmIResponse response = table. coprocessorService( AuthenticationService::newStub, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java index 8a941ee65ce4..98b042849644 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java @@ -87,7 +87,7 @@ public static void setupBeforeClass() throws Exception { TEST_UTIL.startMiniCluster(1); TEST_UTIL.getHBaseCluster().waitForActiveAndReadyMaster(); - TEST_UTIL.waitUntilAllRegionsAssigned(MetaTableName.getInstance()); + TEST_UTIL.waitUntilAllRegionsAssigned(connection.getMetaTableName()); rootDir = CommonFSUtils.getRootDir(conf); fs = TEST_UTIL.getTestFileSystem(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java index 0d0bd7b81b1c..22ac5fc57658 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java @@ -200,7 +200,7 @@ protected void deleteRegion(Configuration conf, final TableDescriptor htd, byte[ if (metaRow) { try ( - Table meta = connection.getTable(MetaTableName.getInstance(), tableExecutorService)) { + Table meta = connection.getTable(connection.getMetaTableName(), tableExecutorService)) { Delete delete = new Delete(deleteRow); meta.delete(delete); } @@ -512,7 +512,7 @@ public boolean tableHasErrors(HbckTableInfo table) { protected void deleteMetaRegion(Configuration conf, boolean unassign, boolean hdfs, boolean regionInfoOnly) throws IOException, InterruptedException { - HRegionLocation metaLocation = connection.getRegionLocator(MetaTableName.getInstance()) + HRegionLocation metaLocation = connection.getRegionLocator(connection.getMetaTableName()) .getRegionLocation(HConstants.EMPTY_START_ROW); ServerName hsa = metaLocation.getServerName(); RegionInfo hri = metaLocation.getRegion(); @@ -527,7 +527,7 @@ protected void deleteMetaRegion(Configuration conf, boolean unassign, boolean hd LOG.info("deleting hdfs .regioninfo data: " + hri.toString() + hsa.toString()); Path rootDir = CommonFSUtils.getRootDir(conf); FileSystem fs = rootDir.getFileSystem(conf); - Path p = new Path(rootDir + "/" + MetaTableName.getInstance().getNameAsString(), + Path p = new Path(rootDir + "/" + connection.getMetaTableName().getNameAsString(), hri.getEncodedName()); Path hriPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE); fs.delete(hriPath, true); @@ -537,7 +537,7 @@ protected void deleteMetaRegion(Configuration conf, boolean unassign, boolean hd LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString()); Path rootDir = CommonFSUtils.getRootDir(conf); FileSystem fs = rootDir.getFileSystem(conf); - Path p = new Path(rootDir + "/" + MetaTableName.getInstance().getNameAsString(), + Path p = new Path(rootDir + "/" + connection.getMetaTableName().getNameAsString(), hri.getEncodedName()); HBaseFsck.debugLsr(conf, p); boolean success = fs.delete(p, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java index 9a25654f87de..d9795110d029 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java @@ -281,7 +281,7 @@ public void testGetAll() throws IOException, InterruptedException { } // add hbase:meta htds.createTableDescriptor( - TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).build()); + TableDescriptorBuilder.newBuilder(connection.getMetaTableName()).build()); assertEquals("getAll() didn't return all TableDescriptors, expected: " + (count + 1) + " got: " + htds.getAll().size(), count + 1, htds.getAll().size()); } @@ -300,7 +300,7 @@ public void testParallelGetAll() throws IOException, InterruptedException { } // add hbase:meta htds.createTableDescriptor( - TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()).build()); + TableDescriptorBuilder.newBuilder(connection.getMetaTableName()).build()); int getTableDescriptorSize = htds.getAll().size(); assertEquals("getAll() didn't return all TableDescriptors, expected: " + (count + 1) + " got: " @@ -327,7 +327,7 @@ public void testGetAllOrdering() throws Exception { // Remove hbase:meta from list. It shows up now since we made it dynamic. The schema // is written into the fs by the FSTableDescriptors constructor now where before it // didn't. - tables.remove(MetaTableName.getInstance().getNameAsString()); + tables.remove(connection.getMetaTableName().getNameAsString()); assertEquals(4, tables.size()); String[] tableNamesOrdered = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java index db5bc77a367c..a6590e982fcc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java @@ -199,7 +199,7 @@ private void addStateAndBarrier(RegionInfo region, RegionState.State state, long put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER, put.getTimestamp() - barriers.length + i, Bytes.toBytes(barriers[i])); } - try (Table table = UTIL.getConnection().getTable(MetaTableName.getInstance())) { + try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { table.put(put); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java index 04f7d74b950b..9f72f45b73e2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java @@ -85,7 +85,7 @@ public static void setUpBeforeClass() throws Exception { admin = connection.getAdmin(); admin.balancerSwitch(false, true); - TEST_UTIL.waitUntilAllRegionsAssigned(MetaTableName.getInstance()); + TEST_UTIL.waitUntilAllRegionsAssigned(connection.getMetaTableName()); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java index 508c2e63ab25..0267fe2051c3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java @@ -288,7 +288,7 @@ public void testRegionServerPort() throws Exception { public void testLoadMetaRegion() throws Exception { HRegionServer rsWithMeta = TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads().stream() .map(t -> t.getRegionServer()) - .filter(rs -> rs.getRegions(MetaTableName.getInstance()).size() > 0).findFirst().get(); + .filter(rs -> rs.getRegions(connection.getMetaTableName()).size() > 0).findFirst().get(); int onlineRegions = rsWithMeta.getNumberOfOnlineRegions(); String rsName = rsWithMeta.getServerName().getAddress().toString(); try (RegionMover rm = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java index 594141f7e802..81c0196dfbf9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java @@ -87,7 +87,7 @@ public void setUp() throws Exception { // Remove rs contains hbase:meta, otherwise test looks unstable and buggy in test env. ServerName rsContainMeta = TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads().stream() .map(t -> t.getRegionServer()) - .filter(rs -> rs.getRegions(MetaTableName.getInstance()).size() > 0).findFirst().get() + .filter(rs -> rs.getRegions(connection.getMetaTableName()).size() > 0).findFirst().get() .getServerName(); LOG.info("{} contains hbase:meta", rsContainMeta); List modifiable = new ArrayList<>(allServers); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java index ef31c35dfca2..baf86fdc39ce 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java @@ -457,7 +457,7 @@ public void testRecoveredEditsPathForMeta() throws IOException { @Test public void testOldRecoveredEditsFileSidelined() throws IOException { Path p = createRecoveredEditsPathForRegion(); - Path tdir = CommonFSUtils.getTableDir(HBASEDIR, MetaTableName.getInstance()); + Path tdir = CommonFSUtils.getTableDir(HBASEDIR, connection.getMetaTableName()); Path regiondir = new Path(tdir, RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName()); fs.mkdirs(regiondir); Path parent = WALSplitUtil.getRegionDirRecoveredEditsDir(regiondir); @@ -470,7 +470,7 @@ public void testOldRecoveredEditsFileSidelined() throws IOException { private Path createRecoveredEditsPathForRegion() throws IOException { byte[] encoded = RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(); - Path p = WALSplitUtil.getRegionSplitEditsPath(MetaTableName.getInstance(), encoded, 1, + Path p = WALSplitUtil.getRegionSplitEditsPath(connection.getMetaTableName(), encoded, 1, FILENAME_BEING_SPLIT, TMPDIRNAME, conf, ""); return p; } diff --git a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 21818f77a1c9..c3a555d1e0fd 100644 --- a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -1107,7 +1107,7 @@ public MiniHBaseCluster startMiniHBaseCluster(StartMiniClusterOption option) // Populate the master address configuration from mini cluster configuration. conf.set(HConstants.MASTER_ADDRS_KEY, MasterRegistry.getMasterAddr(c)); // Don't leave here till we've done a successful scan of the hbase:meta - try (Table t = getConnection().getTable(MetaTableName.getInstance()); + try (Table t = getConnection().getTable(getConnection().getMetaTableName()); ResultScanner s = t.getScanner(new Scan())) { for (;;) { if (s.next() == null) { @@ -1229,7 +1229,7 @@ public void restartHBaseCluster(StartMiniClusterOption option) option.getMasterClass(), option.getRsClass()); // Don't leave here till we've done a successful scan of the hbase:meta Connection conn = ConnectionFactory.createConnection(this.conf); - Table t = conn.getTable(MetaTableName.getInstance()); + Table t = conn.getTable(getConnection().getMetaTableName()); ResultScanner s = t.getScanner(new Scan()); while (s.next() != null) { // do nothing @@ -2397,7 +2397,7 @@ public String checksumRows(final Table table) throws Exception { */ public List createMultiRegionsInMeta(final Configuration conf, final TableDescriptor htd, byte[][] startKeys) throws IOException { - Table meta = getConnection().getTable(MetaTableName.getInstance()); + Table meta = getConnection().getTable(getConnection().getMetaTableName()); Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR); List newRegions = new ArrayList<>(startKeys.length); MetaTableAccessor.updateTableState(getConnection(), htd.getTableName(), @@ -2479,7 +2479,7 @@ public static HRegion createRegionAndWAL(final RegionInfo info, final Path rootD */ public List getMetaTableRows() throws IOException { // TODO: Redo using MetaTableAccessor class - Table t = getConnection().getTable(MetaTableName.getInstance()); + Table t = getConnection().getTable(getConnection().getMetaTableName()); List rows = new ArrayList<>(); ResultScanner s = t.getScanner(new Scan()); for (Result result : s) { @@ -2497,7 +2497,7 @@ public List getMetaTableRows() throws IOException { */ public List getMetaTableRows(TableName tableName) throws IOException { // TODO: Redo using MetaTableAccessor. - Table t = getConnection().getTable(MetaTableName.getInstance()); + Table t = getConnection().getTable(getConnection().getMetaTableName()); List rows = new ArrayList<>(); ResultScanner s = t.getScanner(new Scan()); for (Result result : s) { @@ -2827,7 +2827,7 @@ public void process(WatchedEvent watchedEvent) { monitor.close(); if (checkStatus) { - getConnection().getTable(MetaTableName.getInstance()).close(); + getConnection().getTable(getConnection().getMetaTableName()).close(); } } @@ -3352,7 +3352,7 @@ public void waitUntilAllRegionsAssigned(final TableName tableName) throws IOExce * Waith until all system table's regions get assigned */ public void waitUntilAllSystemRegionsAssigned() throws IOException { - waitUntilAllRegionsAssigned(MetaTableName.getInstance()); + waitUntilAllRegionsAssigned(getConnection().getMetaTableName()); } /** @@ -3365,7 +3365,7 @@ public void waitUntilAllSystemRegionsAssigned() throws IOException { public void waitUntilAllRegionsAssigned(final TableName tableName, final long timeout) throws IOException { if (!TableName.isMetaTableName(tableName)) { - try (final Table meta = getConnection().getTable(MetaTableName.getInstance())) { + try (final Table meta = getConnection().getTable(getConnection().getMetaTableName())) { LOG.debug("Waiting until all regions of table " + tableName + " get assigned. Timeout = " + timeout + "ms"); waitFor(timeout, 200, true, new ExplainingPredicate() { @@ -3583,7 +3583,7 @@ public Table createRandomTable(TableName tableName, final Collection fam Bytes.toBytes(String.format(keyFormat, splitEndKey)), numRegions); if (hbaseCluster != null) { - getMiniHBaseCluster().flushcache(MetaTableName.getInstance()); + getMiniHBaseCluster().flushcache(getConnection().getMetaTableName()); } BufferedMutator mutator = getConnection().getBufferedMutator(tableName); @@ -3798,7 +3798,7 @@ public static int createPreSplitLoadTestTable(Configuration conf, TableDescripto } public static int getMetaRSPort(Connection connection) throws IOException { - try (RegionLocator locator = connection.getRegionLocator(MetaTableName.getInstance())) { + try (RegionLocator locator = connection.getRegionLocator(connection.getMetaTableName())) { return locator.getRegionLocation(Bytes.toBytes("")).getPort(); } } diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java index db1b1e1c9870..01919525104e 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift2/client/ThriftConnection.java @@ -369,6 +369,11 @@ public void clearRegionLocationCache() { throw new NotImplementedException("clearRegionLocationCache not supported in ThriftTable"); } + @Override + public TableName getMetaTableName() { + return toAsyncConnection().getMetaTableName(); + } + @Override public AsyncConnection toAsyncConnection() { throw new NotImplementedException("toAsyncConnection not supported in ThriftTable"); From 256ab2fb268113d2701d8ca29f1eb5816c3569fb Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Tue, 20 Jan 2026 09:59:46 -0500 Subject: [PATCH 09/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: Rename references 2 --- .../backup/impl/IncrementalBackupManager.java | 2 +- .../favored/FavoredNodeAssignmentHelper.java | 2 +- .../SnapshotOfRegionAssignmentFromMeta.java | 2 +- ...rgeClusterBalancingMetaTableIsolation.java | 9 ++- ...gTableIsolationAndReplicaDistribution.java | 8 ++- .../hadoop/hbase/ClientMetaTableAccessor.java | 2 +- .../hadoop/hbase/client/AsyncConnection.java | 9 +++ .../client/AsyncNonMetaRegionLocator.java | 1 - .../hbase/client/AsyncRegionLocator.java | 2 +- .../client/AsyncTableRegionLocatorImpl.java | 1 - .../hadoop/hbase/client/Connection.java | 11 ++++ .../hbase/client/ConnectionFactory.java | 18 +++--- .../hbase/client/MutableRegionInfo.java | 6 +- .../hbase/client/RawAsyncHBaseAdmin.java | 2 +- .../hadoop/hbase/client/RegionInfo.java | 3 +- .../hbase/client/RegionInfoBuilder.java | 6 +- .../hbase/client/RegionInfoDisplay.java | 4 +- .../hbase/client/TableDescriptorBuilder.java | 6 +- .../hbase/security/token/ClientTokenUtil.java | 2 +- .../hbase/shaded/protobuf/ProtobufUtil.java | 4 +- .../client/TestAsyncAdminRpcPriority.java | 4 +- .../client/TestAsyncRegionLocatorTracing.java | 16 ++--- .../client/TestAsyncTableRpcPriority.java | 4 +- .../hbase/client/TestCompactFromClient.java | 2 +- .../hbase/client/TestRegionInfoBuilder.java | 4 +- .../client/TestTableDescriptorBuilder.java | 5 +- .../hadoop/hbase/CellComparatorImpl.java | 12 +++- .../org/apache/hadoop/hbase/HConstants.java | 2 +- .../hbase/InnerStoreCellComparator.java | 12 +++- .../apache/hadoop/hbase/MetaTableName.java | 60 ------------------- .../org/apache/hadoop/hbase/TableName.java | 14 ++++- .../hadoop/hbase/TestCellComparator.java | 24 ++++---- .../org/apache/hadoop/hbase/TestKeyValue.java | 24 ++++---- .../util/ProcessBasedLocalHBaseCluster.java | 2 +- .../hadoop/hbase/util/RestartMetaTest.java | 2 +- .../master/MetricsMasterFileSystemSource.java | 3 +- .../hbase/mttr/IntegrationTestMTTR.java | 2 +- .../hbase/mapreduce/TestImportExport.java | 2 +- .../hadoop/hbase/rest/TestStatusResource.java | 2 +- .../model/TestStorageClusterStatusModel.java | 2 +- .../hadoop/hbase/HBaseRpcServicesBase.java | 2 +- .../hadoop/hbase/MetaTableAccessor.java | 8 +-- .../hbase/coprocessor/MetaTableMetrics.java | 7 +-- .../apache/hadoop/hbase/master/HMaster.java | 18 +++--- ...sterAnnotationReadingPriorityFunction.java | 4 +- .../master/RegionPlacementMaintainer.java | 2 +- .../hbase/master/TableNamespaceManager.java | 10 ++-- .../hbase/master/TableStateManager.java | 2 +- .../master/assignment/AssignmentManager.java | 2 +- .../MergeTableRegionsProcedure.java | 2 +- .../master/assignment/RegionStateStore.java | 21 +++---- .../assignment/SplitTableRegionProcedure.java | 2 +- .../cleaner/ReplicationBarrierCleaner.java | 4 +- .../hadoop/hbase/master/http/MetaBrowser.java | 2 +- .../hbase/master/janitor/CatalogJanitor.java | 2 +- .../hbase/master/janitor/MetaFixer.java | 8 +-- .../master/janitor/ReportMakingVisitor.java | 2 +- .../procedure/DeleteTableProcedure.java | 2 +- .../procedure/DisableTableProcedure.java | 2 +- .../procedure/HBCKServerCrashProcedure.java | 5 +- .../master/procedure/InitMetaProcedure.java | 8 ++- .../procedure/MasterProcedureScheduler.java | 13 ++-- .../hbase/master/procedure/MetaQueue.java | 7 ++- .../MigrateNamespaceTableProcedure.java | 2 +- .../procedure/ModifyTableProcedure.java | 2 +- .../hbase/master/procedure/SchemaLocking.java | 10 +++- .../procedure/TruncateRegionProcedure.java | 4 +- .../hbase/regionserver/RSRpcServices.java | 2 +- .../ReplicationBarrierFamilyFormat.java | 6 +- .../hbase/security/access/AccessChecker.java | 4 +- .../hadoop/hbase/tool/BulkLoadHFilesTool.java | 8 +-- .../hadoop/hbase/util/FSTableDescriptors.java | 18 ++++-- .../apache/hadoop/hbase/util/HBaseFsck.java | 2 +- .../hadoop/hbase/util/HBaseFsckRepair.java | 4 +- .../apache/hadoop/hbase/util/RegionMover.java | 2 +- .../wal/BoundedRecoveredHFilesOutputSink.java | 4 +- .../hbase-webapps/master/catalogTables.jsp | 1 - .../resources/hbase-webapps/master/table.jsp | 1 - .../hadoop/hbase/HBaseClusterInterface.java | 2 +- .../apache/hadoop/hbase/HBaseTestingUtil.java | 12 ++-- .../hadoop/hbase/TestHBaseMetaEdit.java | 18 +++--- .../apache/hadoop/hbase/TestInfoServers.java | 2 +- .../TestMetaUpdatesGoToPriorityQueue.java | 2 +- .../apache/hadoop/hbase/TestNamespace.java | 2 +- .../hbase/TestServerInternalsTracing.java | 2 +- .../client/AbstractTestRegionLocator.java | 20 +++---- .../client/MetaWithReplicasTestBase.java | 4 +- .../hbase/client/RegionReplicaTestHelper.java | 4 +- .../apache/hadoop/hbase/client/TestAdmin.java | 4 +- .../hadoop/hbase/client/TestAdmin2.java | 10 ++-- .../TestAsyncAdminWithRegionReplicas.java | 10 ++-- .../client/TestAsyncMetaRegionLocator.java | 6 +- .../client/TestAsyncNonMetaRegionLocator.java | 6 +- .../client/TestAsyncRegionAdminApi2.java | 16 ++--- .../hbase/client/TestAsyncTableAdminApi.java | 8 +-- .../hbase/client/TestAsyncTableAdminApi2.java | 4 +- .../hbase/client/TestAsyncTableAdminApi3.java | 20 +++---- .../client/TestAsyncTableUseMetaReplicas.java | 6 +- ...talogReplicaLoadBalanceSimpleSelector.java | 18 +++--- .../hbase/client/TestCleanupMetaReplica.java | 4 +- .../client/TestClientScannerTimeouts.java | 4 +- .../client/TestClientSideRegionScanner.java | 10 ++-- .../hadoop/hbase/client/TestEnableTable.java | 4 +- .../hbase/client/TestFromClientSide5.java | 4 +- .../TestIncreaseMetaReplicaThroughConfig.java | 4 +- .../hbase/client/TestMasterRegistry.java | 4 +- .../hadoop/hbase/client/TestMetaCache.java | 4 +- .../client/TestMetaRegionLocationCache.java | 4 +- .../client/TestMetaWithReplicasBasic.java | 4 +- .../TestMetaWithReplicasShutdownHandling.java | 6 +- .../TestMultiActionMetricsFromClient.java | 4 +- .../hbase/client/TestReplicaWithCluster.java | 4 +- .../client/TestRpcConnectionRegistry.java | 4 +- .../client/TestSeparateClientZKCluster.java | 10 ++-- .../TestShutdownOfMetaReplicaHolder.java | 4 +- .../hbase/client/TestSnapshotFromClient.java | 4 +- .../client/TestZKConnectionRegistry.java | 6 +- .../hadoop/hbase/http/TestInfoServersACL.java | 6 +- .../hadoop/hbase/master/TestMaster.java | 4 +- ...MasterFileSystemWithStoreFileTracking.java | 4 +- ...TestMasterOperationsForRegionReplicas.java | 4 +- .../hbase/master/TestMasterRepairMode.java | 6 +- .../hbase/master/TestMasterTransitions.java | 4 +- .../TestMetaAssignmentWithStopMaster.java | 4 +- .../TestMigrateAndMirrorMetaLocations.java | 9 ++- .../master/TestMigrateNamespaceTable.java | 6 +- .../hbase/master/TestRecreateCluster.java | 6 +- .../TestRestartWithEmptyWALDirectory.java | 4 +- ...ServerCrashProcedureCarryingMetaStuck.java | 6 +- .../TestCloseRegionWhileRSCrash.java | 8 +-- .../TestOpenRegionProcedureBackoff.java | 4 +- .../assignment/TestRegionStateStore.java | 4 +- .../master/assignment/TestRollbackSCP.java | 4 +- .../BalancerConditionalsTestUtil.java | 4 +- .../TestFavoredStochasticLoadBalancer.java | 4 +- ...MetaTableIsolationBalancerConditional.java | 6 +- .../TestReplicationBarrierCleaner.java | 14 ++--- .../TestCatalogJanitorInMemoryStates.java | 4 +- .../janitor/TestMetaFixerNoCluster.java | 18 +++--- .../TestSimpleRegionNormalizer.java | 3 +- .../hbase/master/procedure/TestHBCKSCP.java | 6 +- .../procedure/TestProcedurePriority.java | 10 +++- ...TestTableProcedureWaitingQueueCleanup.java | 4 +- .../TestTruncateTableWithMasterFailover.java | 4 +- .../TestCompactionInDeadRegionServer.java | 4 +- .../regionserver/TestDefaultMemStore.java | 4 +- .../TestEndToEndSplitTransaction.java | 4 +- .../TestGetClosestAtOrBefore.java | 4 +- .../TestReadAndWriteRegionInfoFile.java | 6 +- .../hbase/regionserver/TestRegionInfo.java | 36 +++++------ .../regionserver/TestRegionReplicas.java | 4 +- .../TestRegionServerCrashDisableWAL.java | 4 +- .../TestRegionServerNoMaster.java | 6 +- .../TestRegionServerRejectDuringAbort.java | 4 +- .../TestShutdownWhileWALBroken.java | 4 +- .../wal/AbstractTestLogRolling.java | 4 +- .../regionserver/wal/TestLogRollAbort.java | 4 +- .../regionserver/wal/TestLogRolling.java | 4 +- .../wal/TestLogRollingNoCluster.java | 8 +-- .../TestAddToSerialReplicationPeer.java | 24 ++++---- .../TestRemoveFromSerialReplicationPeer.java | 4 +- .../TestReplicationWALEntryFilters.java | 4 +- .../replication/TestSerialReplication.java | 16 ++--- .../TestSerialReplicationFailover.java | 4 +- .../TestMetaRegionReplicaReplication.java | 28 ++++----- .../regionserver/TestReplicationSource.java | 4 +- .../TestSerialReplicationChecker.java | 8 +-- .../rsgroup/TestRSGroupsCPHookCalled.java | 3 +- .../hbase/rsgroup/TestRSGroupsKillRS.java | 6 +- .../security/access/TestRpcAccessChecks.java | 6 +- .../token/TestGenerateDelegationToken.java | 2 +- .../snapshot/TestRegionSnapshotTask.java | 4 +- .../hadoop/hbase/util/BaseTestHBaseFsck.java | 10 ++-- .../hbase/util/TestFSTableDescriptors.java | 8 +-- ...TestHBaseFsckCleanReplicationBarriers.java | 4 +- .../hadoop/hbase/util/TestHBaseFsckMOB.java | 1 - .../hadoop/hbase/util/TestRegionMover1.java | 4 +- .../TestRegionMoverWithRSGroupEnable.java | 4 +- .../apache/hadoop/hbase/wal/TestWALSplit.java | 6 +- .../thrift/ThriftHBaseServiceHandler.java | 8 ++- .../hbase/zookeeper/MetaTableLocator.java | 15 ++--- .../apache/hadoop/hbase/zookeeper/ZKDump.java | 5 +- 182 files changed, 607 insertions(+), 598 deletions(-) delete mode 100644 hbase-common/src/main/java/org/apache/hadoop/hbase/MetaTableName.java diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java index f6454a737a3b..25805c4635c9 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Connection; diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java index 88d5ac88caed..393eac965f34 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.CellBuilderType; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java index 0a40e9dcdf8b..8cf62ac41398 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.CatalogFamilyFormat; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java index b8e144b4ac85..b12df5a1fb54 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java @@ -27,7 +27,7 @@ import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -71,7 +71,9 @@ public static void setup() { // Create regions List allRegions = new ArrayList<>(); for (int i = 0; i < NUM_REGIONS; i++) { - TableName tableName = i < 3 ? connection.getMetaTableName() : NON_META_TABLE_NAME; + // TODO(HBASE-XXXXX - Phase 6): Make configurable for replica testing + TableName tableName = i < 3 ? + TableName.valueOf("hbase", "meta") : NON_META_TABLE_NAME; byte[] startKey = new byte[1]; startKey[0] = (byte) i; byte[] endKey = new byte[1]; @@ -99,7 +101,8 @@ public void testMetaTableIsolation() { } private boolean isMetaTableIsolated(BalancerClusterState cluster) { - return isTableIsolated(cluster, connection.getMetaTableName(), "Meta"); + // TODO(HBASE-XXXXX - Phase 6): Make configurable for replica testing + return isTableIsolated(cluster, TableName.valueOf("hbase", "meta"), "Meta"); } } diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java index 94f38e0d8cca..61d2ac2efabe 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java @@ -28,7 +28,7 @@ import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -75,7 +75,8 @@ public static void setup() { for (int i = 0; i < NUM_REGIONS; i++) { TableName tableName; if (i < 1) { - tableName = connection.getMetaTableName(); + // TODO(HBASE-XXXXX - Phase 6): Make configurable for replica testing + tableName = TableName.valueOf("hbase", "meta"); } else if (i < 10) { tableName = SYSTEM_TABLE_NAME; } else { @@ -120,7 +121,8 @@ public void testTableIsolationAndReplicaDistribution() { * Validates whether all meta table regions are isolated. */ private boolean isMetaTableIsolated(BalancerClusterState cluster) { - return isTableIsolated(cluster, connection.getMetaTableName(), "Meta"); + // TODO(HBASE-XXXXX - Phase 6): Make configurable for replica testing + return isTableIsolated(cluster, TableName.valueOf("hbase", "meta"), "Meta"); } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClientMetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClientMetaTableAccessor.java index bf8513ee959d..48a75871bbb2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClientMetaTableAccessor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClientMetaTableAccessor.java @@ -196,7 +196,7 @@ private static CompletableFuture>> getTableReg final AsyncTable metaTable, final TableName tableName, final boolean excludeOfflinedSplitParents) { CompletableFuture>> future = new CompletableFuture<>(); - if (MetaTableName.getInstance().equals(tableName)) { + if (TableName.isMetaTableName(tableName)) { future.completeExceptionally(new IOException( "This method can't be used to locate meta regions;" + " use MetaTableLocator instead")); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java index d858ffdc5aa6..79087d5f5294 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java @@ -117,6 +117,15 @@ default AsyncTable getTable(TableName tableName, ExecutorSer return getTableBuilder(tableName, pool).build(); } + /** + * Retrieve an AsyncTable implementation for accessing the meta table. + * This method returns the correct meta table for this connection (hbase:meta or hbase:meta_suffix). + * @return An AsyncTable to use for interactions with the meta table + */ + default AsyncTable getMetaTable() { + return getTable(getMetaTableName()); + } + /** * Returns an {@link AsyncTableBuilder} for creating {@link AsyncTable}. *

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java index 9877810be5b9..016e5bbac799 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java @@ -51,7 +51,6 @@ import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java index 3681e5b38524..7822672f3b37 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java @@ -35,7 +35,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java index c0d297460330..ab358e1af184 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java @@ -25,7 +25,6 @@ import java.util.concurrent.CompletableFuture; import org.apache.hadoop.hbase.ClientMetaTableAccessor; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.yetus.audience.InterfaceAudience; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java index 1737a787c8bc..107d488f4c4c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java @@ -108,6 +108,17 @@ default Table getTable(TableName tableName, ExecutorService pool) throws IOExcep return getTableBuilder(tableName, pool).build(); } + /** + * Retrieve a Table implementation for accessing the meta table. + * This method returns the correct meta table for this connection (hbase:meta or hbase:meta_suffix). + * The returned Table is not thread safe, a new instance should be created for each using thread. + * This is a lightweight operation, pooling or caching of the returned Table is neither required nor desired. + * @return A Table to use for interactions with the meta table + */ + default Table getMetaTable() throws IOException { + return getTable(getMetaTableName()); + } + /** *

* Retrieve a {@link BufferedMutator} for performing client-side buffering of writes. The diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java index 8e304fcbcac9..34ebed08dee4 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java @@ -607,16 +607,16 @@ public static CompletableFuture createAsyncConnection(URI conne future.completeExceptionally(new IOException("meta table name came back null")); return; } - Class clazz = appliedConf.getClass( - HBASE_CLIENT_ASYNC_CONNECTION_IMPL, AsyncConnectionImpl.class, AsyncConnection.class); - try { - future.complete(user.runAs((PrivilegedExceptionAction< - ? extends AsyncConnection>) () -> ReflectionUtils.newInstance(clazz, appliedConf, + Class clazz = appliedConf.getClass( + HBASE_CLIENT_ASYNC_CONNECTION_IMPL, AsyncConnectionImpl.class, AsyncConnection.class); + try { + future.complete(user.runAs((PrivilegedExceptionAction< + ? extends AsyncConnection>) () -> ReflectionUtils.newInstance(clazz, appliedConf, registry, clusterId, metaTableName, null, user, connectionAttributes))); - } catch (Exception e) { - registry.close(); - future.completeExceptionally(e); - } + } catch (Exception e) { + registry.close(); + future.completeExceptionally(e); + } }); }); return future; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java index 5e629839782f..909d10c6f0a4 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java @@ -21,7 +21,7 @@ import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; @@ -121,7 +121,7 @@ private static int checkReplicaId(int regionId) { this.replicaId = checkReplicaId(replicaId); this.offLine = offLine; this.regionName = RegionInfo.createRegionName(this.tableName, this.startKey, this.regionId, - this.replicaId, !this.tableName.equals(MetaTableName.getInstance())); + this.replicaId, !TableName.isMetaTableName(this.tableName)); this.encodedName = RegionInfo.encodeRegionName(this.regionName); this.hashCode = generateHashCode(this.tableName, this.startKey, this.endKey, this.regionId, this.replicaId, this.offLine, this.regionName); @@ -233,7 +233,7 @@ public boolean containsRow(byte[] row) { /** Returns true if this region is a meta region */ @Override public boolean isMetaRegion() { - return tableName.equals(MetaTableName.getInstance()); + return TableName.isMetaTableName(tableName); } /** Returns True if has been split and has daughters. */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index b09d6360a322..bc51758dd291 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.RegionMetrics; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java index bc3b48a54a28..10c554e26f79 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java @@ -26,7 +26,6 @@ import java.util.List; import java.util.stream.Collectors; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.util.ByteArrayHashKey; @@ -432,7 +431,7 @@ static byte[] toByteArray(RegionInfo ri) { */ static String prettyPrint(final String encodedRegionName) { if (encodedRegionName.equals("1028785192")) { - return encodedRegionName + "/" + MetaTableName.getInstance(); + return encodedRegionName + "/hbase:meta"; } return encodedRegionName; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java index 1c2aab455b55..30b564cff2af 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.client; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.yetus.audience.InterfaceAudience; @@ -43,8 +43,10 @@ public class RegionInfoBuilder { */ // TODO: How come Meta regions still do not have encoded region names? Fix. // hbase:meta,,1.1588230740 should be the hbase:meta first region name. + // TODO(HBASE-XXXXX - Phase 6): Make this configurable for read replica clusters + // For now, hardcode to default. Future: lazy initialization based on config public static final RegionInfo FIRST_META_REGIONINFO = - new MutableRegionInfo(1L, MetaTableName.getInstance(), RegionInfo.DEFAULT_REPLICA_ID); + new MutableRegionInfo(1L, TableName.valueOf("hbase", "meta"), RegionInfo.DEFAULT_REPLICA_ID); private final TableName tableName; private byte[] startKey = HConstants.EMPTY_START_ROW; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoDisplay.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoDisplay.java index b8288e709cb7..68906d8d6382 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoDisplay.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoDisplay.java @@ -20,7 +20,7 @@ import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; @@ -82,7 +82,7 @@ public static String getRegionNameAsStringForDisplay(RegionInfo ri, Configuratio */ public static byte[] getRegionNameForDisplay(RegionInfo ri, Configuration conf) { boolean displayKey = conf.getBoolean(DISPLAY_KEYS_KEY, true); - if (displayKey || ri.getTable().equals(MetaTableName.getInstance())) { + if (displayKey || TableName.isMetaTableName(ri.getTable())) { return ri.getRegionName(); } else { // create a modified regionname with the startkey replaced but preserving diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java index a1b766696250..a9d47460b776 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java @@ -38,7 +38,7 @@ import java.util.stream.Collectors; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.HBaseException; @@ -616,8 +616,8 @@ private ModifyableTableDescriptor(final TableName name, this.name = name; families.forEach(c -> this.families.put(c.getName(), ColumnFamilyDescriptorBuilder.copy(c))); this.values.putAll(values); - this.values.put(IS_META_KEY, - new Bytes(Bytes.toBytes(Boolean.toString(name.equals(MetaTableName.getInstance()))))); + this.values.put(IS_META_KEY, + new Bytes(Bytes.toBytes(Boolean.toString(TableName.isMetaTableName(name))))); } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java index 6c7fcb1b8feb..1a01e1506fe6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java @@ -22,7 +22,7 @@ import java.security.PrivilegedExceptionAction; import java.util.concurrent.CompletableFuture; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.client.AsyncConnection; import org.apache.hadoop.hbase.client.AsyncTable; import org.apache.hadoop.hbase.client.Connection; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java index 7cb4ffe285eb..823e72959645 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java @@ -65,7 +65,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerTask; @@ -3326,7 +3326,7 @@ public static String toLockJson(List lockedRes long regionId = proto.getRegionId(); int defaultReplicaId = org.apache.hadoop.hbase.client.RegionInfo.DEFAULT_REPLICA_ID; int replicaId = proto.hasReplicaId() ? proto.getReplicaId() : defaultReplicaId; - if (tableName.equals(MetaTableName.getInstance()) && replicaId == defaultReplicaId) { + if (TableName.isMetaTableName(tableName) && replicaId == defaultReplicaId) { return RegionInfoBuilder.FIRST_META_REGIONINFO; } byte[] startKey = null; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java index c9eb597da204..ea17a187ad5d 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java @@ -33,7 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.HBaseRpcController; @@ -196,7 +196,7 @@ public void testCreateSystemTable() { // that we pass the correct priority @Test public void testCreateMetaTable() { - conn.getAdmin().createTable(TableDescriptorBuilder.newBuilder(connection.getMetaTableName()) + conn.getAdmin().createTable(TableDescriptorBuilder.newBuilder(conn.getMetaTableName()) .setColumnFamily(ColumnFamilyDescriptorBuilder.of("cf")).build()).join(); verify(masterStub, times(1)).createTable(assertPriority(SYSTEMTABLE_QOS), any(CreateTableRequest.class), any()); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java index 6215fea6853e..53ae717275d9 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java @@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MatcherPredicate; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.Waiter; @@ -85,7 +85,7 @@ public class TestAsyncRegionLocatorTracing { @Before public void setUp() throws IOException { - RegionInfo metaRegionInfo = RegionInfoBuilder.newBuilder(connection.getMetaTableName()).build(); + RegionInfo metaRegionInfo = RegionInfoBuilder.newBuilder(conn.getMetaTableName()).build(); locs = new RegionLocations( new HRegionLocation(metaRegionInfo, ServerName.valueOf("127.0.0.1", 12345, EnvironmentEdgeManager.currentTime())), @@ -147,30 +147,30 @@ public void testClearCacheServerName() { @Test public void testClearCacheTableName() { - conn.getLocator().clearCache(connection.getMetaTableName()); + conn.getLocator().clearCache(conn.getMetaTableName()); SpanData span = waitSpan("AsyncRegionLocator.clearCache"); assertThat(span, allOf(hasStatusWithCode(StatusCode.OK), hasKind(SpanKind.INTERNAL), buildConnectionAttributesMatcher(conn), - buildTableAttributesMatcher(connection.getMetaTableName()))); + buildTableAttributesMatcher(conn.getMetaTableName()))); } @Test public void testGetRegionLocation() { - conn.getLocator().getRegionLocation(connection.getMetaTableName(), HConstants.EMPTY_START_ROW, + conn.getLocator().getRegionLocation(conn.getMetaTableName(), HConstants.EMPTY_START_ROW, RegionLocateType.CURRENT, TimeUnit.SECONDS.toNanos(1)).join(); SpanData span = waitSpan("AsyncRegionLocator.getRegionLocation"); assertThat(span, allOf(hasStatusWithCode(StatusCode.OK), hasKind(SpanKind.INTERNAL), buildConnectionAttributesMatcher(conn), - buildTableAttributesMatcher(connection.getMetaTableName()), + buildTableAttributesMatcher(conn.getMetaTableName()), hasAttributes(containsEntryWithStringValuesOf("db.hbase.regions", locs.getDefaultRegionLocation().getRegion().getRegionNameAsString())))); } @Test public void testGetRegionLocations() { - conn.getLocator().getRegionLocations(connection.getMetaTableName(), HConstants.EMPTY_START_ROW, + conn.getLocator().getRegionLocations(conn.getMetaTableName(), HConstants.EMPTY_START_ROW, RegionLocateType.CURRENT, false, TimeUnit.SECONDS.toNanos(1)).join(); SpanData span = waitSpan("AsyncRegionLocator.getRegionLocations"); String[] expectedRegions = @@ -178,7 +178,7 @@ public void testGetRegionLocations() { .map(RegionInfo::getRegionNameAsString).toArray(String[]::new); assertThat(span, allOf(hasStatusWithCode(StatusCode.OK), hasKind(SpanKind.INTERNAL), buildConnectionAttributesMatcher(conn), - buildTableAttributesMatcher(connection.getMetaTableName()), hasAttributes( + buildTableAttributesMatcher(conn.getMetaTableName()), hasAttributes( containsEntryWithStringValuesOf("db.hbase.regions", containsInAnyOrder(expectedRegions))))); } } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java index f3cfe1c2e7f7..13ad29312f2c 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java @@ -50,7 +50,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.HBaseRpcController; @@ -556,7 +556,7 @@ public void testScanSystemTable() throws Exception { @Test public void testScanMetaTable() throws Exception { CompletableFuture renewFuture = mockScanReturnRenewFuture(SYSTEMTABLE_QOS); - testForTable(connection.getMetaTableName(), renewFuture, Optional.empty()); + testForTable(conn.getMetaTableName(), renewFuture, Optional.empty()); } private void testForTable(TableName tableName, CompletableFuture renewFuture, diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java index 3d4417947c52..5461509379ec 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java @@ -33,7 +33,7 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.hbase.ClientMetaTableAccessor; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.testclassification.ClientTests; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java index 8ed7b2739120..462559c8b1d4 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java @@ -26,8 +26,8 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseCommonTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.exceptions.DeserializationException; @@ -142,7 +142,7 @@ public void testContainsRange() { @Test public void testContainsRangeForMetaTable() { TableDescriptor tableDesc = - TableDescriptorBuilder.newBuilder(connection.getMetaTableName()).build(); + TableDescriptorBuilder.newBuilder(TableName.valueOf("hbase:meta")).build(); RegionInfo hri = RegionInfoBuilder.newBuilder(tableDesc.getTableName()).build(); byte[] startRow = HConstants.EMPTY_START_ROW; byte[] row1 = Bytes.toBytes("a,a,0"); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java index b94b73aedf59..b93ee2174c36 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java @@ -26,7 +26,6 @@ import java.io.IOException; import java.util.regex.Pattern; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.HBaseException; @@ -60,7 +59,7 @@ public class TestTableDescriptorBuilder { @Test(expected = IOException.class) public void testAddCoprocessorTwice() throws IOException { String cpName = "a.b.c.d"; - TableDescriptorBuilder.newBuilder(connection.getMetaTableName()).setCoprocessor(cpName) + TableDescriptorBuilder.newBuilder(TableName.valueOf("hbase:meta")).setCoprocessor(cpName) .setCoprocessor(cpName).build(); } @@ -68,7 +67,7 @@ public void testAddCoprocessorTwice() throws IOException { public void testPb() throws DeserializationException, IOException { final int v = 123; TableDescriptor htd = - TableDescriptorBuilder.newBuilder(connection.getMetaTableName()).setMaxFileSize(v) + TableDescriptorBuilder.newBuilder(TableName.valueOf("hbase:meta")).setMaxFileSize(v) .setDurability(Durability.ASYNC_WAL).setReadOnly(true).setRegionReplication(2).build(); byte[] bytes = TableDescriptorBuilder.toByteArray(htd); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java index 49eb3b9cce62..6a85f13406e9 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java @@ -787,8 +787,18 @@ public static CellComparator getCellComparator(TableName tableName) { */ public static CellComparator getCellComparator(byte[] tableName) { // FYI, TableName.toBytes does not create an array; just returns existing array pointer. - return Bytes.equals(tableName, MetaTableName.getInstance().toBytes()) + // Check if this is a meta table (hbase:meta or hbase:meta_*) + return isMetaTable(tableName) ? MetaCellComparator.META_COMPARATOR : CellComparatorImpl.COMPARATOR; } + + private static boolean isMetaTable(byte[] tableName) { + // Quick check: meta tables are in hbase namespace and start with "meta" + return tableName != null && tableName.length > 11 && + tableName[0] == 'h' && tableName[1] == 'b' && tableName[2] == 'a' && + tableName[3] == 's' && tableName[4] == 'e' && tableName[5] == ':' && + tableName[6] == 'm' && tableName[7] == 'e' && tableName[8] == 't' && + tableName[9] == 'a'; + } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index ded87f4a00b0..9ac69aaefb46 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1209,7 +1209,7 @@ public enum OperationStatusCode { @Deprecated public static final List HBASE_NON_USER_TABLE_DIRS = Collections.unmodifiableList(Arrays.asList( - (String[]) ArrayUtils.addAll(new String[] { MetaTableName.getInstance().getNameAsString() }, + (String[]) ArrayUtils.addAll(new String[] { "hbase:meta" }, HBASE_NON_TABLE_DIRS.toArray()))); /** Health script related settings. */ diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java index e2fd632be19f..29fdc51adffc 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java @@ -75,8 +75,18 @@ public static CellComparator getInnerStoreCellComparator(TableName tableName) { * @return CellComparator to use going off the {@code tableName} passed. */ public static CellComparator getInnerStoreCellComparator(byte[] tableName) { - return Bytes.equals(tableName, MetaTableName.getInstance().toBytes()) + // Check if this is a meta table (hbase:meta or hbase:meta_*) + return isMetaTable(tableName) ? MetaCellComparator.META_COMPARATOR : InnerStoreCellComparator.INNER_STORE_COMPARATOR; } + + private static boolean isMetaTable(byte[] tableName) { + // Quick check: meta tables are in hbase namespace and start with "meta" + return tableName != null && tableName.length > 11 && + tableName[0] == 'h' && tableName[1] == 'b' && tableName[2] == 'a' && + tableName[3] == 's' && tableName[4] == 'e' && tableName[5] == ':' && + tableName[6] == 'm' && tableName[7] == 'e' && tableName[8] == 't' && + tableName[9] == 'a'; + } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaTableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaTableName.java deleted file mode 100644 index bbacedc28390..000000000000 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaTableName.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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 - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hadoop.hbase; - -import org.apache.hadoop.conf.Configuration; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@InterfaceAudience.Public -public class MetaTableName { - private static final Logger LOG = LoggerFactory.getLogger(MetaTableName.class); - private static volatile TableName instance; - - private MetaTableName() { - } - - /** - * Get the singleton instance of the meta table name. - * @return The meta table name instance - */ - public static TableName getInstance() { - if (instance == null) { - synchronized (MetaTableName.class) { - if (instance == null) { - instance = initializeHbaseMetaTableName(HBaseConfiguration.create()); - LOG.info("Meta table name initialized: {}", instance.getName()); - } - } - } - return instance; - } - - /** - * Initialize the meta table name from the given configuration. - * @param conf The configuration to use - * @return The initialized meta table name - */ - private static TableName initializeHbaseMetaTableName(Configuration conf) { - TableName metaTableName = - TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); - LOG.info("Meta table suffix value: {}", metaTableName); - return metaTableName; - } -} diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java index 442dc10d8721..658263ac747b 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java @@ -92,9 +92,17 @@ public final class TableName implements Comparable { /** One globally disallowed name */ public static final String DISALLOWED_TABLE_NAME = "zookeeper"; - /** Returns True if tn is the hbase:meta table name. */ + /** + * Returns True if tn is a meta table (hbase:meta or hbase:meta_suffix). + * This handles both the default meta table and read replica meta tables. + */ public static boolean isMetaTableName(final TableName tn) { - return tn.equals(MetaTableName.getInstance()); + if (tn == null) return false; + if (!tn.getNamespaceAsString().equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) { + return false; + } + String qualifier = tn.getQualifierAsString(); + return qualifier.equals("meta") || qualifier.startsWith("meta_"); } /** @@ -296,7 +304,7 @@ private TableName(ByteBuffer namespace, ByteBuffer qualifier) throws IllegalArgu } if (qualifierAsString.equals(OLD_META_STR)) { throw new IllegalArgumentException(OLD_META_STR + " no longer exists. The table has been " - + "renamed to " + MetaTableName.getInstance()); + + "renamed to hbase:meta"); } if (Bytes.equals(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME, namespace)) { diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java index 88b17cc051a5..254dce9665b7 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java @@ -197,36 +197,36 @@ public void testMetaComparisons2() { CellComparator c = MetaCellComparator.META_COMPARATOR; assertTrue(c.compare( createByteBufferKeyValueFromKeyValue(new KeyValue( - Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,1"), now)), + Bytes.toBytes("hbase:meta" + ",a,,0,1"), now)), createByteBufferKeyValueFromKeyValue(new KeyValue( - Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,1"), now))) == 0); + Bytes.toBytes("hbase:meta" + ",a,,0,1"), now))) == 0); Cell a = createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,1"), now)); + new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now)); Cell b = createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,2"), now)); + new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,2"), now)); assertTrue(c.compare(a, b) < 0); assertTrue(c.compare( createByteBufferKeyValueFromKeyValue(new KeyValue( - Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,2"), now)), + Bytes.toBytes("hbase:meta" + ",a,,0,2"), now)), createByteBufferKeyValueFromKeyValue(new KeyValue( - Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,1"), now))) > 0); + Bytes.toBytes("hbase:meta" + ",a,,0,1"), now))) > 0); assertTrue(c.compare( createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,1"), now)), + new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now)), createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,1"), now))) + new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now))) == 0); assertTrue(c.compare( createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,1"), now)), + new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now)), createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,2"), now))) + new KeyValue(Bytes.toBytes("hbase:meta" + ",,2"), now))) < 0); assertTrue(c.compare( createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,2"), now)), + new KeyValue(Bytes.toBytes("hbase:meta" + ",,2"), now)), createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,1"), now))) + new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now))) > 0); } diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java index 24e751215149..a55bcae490af 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java @@ -198,32 +198,32 @@ public void testKeyValueBorderCases() { private void metacomparisons(final CellComparatorImpl c) { long now = EnvironmentEdgeManager.currentTime(); assertTrue(c.compare( - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,1"), now), - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,1"), now)) + new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now), + new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now)) == 0); KeyValue a = - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,1"), now); + new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now); KeyValue b = - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,2"), now); + new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,2"), now); assertTrue(c.compare(a, b) < 0); assertTrue(c.compare( - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,2"), now), - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",a,,0,1"), now)) + new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,2"), now), + new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now)) > 0); } private void comparisons(final CellComparatorImpl c) { long now = EnvironmentEdgeManager.currentTime(); assertTrue(c.compare( - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,1"), now), - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,1"), now)) + new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now), + new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now)) == 0); assertTrue(c.compare( - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,1"), now), - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,2"), now)) < 0); + new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now), + new KeyValue(Bytes.toBytes("hbase:meta" + ",,2"), now)) < 0); assertTrue(c.compare( - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,2"), now), - new KeyValue(Bytes.toBytes(connection.getMetaTableName().getNameAsString() + ",,1"), now)) > 0); + new KeyValue(Bytes.toBytes("hbase:meta" + ",,2"), now), + new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now)) > 0); } @Test diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java index ea0d3cf16236..c79de017f729 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java @@ -42,7 +42,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hdfs.MiniDFSCluster; diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java index 4b56ed4c00c5..a6f10a477ce0 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java @@ -20,7 +20,7 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java index 7aea9b356259..53ed8a25ed0e 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/master/MetricsMasterFileSystemSource.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.master; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.metrics.BaseSource; import org.apache.yetus.audience.InterfaceAudience; @@ -50,7 +49,7 @@ public interface MetricsMasterFileSystemSource extends BaseSource { String SPLIT_SIZE_NAME = "hlogSplitSize"; String META_SPLIT_TIME_DESC = "Time it takes to finish splitMetaLog()"; - String META_SPLIT_SIZE_DESC = "Size of " + MetaTableName.getInstance() + " WAL files being split"; + String META_SPLIT_SIZE_DESC = "Size of hbase:meta WAL files being split"; String SPLIT_TIME_DESC = "Time it takes to finish WAL.splitLog()"; String SPLIT_SIZE_DESC = "Size of WAL files being split"; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java index 3fe10281264d..7c6c9f71c024 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.InvalidFamilyOperationException; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.NamespaceExistException; import org.apache.hadoop.hbase.NamespaceNotFoundException; import org.apache.hadoop.hbase.TableExistsException; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java index 0e5d7ec4c572..fb24172770e6 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java @@ -50,7 +50,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java index c429bdb48f99..b3c77af967d4 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java @@ -29,7 +29,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.rest.client.Client; diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java index 8be9a2b18f1a..99069113c9a7 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java @@ -23,7 +23,7 @@ import java.util.Iterator; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java index 6a6167272359..e2eb25d0f468 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java @@ -396,7 +396,7 @@ public final GetMetaTableNameResponse getMetaTableName(RpcController controller, } catch (Exception e) { throw new ServiceException(e); } - + return builder.build(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java index 24bbfb5079f7..ee5104d50201 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java @@ -151,7 +151,7 @@ public static Table getMetaHTable(final Connection connection) throws IOExceptio if (connection.isClosed()) { throw new IOException("connection is closed"); } - return connection.getTable(MetaTableName.getInstance()); + return connection.getTable(connection.getMetaTableName()); } /** @@ -366,7 +366,7 @@ private static Scan getMetaScan(Configuration conf, int rowUpperLimit) { public static List> getTableRegionsAndLocations( Connection connection, @Nullable final TableName tableName, final boolean excludeOfflinedSplitParents) throws IOException { - if (tableName != null && tableName.equals(MetaTableName.getInstance())) { + if (tableName != null && tableName.equals(connection.getMetaTableName())) { throw new IOException( "This method can't be used to locate meta regions; use MetaTableLocator instead"); } @@ -592,7 +592,7 @@ public static PairOfSameType getDaughterRegions(Result data) { */ @Nullable public static TableState getTableState(Connection conn, TableName tableName) throws IOException { - if (tableName.equals(MetaTableName.getInstance())) { + if (TableName.isMetaTableName(tableName)) { return new TableState(tableName, TableState.State.ENABLED); } Table metaHTable = getMetaHTable(conn); @@ -859,7 +859,7 @@ public static void addRegionsToMeta(Connection connection, List regi private static void updateTableState(Connection connection, TableState state) throws IOException { Put put = makePutFromTableState(state, EnvironmentEdgeManager.currentTime()); putToMetaTable(connection, put); - LOG.info("Updated {} in {}", state, MetaTableName.getInstance()); + LOG.info("Updated {} in {}", state, connection.getMetaTableName()); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java index aa934cc6d462..da93a29423b4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java @@ -26,7 +26,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CoprocessorEnvironment; -import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Get; @@ -133,7 +133,7 @@ private String getRegionIdFromOp(Row op) { } private boolean isMetaTableOp(ObserverContext e) { - return MetaTableName.getInstance().equals(e.getEnvironment().getRegionInfo().getTable()); + return TableName.isMetaTableName(e.getEnvironment().getRegionInfo().getTable()); } private void clientMetricRegisterAndMark() { @@ -267,8 +267,7 @@ public void start(CoprocessorEnvironment env) throws IOException { if ( env instanceof RegionCoprocessorEnvironment && ((RegionCoprocessorEnvironment) env).getRegionInfo().getTable() != null - && ((RegionCoprocessorEnvironment) env).getRegionInfo().getTable() - .equals(MetaTableName.getInstance()) + && TableName.isMetaTableName(((RegionCoprocessorEnvironment) env).getRegionInfo().getTable()) ) { RegionCoprocessorEnvironment regionCoprocessorEnv = (RegionCoprocessorEnvironment) env; registry = regionCoprocessorEnv.getMetricRegistryForRegionServer(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 601da7239f3e..3238a3d9cc8d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -83,7 +83,7 @@ import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.PleaseHoldException; import org.apache.hadoop.hbase.PleaseRestartMasterException; @@ -1094,7 +1094,7 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE InitMetaProcedure initMetaProc = null; // Print out state of hbase:meta on startup; helps debugging. if ( - !this.assignmentManager.getRegionStates().hasTableRegionStates(MetaTableName.getInstance()) + !this.assignmentManager.getRegionStates().hasTableRegionStates(getConnection().getMetaTableName()) ) { Optional optProc = procedureExecutor.getProcedures().stream() .filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny(); @@ -1159,7 +1159,7 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE return; } - TableDescriptor metaDescriptor = tableDescriptors.get(MetaTableName.getInstance()); + TableDescriptor metaDescriptor = tableDescriptors.get(getConnection().getMetaTableName()); final ColumnFamilyDescriptor tableFamilyDesc = metaDescriptor.getColumnFamily(HConstants.TABLE_FAMILY); final ColumnFamilyDescriptor replBarrierFamilyDesc = @@ -1177,17 +1177,17 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE if (conf.get(HConstants.META_REPLICAS_NUM) != null) { int replicasNumInConf = conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM); - TableDescriptor metaDesc = tableDescriptors.get(MetaTableName.getInstance()); + TableDescriptor metaDesc = tableDescriptors.get(getConnection().getMetaTableName()); if (metaDesc.getRegionReplication() != replicasNumInConf) { // it is possible that we already have some replicas before upgrading, so we must set the // region replication number in meta TableDescriptor directly first, without creating a // ModifyTableProcedure, otherwise it may cause a double assign for the meta replicas. int existingReplicasCount = - assignmentManager.getRegionStates().getRegionsOfTable(MetaTableName.getInstance()).size(); + assignmentManager.getRegionStates().getRegionsOfTable(getConnection().getMetaTableName()).size(); if (existingReplicasCount > metaDesc.getRegionReplication()) { LOG.info( "Update replica count of {} from {}(in TableDescriptor)" + " to {}(existing ZNodes)", - MetaTableName.getInstance(), metaDesc.getRegionReplication(), existingReplicasCount); + getConnection().getMetaTableName(), metaDesc.getRegionReplication(), existingReplicasCount); metaDesc = TableDescriptorBuilder.newBuilder(metaDesc) .setRegionReplication(existingReplicasCount).build(); tableDescriptors.update(metaDesc); @@ -1198,7 +1198,7 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE "The {} config is {} while the replica count in TableDescriptor is {}" + " for hbase:meta, altering...", HConstants.META_REPLICAS_NUM, replicasNumInConf, metaDesc.getRegionReplication(), - MetaTableName.getInstance()); + getConnection().getMetaTableName()); procedureExecutor.submitProcedure(new ModifyTableProcedure( procedureExecutor.getEnvironment(), TableDescriptorBuilder.newBuilder(metaDesc) .setRegionReplication(replicasNumInConf).build(), @@ -1428,7 +1428,7 @@ private void createMissingCFsInMetaDuringUpgrade(TableDescriptor metaDescriptor) TableDescriptor newMetaDesc = TableDescriptorBuilder.newBuilder(metaDescriptor) .setColumnFamily(FSTableDescriptors.getTableFamilyDescForMeta(conf)) .setColumnFamily(FSTableDescriptors.getReplBarrierFamilyDescForMeta()).build(); - long pid = this.modifyTable(TableName.META_TABLE_NAME, () -> newMetaDesc, 0, 0, false); + long pid = this.modifyTable(getConnection().getMetaTableName(), () -> newMetaDesc, 0, 0, false); waitForProcedureToComplete(pid, "Failed to add table and rep_barrier CFs to meta"); } @@ -2606,7 +2606,7 @@ private void startActiveMasterManager(int infoPort) throws KeeperException { } private static boolean isCatalogTable(final TableName tableName) { - return tableName.equals(MetaTableName.getInstance()); + return TableName.isMetaTableName(tableName); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterAnnotationReadingPriorityFunction.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterAnnotationReadingPriorityFunction.java index c77d2c0c1a85..f0d5ea07749f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterAnnotationReadingPriorityFunction.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterAnnotationReadingPriorityFunction.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.master; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.AnnotationReadingPriorityFunction; import org.apache.yetus.audience.InterfaceAudience; @@ -85,7 +85,7 @@ protected int getBasePriority(RequestHeader header, Message param) { if (rst.getRegionInfoList() != null) { for (HBaseProtos.RegionInfo info : rst.getRegionInfoList()) { TableName tn = ProtobufUtil.toTableName(info.getTableName()); - if (MetaTableName.getInstance().equals(tn)) { + if (TableName.isMetaTableName(tn)) { return META_TRANSITION_QOS; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java index 3b16014ead55..e44b149820db 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java @@ -35,7 +35,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AsyncClusterConnection; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java index 9903c84785e9..52bb7d2d7667 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; @@ -80,7 +80,7 @@ private void tryMigrateNamespaceTable() throws IOException, InterruptedException if (!opt.isPresent()) { // the procedure is not present, check whether have the ns family in meta table TableDescriptor metaTableDesc = - masterServices.getTableDescriptors().get(MetaTableName.getInstance()); + masterServices.getTableDescriptors().get(masterServices.getConnection().getMetaTableName()); if (metaTableDesc.hasColumnFamily(HConstants.NAMESPACE_FAMILY)) { // normal case, upgrading is done or the cluster is created with 3.x code migrationDone = true; @@ -107,7 +107,7 @@ private void addToCache(Result result, byte[] family, byte[] qualifier) throws I } private void loadFromMeta() throws IOException { - try (Table table = masterServices.getConnection().getTable(MetaTableName.getInstance()); + try (Table table = masterServices.getConnection().getTable(masterServices.getConnection().getMetaTableName()); ResultScanner scanner = table.getScanner(HConstants.NAMESPACE_FAMILY)) { for (Result result;;) { result = scanner.next(); @@ -205,7 +205,7 @@ public static void insertNamespaceToMeta(Connection conn, NamespaceDescriptor ns Put put = new Put(row, true).addColumn(HConstants.NAMESPACE_FAMILY, HConstants.NAMESPACE_COL_DESC_QUALIFIER, ProtobufUtil.toProtoNamespaceDescriptor(ns).toByteArray()); - try (Table table = conn.getTable(MetaTableName.getInstance())) { + try (Table table = conn.getTable(conn.getMetaTableName())) { table.put(put); } } @@ -213,7 +213,7 @@ public static void insertNamespaceToMeta(Connection conn, NamespaceDescriptor ns public void deleteNamespace(String namespaceName) throws IOException { checkMigrationDone(); Delete d = new Delete(Bytes.toBytes(namespaceName)); - try (Table table = masterServices.getConnection().getTable(MetaTableName.getInstance())) { + try (Table table = masterServices.getConnection().getTable(masterServices.getConnection().getMetaTableName())) { table.delete(d); } cache.remove(namespaceName); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java index bfe9a9451c89..243bedcce8e0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java @@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.CatalogFamilyFormat; import org.apache.hadoop.hbase.ClientMetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Result; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java index 58dac734823c..e604e97aaca5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java @@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.PleaseHoldException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java index c08ed70dd538..fbf7c9af770b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.MetaMutationAnnotation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownRegionException; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java index afe7a18ae433..50e9f914d7b5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -170,9 +170,9 @@ public static void visitMetaEntry(final RegionStateVisitor visitor, final Result final long openSeqNum = hrl.getSeqNum(); LOG.debug( - "Load {} entry region={}, regionState={}, lastHost={}, " + "Load hbase:meta entry region={}, regionState={}, lastHost={}, " + "regionLocation={}, openSeqNum={}", - MetaTableName.getInstance(), regionInfo.getEncodedName(), state, lastHost, + regionInfo.getEncodedName(), state, lastHost, regionLocation, openSeqNum); visitor.visitRegionState(result, regionInfo, state, regionLocation, lastHost, openSeqNum); } @@ -192,7 +192,7 @@ private Put generateUpdateRegionLocationPut(RegionStateNode regionStateNode) thr final Put put = new Put(CatalogFamilyFormat.getMetaKeyForRegion(regionInfo), time); MetaTableAccessor.addRegionInfo(put, regionInfo); final StringBuilder info = new StringBuilder("pid=").append(pid).append(" updating ") - .append(MetaTableName.getInstance()).append(" row=") + .append(master.getConnection().getMetaTableName()).append(" row=") .append(regionInfo.getEncodedName()).append(", regionState=").append(state); if (openSeqNum >= 0) { Preconditions.checkArgument(state == State.OPEN && regionLocation != null, @@ -286,7 +286,7 @@ private CompletableFuture updateRegionLocation(RegionInfo regionInfo, Stat } } else { AsyncTable table = - master.getAsyncConnection().getTable(MetaTableName.getInstance()); + master.getAsyncConnection().getTable(master.getConnection().getMetaTableName()); future = table.put(put); } FutureUtils.addListener(future, (r, e) -> { @@ -333,7 +333,7 @@ private void multiMutate(RegionInfo ri, List mutations) throws IOExcep } MutateRowsRequest request = builder.build(); AsyncTable table = master.getConnection().toAsyncConnection() - .getTable(MetaTableName.getInstance()); + .getTable(master.getConnection().getMetaTableName()); CompletableFuture future = table. coprocessorService(MultiRowMutationService::newStub, (stub, controller, done) -> stub.mutateRows(controller, request, done), row); @@ -341,7 +341,7 @@ MutateRowsResponse> coprocessorService(MultiRowMutationService::newStub, } private Table getMetaTable() throws IOException { - return master.getConnection().getTable(MetaTableName.getInstance()); + return master.getConnection().getTable(master.getConnection().getMetaTableName()); } private Result getRegionCatalogResult(RegionInfo region) throws IOException { @@ -479,7 +479,6 @@ public List getMergeRegions(RegionInfo region) throws IOException { /** * Deletes merge qualifiers for the specified merge region. - * @param connection connection we're using * @param mergeRegion the merged region */ public void deleteMergeQualifiers(RegionInfo mergeRegion) throws IOException { @@ -508,7 +507,7 @@ public void deleteMergeQualifiers(RegionInfo mergeRegion) throws IOException { return; } try (Table table = - master.getConnection().getTable(MetaTableName.getInstance())) { + master.getConnection().getTable(master.getConnection().getMetaTableName())) { table.delete(delete); } LOG.info( @@ -570,7 +569,6 @@ private void deleteRegions(List regions, long ts) throws IOException /** * Overwrites the specified regions from hbase:meta. Deletes old rows for the given regions and * adds new ones. Regions added back have state CLOSED. - * @param connection connection we're using * @param regionInfos list of regions to be added to META */ public void overwriteRegions(List regionInfos, int regionReplication) @@ -698,8 +696,7 @@ public static State getRegionState(final Result r, RegionInfo regionInfo) { return State.valueOf(state); } catch (IllegalArgumentException e) { LOG.warn( - "BAD value {} in " + MetaTableName.getInstance() - + " info:state column for region {} , " + "BAD value {} in hbase:meta info:state column for region {} , " + "Consider using HBCK2 setRegionState ENCODED_REGION_NAME STATE", state, regionInfo.getEncodedName()); return null; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java index 950e4cf13aeb..d121bede0a57 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java @@ -38,7 +38,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownRegionException; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java index 419f8fb15312..0695bdd01483 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java @@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; @@ -81,7 +81,7 @@ public synchronized void chore() { long deletedLastPushedSeqIds = 0; TableName tableName = null; List peerIds = null; - try (Table metaTable = conn.getTable(MetaTableName.getInstance()); + try (Table metaTable = conn.getTable(conn.getMetaTableName()); ResultScanner scanner = metaTable.getScanner( new Scan().addFamily(HConstants.REPLICATION_BARRIER_FAMILY).readAllVersions())) { for (;;) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/http/MetaBrowser.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/http/MetaBrowser.java index b6c1daade0ed..231a2c565152 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/http/MetaBrowser.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/http/MetaBrowser.java @@ -33,7 +33,7 @@ import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer; import org.apache.hadoop.hbase.client.AsyncConnection; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java index 0af596e26a41..4a58e80b096a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java index 9b30d5198510..f14872401b94 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java @@ -32,7 +32,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -205,18 +205,18 @@ private static List createMetaEntries(final MasterServices masterSer final List createMetaEntriesFailures = addMetaEntriesResults.stream() .filter(Either::hasRight).map(Either::getRight).collect(Collectors.toList()); LOG.debug("Added {}/{} entries to {}", createMetaEntriesSuccesses.size(), newRegionInfos.size(), - MetaTableName.getInstance()); + "hbase:meta"); if (!createMetaEntriesFailures.isEmpty()) { LOG.warn( "Failed to create entries in {}} for {}/{} RegionInfo descriptors. First" + " failure message included; full list of failures with accompanying stack traces is" + " available at log level DEBUG. message={}", - MetaTableName.getInstance(), createMetaEntriesFailures.size(), addMetaEntriesResults.size(), + "hbase:meta", createMetaEntriesFailures.size(), addMetaEntriesResults.size(), createMetaEntriesFailures.get(0).getMessage()); if (LOG.isDebugEnabled()) { createMetaEntriesFailures.forEach(ioe -> LOG - .debug("Attempt to fix region hole in {} failed.", MetaTableName.getInstance(), ioe)); + .debug("Attempt to fix region hole in {} failed.", "hbase:meta", ioe)); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java index c915db1f0fe9..965c9d728645 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java @@ -22,7 +22,7 @@ import org.apache.hadoop.hbase.ClientMetaTableAccessor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.RegionInfo; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java index 59cf23be8a44..ed21e7f4b132 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java @@ -24,7 +24,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java index 5cb9aca6962b..cd8f2f9d1f45 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java @@ -20,7 +20,7 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java index 88eaf7efcfb0..744d7ae7df11 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java @@ -25,9 +25,10 @@ import org.apache.hadoop.hbase.ClientMetaTableAccessor; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.Result; @@ -152,7 +153,7 @@ public boolean visit(Result result) throws IOException { if (rs.isClosing()) { // Move region to CLOSED in hbase:meta. LOG.info("Moving {} from CLOSING to CLOSED in {}", - hrl.getRegion().getRegionNameAsString(), MetaTableName.getInstance()); + hrl.getRegion().getRegionNameAsString(), "hbase:meta"); try { MetaTableAccessor.updateRegionState(this.connection, hrl.getRegion(), RegionState.State.CLOSED); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java index e2ab040cd850..045249e746d5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java @@ -30,7 +30,7 @@ import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; @@ -68,7 +68,9 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure proc) { // Meta Queue Lookup Helpers // ============================================================================ private MetaQueue getMetaQueue() { - MetaQueue node = AvlTree.get(metaMap, MetaTableName.getInstance(), META_QUEUE_KEY_COMPARATOR); + // TODO(HBASE-XXXXX - Phase 6): Make meta table name dynamic from MasterServices + // For now, hardcode default. Future: pass metaTableName via constructor from Master + TableName metaTableName = TableName.valueOf("hbase", "meta"); + MetaQueue node = AvlTree.get(metaMap, metaTableName, META_QUEUE_KEY_COMPARATOR); if (node != null) { return node; } - node = new MetaQueue(locking.getMetaLock()); + node = new MetaQueue(metaTableName, locking.getMetaLock()); metaMap = AvlTree.insert(metaMap, node); return node; } @@ -1080,7 +1083,9 @@ public boolean waitMetaExclusiveLock(Procedure procedure) { return false; } waitProcedure(lock, procedure); - logLockedResource(LockedResourceType.META, MetaTableName.getInstance().getNameAsString()); + // TODO(Phase 6): Support replica-specific meta table names + // TODO(HBASE-XXXXX - Phase 6): Get dynamic name from MasterServices + logLockedResource(LockedResourceType.META, "hbase:meta"); return true; } finally { schedUnlock(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MetaQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MetaQueue.java index 5915971bd4c1..81fe367f1d4a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MetaQueue.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MetaQueue.java @@ -19,7 +19,7 @@ import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.procedure2.LockStatus; import org.apache.hadoop.hbase.procedure2.Procedure; @@ -33,8 +33,9 @@ @InterfaceAudience.Private class MetaQueue extends Queue { - protected MetaQueue(LockStatus lockStatus) { - super(MetaTableName.getInstance(), 1, lockStatus); + protected // Note: This is called during initialization, tableName is passed from parent + MetaQueue(TableName metaTableName, LockStatus lockStatus) { + super(metaTableName, 1, lockStatus); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java index 909d6862e636..721907727820 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java @@ -20,7 +20,7 @@ import java.io.IOException; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.BufferedMutator; import org.apache.hadoop.hbase.client.Connection; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java index 74603c226a61..e54c515d988c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java @@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.CoprocessorDescriptor; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java index 1b9b95889e05..8c1c0a3832e2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java @@ -25,7 +25,7 @@ import java.util.stream.Collectors; import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.locking.LockProcedure; @@ -175,7 +175,9 @@ List getLocks() { addToLockedResources(lockedResources, regionLocks, Function.identity(), LockedResourceType.REGION); addToLockedResources(lockedResources, peerLocks, Function.identity(), LockedResourceType.PEER); - addToLockedResources(lockedResources, ImmutableMap.of(MetaTableName.getInstance(), metaLock), + // TODO(Phase 6): Support replica-specific meta table names + // TODO(HBASE-XXXXX - Phase 6): Get dynamic name from MasterServices + addToLockedResources(lockedResources, ImmutableMap.of(TableName.valueOf("hbase", "meta"), metaLock), tn -> tn.getNameAsString(), LockedResourceType.META); addToLockedResources(lockedResources, globalLocks, Function.identity(), LockedResourceType.GLOBAL); @@ -237,7 +239,9 @@ public String toString() { .append("tableLocks", filterUnlocked(tableLocks)) .append("regionLocks", filterUnlocked(regionLocks)) .append("peerLocks", filterUnlocked(peerLocks)) - .append("metaLocks", filterUnlocked(ImmutableMap.of(MetaTableName.getInstance(), metaLock))) + // TODO(Phase 6): Support replica-specific meta table names + // TODO(HBASE-XXXXX - Phase 6): Get dynamic name from MasterServices + .append("metaLocks", filterUnlocked(ImmutableMap.of(TableName.valueOf("hbase", "meta"), metaLock))) .append("globalLocks", filterUnlocked(globalLocks)).build(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java index 4adf36efc7f6..3da0c61c9bf5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java @@ -20,7 +20,7 @@ import java.io.IOException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseIOException; -import org.apache.hadoop.hbase.MetaTableName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.MasterFileSystem; @@ -231,7 +231,7 @@ public void toStringClassDetails(StringBuilder sb) { } private boolean prepareTruncate() throws IOException { - if (getTableName().equals(MetaTableName.getInstance())) { + if (TableName.isMetaTableName(getTableName())) { throw new IOException("Can't truncate region in catalog tables"); } return true; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 957650564e20..3ef578c4f63c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HBaseRpcServicesBase; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.MultiActionResultTooLarge; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.PrivateCellUtil; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java index ebdd54a081d0..5bb7e30e5e77 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Get; @@ -193,7 +193,7 @@ public static ReplicationBarrierResult getReplicationBarrierResult(Connection co .addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER) .addFamily(HConstants.REPLICATION_BARRIER_FAMILY).readAllVersions().setReversed(true) .setCaching(10); - try (Table table = conn.getTable(MetaTableName.getInstance()); + try (Table table = conn.getTable(conn.getMetaTableName()); ResultScanner scanner = table.getScanner(scan)) { for (Result result;;) { result = scanner.next(); @@ -216,7 +216,7 @@ public static ReplicationBarrierResult getReplicationBarrierResult(Connection co public static long[] getReplicationBarriers(Connection conn, byte[] regionName) throws IOException { - try (Table table = conn.getTable(MetaTableName.getInstance())) { + try (Table table = conn.getTable(conn.getMetaTableName())) { Result result = table.get(new Get(regionName) .addColumn(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER) .readAllVersions()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java index d90d671feffc..0f7c71354ff8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -545,7 +545,7 @@ public AuthResult permissionGranted(String request, User user, Action permReques TableName tableName, Map> families) { // 1. All users need read access to hbase:meta table. // this is a very common operation, so deal with it quickly. - if (MetaTableName.getInstance().equals(tableName)) { + if (TableName.isMetaTableName(tableName)) { if (permRequest == Action.READ) { return AuthResult.allow(request, "All users allowed", user, permRequest, tableName, families); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java index 7c0dbffb1330..c6c2a88172a4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java @@ -65,7 +65,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.AsyncAdmin; @@ -662,19 +662,19 @@ private void checkRegionIndexValid(int idx, List> startEndK TableName tableName) throws IOException { if (idx < 0) { throw new IOException("The first region info for table " + tableName + " can't be found in " - + MetaTableName.getInstance() + ". Please use hbck tool to fix it" + " first."); + + "hbase:meta. Please use hbck tool to fix it first."); } else if ( (idx == startEndKeys.size() - 1) && !Bytes.equals(startEndKeys.get(idx).getSecond(), HConstants.EMPTY_BYTE_ARRAY) ) { throw new IOException("The last region info for table " + tableName + " can't be found in " - + MetaTableName.getInstance() + ". Please use hbck tool to fix it" + " first."); + + "hbase:meta. Please use hbck tool to fix it first."); } else if ( idx + 1 < startEndKeys.size() && !(Bytes.compareTo(startEndKeys.get(idx).getSecond(), startEndKeys.get(idx + 1).getFirst()) == 0) ) { throw new IOException("The endkey of one region for table " + tableName - + " is not equal to the startkey of the next region in " + MetaTableName.getInstance() + "." + + " is not equal to the startkey of the next region in hbase:meta." + " Please use hbck tool to fix it first."); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java index e3c13f92a017..1204aaa73d6c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java @@ -46,7 +46,7 @@ import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -144,24 +144,30 @@ public static void tryUpdateMetaTableDescriptor(Configuration conf) throws IOExc CommonFSUtils.getRootDir(conf)); } + private static TableName getMetaTableNameFromConf(Configuration conf) { + // TODO(Phase 6): Support replica-specific meta table names from configuration + return TableName.valueOf("hbase", "meta"); + } + public static TableDescriptor tryUpdateAndGetMetaTableDescriptor(Configuration conf, FileSystem fs, Path rootdir) throws IOException { // see if we already have meta descriptor on fs. Write one if not. + TableName metaTableName = getMetaTableNameFromConf(conf); Optional> opt = getTableDescriptorFromFs(fs, - CommonFSUtils.getTableDir(rootdir, MetaTableName.getInstance()), false); + CommonFSUtils.getTableDir(rootdir, metaTableName), false); if (opt.isPresent()) { return opt.get().getSecond(); } TableDescriptorBuilder builder = createMetaTableDescriptorBuilder(conf); TableDescriptor td = StoreFileTrackerFactory.updateWithTrackerConfigs(conf, builder.build()); - LOG.info("Creating new {} table descriptor {}", MetaTableName.getInstance(), td); + LOG.info("Creating new {} table descriptor {}", metaTableName, td); TableName tableName = td.getTableName(); Path tableDir = CommonFSUtils.getTableDir(rootdir, tableName); Path p = writeTableDescriptor(fs, td, tableDir, null); if (p == null) { - throw new IOException("Failed update " + MetaTableName.getInstance() + " table descriptor"); + throw new IOException("Failed update " + metaTableName + " table descriptor"); } - LOG.info("Updated {} table descriptor to {}", MetaTableName.getInstance(), p); + LOG.info("Updated {} table descriptor to {}", metaTableName, p); return td; } @@ -199,7 +205,7 @@ private static TableDescriptorBuilder createMetaTableDescriptorBuilder(final Con // TODO We used to set CacheDataInL1 for META table. When we have BucketCache in file mode, now // the META table data goes to File mode BC only. Test how that affect the system. If too much, // we have to rethink about adding back the setCacheDataInL1 for META table CFs. - return TableDescriptorBuilder.newBuilder(MetaTableName.getInstance()) + return TableDescriptorBuilder.newBuilder(getMetaTableNameFromConf(conf)) .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(HConstants.CATALOG_FAMILY) .setMaxVersions( conf.getInt(HConstants.HBASE_META_VERSIONS, HConstants.DEFAULT_HBASE_META_VERSIONS)) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index b94511c9c264..d121da6e506d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -81,7 +81,7 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java index e3516ed855b0..a49b0a15c1a5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ClusterMetrics.Option; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; @@ -149,7 +149,7 @@ public static void closeRegionSilentlyAndWait(Connection connection, ServerName public static void fixMetaHoleOnlineAndAddReplicas(Configuration conf, RegionInfo hri, Collection servers, int numReplicas) throws IOException { Connection conn = ConnectionFactory.createConnection(conf); - Table meta = conn.getTable(MetaTableName.getInstance()); + Table meta = conn.getTable(conn.getMetaTableName()); Put put = MetaTableAccessor.makePutFromRegionInfo(hri); if (numReplicas > 1) { Random rand = ThreadLocalRandom.current(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java index 11d295435f73..bb582ad4498b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java @@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.client.Admin; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java index 36ecfbd1a247..bbb7b8499cd2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.ExtendedCell; import org.apache.hadoop.hbase.MetaCellComparator; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.io.hfile.CacheConfig; @@ -77,7 +77,7 @@ public BoundedRecoveredHFilesOutputSink(WALSplitter walSplitter, void append(RegionEntryBuffer buffer) throws IOException { Map> familyCells = new HashMap<>(); Map familySeqIds = new HashMap<>(); - boolean isMetaTable = buffer.tableName.equals(MetaTableName.getInstance()); + boolean isMetaTable = TableName.isMetaTableName(buffer.tableName); // First iterate all Cells to find which column families are present and to stamp Cell with // sequence id. for (WAL.Entry entry : buffer.entryBuffer) { diff --git a/hbase-server/src/main/resources/hbase-webapps/master/catalogTables.jsp b/hbase-server/src/main/resources/hbase-webapps/master/catalogTables.jsp index be5b63cff9ea..9f79df924014 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/catalogTables.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/catalogTables.jsp @@ -20,7 +20,6 @@ <%@ page contentType="text/html;charset=UTF-8" import="java.util.*" - import="org.apache.hadoop.hbase.MetaTableName" import="org.apache.hadoop.hbase.NamespaceDescriptor" import="org.apache.hadoop.hbase.TableName" import="org.apache.hadoop.hbase.master.HMaster" diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp index c94b32e5d651..2a300e2629bc 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp @@ -41,7 +41,6 @@ import="org.apache.hadoop.hbase.ServerMetrics" import="org.apache.hadoop.hbase.ServerName" import="org.apache.hadoop.hbase.Size" - import="org.apache.hadoop.hbase.MetaTableName" import="org.apache.hadoop.hbase.TableName" import="org.apache.hadoop.hbase.client.AsyncAdmin" import="org.apache.hadoop.hbase.client.AsyncConnection" diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseClusterInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseClusterInterface.java index 686ba6204b01..e5d73c2296fb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseClusterInterface.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseClusterInterface.java @@ -388,7 +388,7 @@ public boolean restoreClusterMetrics(ClusterMetrics desiredStatus) throws IOExce * Get the ServerName of region server serving the first hbase:meta region */ public ServerName getServerHoldingMeta() throws IOException { - return getServerHoldingRegion(connection.getMetaTableName(), + return getServerHoldingRegion(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable(), RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java index 28a74b0f79e1..a7df353f88bc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java @@ -902,7 +902,7 @@ public SingleProcessHBaseCluster startMiniHBaseCluster(StartTestingClusterOption // Populate the master address configuration from mini cluster configuration. conf.set(HConstants.MASTER_ADDRS_KEY, MasterRegistry.getMasterAddr(c)); // Don't leave here till we've done a successful scan of the hbase:meta - try (Table t = getConnection().getTable(connection.getMetaTableName()); + try (Table t = getConnection().getMetaTable(); ResultScanner s = t.getScanner(new Scan())) { for (;;) { if (s.next() == null) { @@ -2169,7 +2169,7 @@ public String checksumRows(final Table table) throws Exception { */ public List createMultiRegionsInMeta(final Configuration conf, final TableDescriptor htd, byte[][] startKeys) throws IOException { - try (Table meta = getConnection().getTable(connection.getMetaTableName())) { + try (Table meta = getConnection().getMetaTable()) { Arrays.sort(startKeys, Bytes.BYTES_COMPARATOR); List newRegions = new ArrayList<>(startKeys.length); MetaTableAccessor.updateTableState(getConnection(), htd.getTableName(), @@ -2522,7 +2522,7 @@ public void process(WatchedEvent watchedEvent) { monitor.close(); if (checkStatus) { - getConnection().getTable(connection.getMetaTableName()).close(); + getConnection().getMetaTable().close(); } } @@ -3047,7 +3047,7 @@ public void waitUntilAllRegionsAssigned(final TableName tableName) throws IOExce * Waith until all system table's regions get assigned */ public void waitUntilAllSystemRegionsAssigned() throws IOException { - waitUntilAllRegionsAssigned(connection.getMetaTableName()); + waitUntilAllRegionsAssigned(getConnection().getMetaTableName()); } /** @@ -3060,7 +3060,7 @@ public void waitUntilAllSystemRegionsAssigned() throws IOException { public void waitUntilAllRegionsAssigned(final TableName tableName, final long timeout) throws IOException { if (!TableName.isMetaTableName(tableName)) { - try (final Table meta = getConnection().getTable(connection.getMetaTableName())) { + try (final Table meta = getConnection().getMetaTable()) { LOG.debug("Waiting until all regions of table " + tableName + " get assigned. Timeout = " + timeout + "ms"); waitFor(timeout, 200, true, new ExplainingPredicate() { @@ -3280,7 +3280,7 @@ public Table createRandomTable(TableName tableName, final Collection fam Bytes.toBytes(String.format(keyFormat, splitEndKey)), numRegions); if (hbaseCluster != null) { - getMiniHBaseCluster().flushcache(connection.getMetaTableName()); + getMiniHBaseCluster().flushcache(getConnection().getMetaTableName()); } BufferedMutator mutator = getConnection().getBufferedMutator(tableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java index ae53760d7924..51e9bd458b06 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java @@ -68,11 +68,11 @@ public void after() throws Exception { // make sure that with every possible way, we get the same meta table descriptor. private TableDescriptor getMetaDescriptor() throws TableNotFoundException, IOException { Admin admin = UTIL.getAdmin(); - TableDescriptor get = admin.getDescriptor(connection.getMetaTableName()); + TableDescriptor get = admin.getDescriptor(UTIL.getConnection().getMetaTableName()); TableDescriptor list = admin.listTableDescriptors(true).stream().filter(td -> td.isMetaTable()).findAny().get(); TableDescriptor listByName = - admin.listTableDescriptors(Collections.singletonList(connection.getMetaTableName())).get(0); + admin.listTableDescriptors(Collections.singletonList(UTIL.getConnection().getMetaTableName())).get(0); TableDescriptor listByNs = admin.listTableDescriptorsByNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME).stream() .filter(td -> td.isMetaTable()).findAny().get(); @@ -91,7 +91,7 @@ private TableDescriptor getMetaDescriptor() throws TableNotFoundException, IOExc @Test public void testEditMeta() throws IOException { Admin admin = UTIL.getAdmin(); - admin.tableExists(connection.getMetaTableName()); + admin.tableExists(UTIL.getConnection().getMetaTableName()); TableDescriptor originalDescriptor = getMetaDescriptor(); ColumnFamilyDescriptor cfd = originalDescriptor.getColumnFamily(HConstants.CATALOG_FAMILY); int oldVersions = cfd.getMaxVersions(); @@ -100,11 +100,11 @@ public void testEditMeta() throws IOException { .setConfiguration(ColumnFamilyDescriptorBuilder.DATA_BLOCK_ENCODING, DataBlockEncoding.ROW_INDEX_V1.toString()) .build(); - admin.modifyColumnFamily(connection.getMetaTableName(), cfd); + admin.modifyColumnFamily(UTIL.getConnection().getMetaTableName(), cfd); byte[] extraColumnFamilyName = Bytes.toBytes("xtra"); ColumnFamilyDescriptor newCfd = ColumnFamilyDescriptorBuilder.newBuilder(extraColumnFamilyName).build(); - admin.addColumnFamily(connection.getMetaTableName(), newCfd); + admin.addColumnFamily(UTIL.getConnection().getMetaTableName(), newCfd); TableDescriptor descriptor = getMetaDescriptor(); // Assert new max versions is == old versions plus 1. assertEquals(oldVersions + 1, @@ -126,11 +126,11 @@ public void testEditMeta() throws IOException { assertEquals(encoding, DataBlockEncoding.ROW_INDEX_V1.toString()); assertTrue(r.getStore(extraColumnFamilyName) != null); // Assert we can't drop critical hbase:meta column family but we can drop any other. - admin.deleteColumnFamily(connection.getMetaTableName(), newCfd.getName()); + admin.deleteColumnFamily(UTIL.getConnection().getMetaTableName(), newCfd.getName()); descriptor = getMetaDescriptor(); assertTrue(descriptor.getColumnFamily(newCfd.getName()) == null); try { - admin.deleteColumnFamily(connection.getMetaTableName(), HConstants.CATALOG_FAMILY); + admin.deleteColumnFamily(UTIL.getConnection().getMetaTableName(), HConstants.CATALOG_FAMILY); fail("Should not reach here"); } catch (HBaseIOException hioe) { assertTrue(hioe.getMessage().contains("Delete of hbase:meta")); @@ -144,7 +144,7 @@ public void testEditMeta() throws IOException { @Test public void testAlterMetaWithReadOnly() throws IOException { Admin admin = UTIL.getAdmin(); - TableDescriptor origMetaTableDesc = admin.getDescriptor(connection.getMetaTableName()); + TableDescriptor origMetaTableDesc = admin.getDescriptor(UTIL.getConnection().getMetaTableName()); assertFalse(origMetaTableDesc.isReadOnly()); TableDescriptor newTD = TableDescriptorBuilder.newBuilder(origMetaTableDesc).setReadOnly(true).build(); @@ -152,7 +152,7 @@ public void testAlterMetaWithReadOnly() throws IOException { admin.modifyTable(newTD); fail("Meta table can't be set as read only"); } catch (Exception e) { - assertFalse(admin.getDescriptor(connection.getMetaTableName()).isReadOnly()); + assertFalse(admin.getDescriptor(UTIL.getConnection().getMetaTableName()).isReadOnly()); } // Create a table to check region assignment & meta operation diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java index c14a3cbd1a71..286c227fb2c5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestInfoServers.java @@ -91,7 +91,7 @@ public void testGetMasterInfoPort() throws Exception { @Test public void testInfoServersRedirect() throws Exception { // give the cluster time to start up - UTIL.getConnection().getTable(connection.getMetaTableName()).close(); + UTIL.getConnection().getMetaTable().close(); int port = UTIL.getHBaseCluster().getMaster().getInfoServer().getPort(); assertContainsContent(new URL("http://localhost:" + port + "/index.html"), "master.jsp"); assertContainsContent(new URL("http://localhost:" + port + "/master-status"), "master.jsp"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java index 01e18b23024c..99e59c73a440 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java @@ -87,7 +87,7 @@ private void multiMutate(byte[] row, List mutations) throws IOExceptio } } MutateRowsRequest request = builder.build(); - AsyncTable table = UTIL.getAsyncConnection().getTable(connection.getMetaTableName()); + AsyncTable table = UTIL.getAsyncConnection().getTable(TEST_UTIL.getConnection().getMetaTableName()); CompletableFuture future = table. coprocessorService(MultiRowMutationService::newStub, (stub, controller, done) -> stub.mutateRows(controller, request, done), row); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java index 3508dd1b0d92..44c43885aa77 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestNamespace.java @@ -117,7 +117,7 @@ public void verifyReservedNS() throws IOException { assertEquals(2, admin.listNamespaceDescriptors().length); // verify existence of system tables - Set systemTables = Sets.newHashSet(connection.getMetaTableName()); + Set systemTables = Sets.newHashSet(TEST_UTIL.getConnection().getMetaTableName()); List descs = admin.listTableDescriptorsByNamespace( Bytes.toBytes(NamespaceDescriptor.SYSTEM_NAMESPACE.getName())); assertEquals(systemTables.size(), descs.size()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerInternalsTracing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerInternalsTracing.java index a05e54897070..b6e639d95b77 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerInternalsTracing.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestServerInternalsTracing.java @@ -77,7 +77,7 @@ public Setup(final Supplier testingUtilSupplier) { @Override protected void before() throws Throwable { final HBaseTestingUtil testingUtil = testingUtilSupplier.get(); - testingUtil.waitTableAvailable(connection.getMetaTableName()); + testingUtil.waitTableAvailable(testingUtil.getConnection().getMetaTableName()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java index 99e2920eccaf..66e86cf031bf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.Region; @@ -50,7 +50,7 @@ public abstract class AbstractTestRegionLocator { protected static void startClusterAndCreateTable() throws Exception { UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), connection.getMetaTableName(), REGION_REPLICATION); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), TEST_UTIL.getConnection().getMetaTableName(), REGION_REPLICATION); TableDescriptor td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRegionReplication(REGION_REPLICATION) .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build(); @@ -70,7 +70,7 @@ protected static void startClusterAndCreateTable() throws Exception { @After public void tearDownAfterTest() throws IOException { clearCache(TABLE_NAME); - clearCache(connection.getMetaTableName()); + clearCache(TEST_UTIL.getConnection().getMetaTableName()); } private byte[] getStartKey(int index) { @@ -172,7 +172,7 @@ private void assertMetaRegionLocation(HRegionLocation loc, int replicaId) { assertArrayEquals(HConstants.EMPTY_END_ROW, region.getEndKey()); assertEquals(replicaId, region.getReplicaId()); ServerName expected = - findRegionLocation(connection.getMetaTableName(), region.getStartKey(), replicaId); + findRegionLocation(TEST_UTIL.getConnection().getMetaTableName(), region.getStartKey(), replicaId); assertEquals(expected, loc.getServerName()); } @@ -185,19 +185,19 @@ private void assertMetaRegionLocations(List locs) { @Test public void testMeta() throws IOException { - assertMetaStartOrEndKeys(getStartKeys(connection.getMetaTableName())); - assertMetaStartOrEndKeys(getEndKeys(connection.getMetaTableName())); - Pair startEndKeys = getStartEndKeys(connection.getMetaTableName()); + assertMetaStartOrEndKeys(getStartKeys(TEST_UTIL.getConnection().getMetaTableName())); + assertMetaStartOrEndKeys(getEndKeys(TEST_UTIL.getConnection().getMetaTableName())); + Pair startEndKeys = getStartEndKeys(TEST_UTIL.getConnection().getMetaTableName()); assertMetaStartOrEndKeys(startEndKeys.getFirst()); assertMetaStartOrEndKeys(startEndKeys.getSecond()); for (int replicaId = 0; replicaId < REGION_REPLICATION; replicaId++) { assertMetaRegionLocation( - getRegionLocation(connection.getMetaTableName(), HConstants.EMPTY_START_ROW, replicaId), + getRegionLocation(TEST_UTIL.getConnection().getMetaTableName(), HConstants.EMPTY_START_ROW, replicaId), replicaId); } assertMetaRegionLocations( - getRegionLocations(connection.getMetaTableName(), HConstants.EMPTY_START_ROW)); - assertMetaRegionLocations(getAllRegionLocations(connection.getMetaTableName())); + getRegionLocations(TEST_UTIL.getConnection().getMetaTableName(), HConstants.EMPTY_START_ROW)); + assertMetaRegionLocations(getAllRegionLocations(TEST_UTIL.getConnection().getMetaTableName())); } protected abstract byte[][] getStartKeys(TableName tableName) throws IOException; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java index c1cad1a911dd..4d69ba1ff69d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java @@ -24,7 +24,7 @@ import java.util.HashSet; import java.util.Set; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableNameTestRule; @@ -58,7 +58,7 @@ protected static void startCluster() throws Exception { .numAlwaysStandByMasters(1).numMasters(1).numRegionServers(REGIONSERVERS_COUNT).build(); TEST_UTIL.startMiniCluster(option); Admin admin = TEST_UTIL.getAdmin(); - HBaseTestingUtil.setReplicas(admin, connection.getMetaTableName(), 3); + HBaseTestingUtil.setReplicas(admin, TEST_UTIL.getConnection().getMetaTableName(), 3); AssignmentManager am = TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager(); Set sns = new HashSet(); ServerName hbaseMetaServerName = am.getRegionStates() diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java index 459deb9f3157..e8dd3b58e86f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java @@ -28,7 +28,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; @@ -48,7 +48,7 @@ static void waitUntilAllMetaReplicasAreReady(HBaseTestingUtil util, ConnectionRe throws IOException { Configuration conf = util.getConfiguration(); int regionReplicaCount = - util.getAdmin().getDescriptor(connection.getMetaTableName()).getRegionReplication(); + util.getAdmin().getDescriptor(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).getRegionReplication(); Waiter.waitFor(conf, conf.getLong("hbase.client.sync.wait.timeout.msec", 60000), 200, true, new ExplainingPredicate() { @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java index c1623b2bbc96..17912ec15354 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; @@ -65,7 +65,7 @@ public class TestAdmin extends TestAdminBase { @Test public void testListTableDescriptors() throws IOException { TableDescriptor metaTableDescriptor = - TEST_UTIL.getAdmin().getDescriptor(connection.getMetaTableName()); + TEST_UTIL.getAdmin().getDescriptor(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); List tableDescriptors = TEST_UTIL.getAdmin().listTableDescriptors(true); assertTrue(tableDescriptors.contains(metaTableDescriptor)); tableDescriptors = TEST_UTIL.getAdmin().listTableDescriptors(false); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index 67bee384cee6..30521b350800 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.ClusterMetrics.Option; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableExistsException; @@ -85,13 +85,13 @@ public class TestAdmin2 extends TestAdminBase { public void testCreateBadTables() throws IOException { String msg = null; try { - ADMIN.createTable(TableDescriptorBuilder.newBuilder(connection.getMetaTableName()).build()); + ADMIN.createTable(TableDescriptorBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).build()); } catch (TableExistsException e) { msg = e.toString(); } assertTrue("Unexcepted exception message " + msg, msg != null && msg.startsWith(TableExistsException.class.getName()) - && msg.contains(connection.getMetaTableName().getNameAsString())); + && msg.contains(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable().getNameAsString())); // Now try and do concurrent creation with a bunch of threads. TableDescriptor tableDescriptor = @@ -457,7 +457,7 @@ private void setUpforLogRolling() { private HRegionServer startAndWriteData(TableName tableName, byte[] value) throws IOException, InterruptedException { // When the hbase:meta table can be opened, the region servers are running - TEST_UTIL.getConnection().getTable(connection.getMetaTableName()).close(); + TEST_UTIL.getConnection().getMetaTable().close(); // Create the test table and open it TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(tableName) @@ -487,7 +487,7 @@ private HRegionServer startAndWriteData(TableName tableName, byte[] value) @Test public void testDisableCatalogTable() throws Exception { try { - ADMIN.disableTable(connection.getMetaTableName()); + ADMIN.disableTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); fail("Expected to throw ConstraintException"); } catch (ConstraintException e) { } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java index 61d9e774ca32..bbf784144cdb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java @@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -55,7 +55,7 @@ public class TestAsyncAdminWithRegionReplicas extends TestAsyncAdminBase { @BeforeClass public static void setUpBeforeClass() throws Exception { TestAsyncAdminBase.setUpBeforeClass(); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), connection.getMetaTableName(), 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), RegionInfoBuilder.FIRST_META_REGIONINFO.getTable(), 3); try (ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent())) { RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, registry); @@ -81,7 +81,7 @@ public void testMoveNonDefaultReplica() throws InterruptedException, ExecutionException, IOException { createTableWithDefaultConf(tableName, 3); testMoveNonDefaultReplica(tableName); - testMoveNonDefaultReplica(connection.getMetaTableName()); + testMoveNonDefaultReplica(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); } @Test @@ -139,11 +139,11 @@ public void testCloneTableSchema() throws IOException, InterruptedException, Exe @Test public void testGetTableRegions() throws InterruptedException, ExecutionException, IOException { - List metaRegions = admin.getRegions(connection.getMetaTableName()).get(); + List metaRegions = admin.getRegions(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).get(); assertEquals(3, metaRegions.size()); for (int i = 0; i < 3; i++) { RegionInfo metaRegion = metaRegions.get(i); - assertEquals(connection.getMetaTableName(), metaRegion.getTable()); + assertEquals(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable(), metaRegion.getTable()); assertEquals(i, metaRegion.getReplicaId()); } createTableWithDefaultConf(tableName, 3); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java index a9811f9243fb..c503e133fb6f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java @@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MatcherPredicate; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.MiniClusterRule; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.StartTestingClusterOption; @@ -106,7 +106,7 @@ private void assertInitialized() { protected void before() throws Throwable { final AsyncAdmin admin = connectionRule.getAsyncConnection().getAdmin(); testUtil = miniClusterRule.getTestingUtility(); - HBaseTestingUtil.setReplicas(admin, connection.getMetaTableName(), 3); + HBaseTestingUtil.setReplicas(admin, testUtil.getConnection().getMetaTableName(), 3); testUtil.waitUntilNoRegionsInTransition(); registry = ConnectionRegistryFactory.create(testUtil.getConfiguration(), User.getCurrent()); RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(testUtil, registry); @@ -164,7 +164,7 @@ public void test() throws Exception { TraceUtil.trace(() -> { try { - testLocator(miniClusterRule.getTestingUtility(), connection.getMetaTableName(), + testLocator(miniClusterRule.getTestingUtility(), testUtil.getConnection().getMetaTableName(), new Locator() { @Override public void updateCachedLocationOnError(HRegionLocation loc, Throwable error) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java index e0164c65bf08..c72d5bc1c5b2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java @@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; @@ -107,9 +107,9 @@ public static void setUp() throws Exception { admin.balancerSwitch(false, true); // Enable hbase:meta replication. - HBaseTestingUtil.setReplicas(admin, connection.getMetaTableName(), NUM_OF_META_REPLICA); + HBaseTestingUtil.setReplicas(admin, TEST_UTIL.getConnection().getMetaTableName(), NUM_OF_META_REPLICA); TEST_UTIL.waitFor(30000, - () -> TEST_UTIL.getMiniHBaseCluster().getRegions(connection.getMetaTableName()).size() + () -> TEST_UTIL.getMiniHBaseCluster().getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() >= NUM_OF_META_REPLICA); SPLIT_KEYS = new byte[8][]; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java index 374da4811657..e7e944fcd232 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil; @@ -87,7 +87,7 @@ public void testSplitSwitch() throws Exception { TestAsyncRegionAdminApi.loadData(tableName, families, rows); AsyncTable metaTable = - ASYNC_CONN.getTable(connection.getMetaTableName()); + ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); int originalCount = regionLocations.size(); @@ -119,7 +119,7 @@ public void testMergeSwitch() throws Exception { TestAsyncRegionAdminApi.loadData(tableName, families, 1000); AsyncTable metaTable = - ASYNC_CONN.getTable(connection.getMetaTableName()); + ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); int originalCount = regionLocations.size(); @@ -165,7 +165,7 @@ public void testMergeRegions() throws Exception { createTableWithDefaultConf(tableName, splitRows); AsyncTable metaTable = - ASYNC_CONN.getTable(connection.getMetaTableName()); + ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); RegionInfo regionA; @@ -246,7 +246,7 @@ private void splitTest(TableName tableName, int rowCount, boolean isSplitRegion, createTableWithDefaultConf(tableName); AsyncTable metaTable = - ASYNC_CONN.getTable(connection.getMetaTableName()); + ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); assertEquals(1, regionLocations.size()); @@ -304,7 +304,7 @@ public void testTruncateRegion() throws Exception { createTableWithDefaultConf(tableName, splitKeys, bFamilies); AsyncTable metaTable = - ASYNC_CONN.getTable(connection.getMetaTableName()); + ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); RegionInfo regionToBeTruncated = regionLocations.get(0).getRegion(); @@ -339,7 +339,7 @@ public void testTruncateReplicaRegionNotAllowed() throws Exception { createTableWithDefaultConf(tableName, 2, splitKeys, bFamilies); AsyncTable metaTable = - ASYNC_CONN.getTable(connection.getMetaTableName()); + ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); RegionInfo primaryRegion = regionLocations.get(0).getRegion(); @@ -360,7 +360,7 @@ public void testTruncateReplicaRegionNotAllowed() throws Exception { @Test public void testTruncateRegionsMetaTableRegionsNotAllowed() throws Exception { - AsyncTableRegionLocator locator = ASYNC_CONN.getRegionLocator(connection.getMetaTableName()); + AsyncTableRegionLocator locator = ASYNC_CONN.getRegionLocator(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); List regionLocations = locator.getAllRegionLocations().get(); HRegionLocation regionToBeTruncated = regionLocations.get(0); // 1 diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java index 7b0b0c93a95d..e044afdab239 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java @@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; @@ -73,7 +73,7 @@ public void testCreateTable() throws Exception { static TableState.State getStateFromMeta(TableName table) throws Exception { Optional state = ClientMetaTableAccessor - .getTableState(ASYNC_CONN.getTable(connection.getMetaTableName()), table).get(); + .getTableState(ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()), table).get(); assertTrue(state.isPresent()); return state.get().getState(); } @@ -81,7 +81,7 @@ static TableState.State getStateFromMeta(TableName table) throws Exception { @Test public void testCreateTableNumberOfRegions() throws Exception { AsyncTable metaTable = - ASYNC_CONN.getTable(connection.getMetaTableName()); + ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); createTableWithDefaultConf(tableName); List regionLocations = @@ -130,7 +130,7 @@ public void testCreateTableWithRegions() throws Exception { assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable); AsyncTable metaTable = - ASYNC_CONN.getTable(connection.getMetaTableName()); + ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); List regions = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); Iterator hris = regions.iterator(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java index 08ea9689c889..8c3628346987 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java @@ -26,7 +26,7 @@ import java.util.Set; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -58,7 +58,7 @@ public class TestAsyncTableAdminApi2 extends TestAsyncAdminBase { @Test public void testDisableCatalogTable() throws Exception { try { - this.admin.disableTable(connection.getMetaTableName()).join(); + this.admin.disableTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).join(); fail("Expected to throw ConstraintException"); } catch (Exception e) { } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java index c6081bd8a29b..4b6d9e43056e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -66,10 +66,10 @@ public void testTableExist() throws Exception { TEST_UTIL.createTable(tableName, FAMILY); exist = admin.tableExists(tableName).get(); assertTrue(exist); - exist = admin.tableExists(connection.getMetaTableName()).get(); + exist = admin.tableExists(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).get(); assertTrue(exist); // meta table already exists - exist = admin.tableExists(connection.getMetaTableName()).get(); + exist = admin.tableExists(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).get(); assertTrue(exist); } @@ -118,7 +118,7 @@ public void testListTables() throws Exception { assertEquals(0, size); Collections.addAll(tableNames, tables); - tableNames.add(connection.getMetaTableName()); + tableNames.add(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); tableDescs = admin.listTableDescriptors(tableNames).get(); size = tableDescs.size(); assertEquals(tables.length + 1, size); @@ -126,7 +126,7 @@ public void testListTables() throws Exception { assertTrue("tableName should be equal in order", tableDescs.get(j).getTableName().equals(tables[i])); } - assertTrue(tableDescs.get(size - 1).getTableName().equals(connection.getMetaTableName())); + assertTrue(tableDescs.get(size - 1).getTableName().equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable())); for (int i = 0; i < tables.length; i++) { admin.disableTable(tables[i]).join(); @@ -205,7 +205,7 @@ public void testDisableAndEnableTable() throws Exception { assertTrue(ok); // meta table can not be disabled. try { - admin.disableTable(connection.getMetaTableName()).get(); + admin.disableTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).get(); fail("meta table can not be disabled"); } catch (ExecutionException e) { Throwable cause = e.getCause(); @@ -286,7 +286,7 @@ public void testEnableTableRetainAssignment() throws Exception { createTableWithDefaultConf(tableName, splitKeys); AsyncTable metaTable = - ASYNC_CONN.getTable(connection.getMetaTableName()); + ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); List regions = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); assertEquals( @@ -315,8 +315,8 @@ public void testIsTableEnabledAndDisabled() throws Exception { assertTrue(admin.isTableDisabled(tableName).get()); // meta table is always enabled - assertTrue(admin.isTableEnabled(connection.getMetaTableName()).get()); - assertFalse(admin.isTableDisabled(connection.getMetaTableName()).get()); + assertTrue(admin.isTableEnabled(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).get()); + assertFalse(admin.isTableDisabled(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).get()); } @Test @@ -324,6 +324,6 @@ public void testIsTableAvailable() throws Exception { createTableWithDefaultConf(tableName); TEST_UTIL.waitTableAvailable(tableName); assertTrue(admin.isTableAvailable(tableName).get()); - assertTrue(admin.isTableAvailable(connection.getMetaTableName()).get()); + assertTrue(admin.isTableAvailable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).get()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java index d7670fd4e872..ac21c6ea8dcc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java @@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.ObserverContext; @@ -95,14 +95,14 @@ public static void setUp() throws Exception { conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, FailPrimaryMetaScanCp.class.getName()); UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), connection.getMetaTableName(), 3); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), TEST_UTIL.getConnection().getMetaTableName(), 3); try (ConnectionRegistry registry = ConnectionRegistryFactory.create(conf, User.getCurrent())) { RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(UTIL, registry); } try (Table table = UTIL.createTable(TABLE_NAME, FAMILY)) { table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)); } - UTIL.flush(connection.getMetaTableName()); + UTIL.flush(TEST_UTIL.getConnection().getMetaTableName()); // wait for the store file refresh so we can read the region location from secondary meta // replicas Thread.sleep(2000); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java index 2dd1baa7ac24..12dc36223479 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java @@ -27,7 +27,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.security.User; @@ -72,9 +72,9 @@ public static void setUp() throws Exception { admin.balancerSwitch(false, true); // Enable hbase:meta replication. - HBaseTestingUtil.setReplicas(admin, connection.getMetaTableName(), numOfMetaReplica); + HBaseTestingUtil.setReplicas(admin, TEST_UTIL.getConnection().getMetaTableName(), numOfMetaReplica); TEST_UTIL.waitFor(30000, - () -> TEST_UTIL.getMiniHBaseCluster().getRegions(connection.getMetaTableName()).size() + () -> TEST_UTIL.getMiniHBaseCluster().getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() >= numOfMetaReplica); registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); @@ -95,14 +95,14 @@ public void testMetaChangeFromReplicaNoReplica() throws IOException, Interrupted CatalogReplicaLoadBalanceSimpleSelector.class.getName()); CatalogReplicaLoadBalanceSelector metaSelector = CatalogReplicaLoadBalanceSelectorFactory - .createSelector(replicaSelectorClass, connection.getMetaTableName(), CONN, () -> { + .createSelector(replicaSelectorClass, TEST_UTIL.getConnection().getMetaTableName(), CONN, () -> { int numOfReplicas = CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS; try { RegionLocations metaLocations = CONN.registry.getMetaRegionLocations() .get(CONN.connConf.getMetaReadRpcTimeoutNs(), TimeUnit.NANOSECONDS); numOfReplicas = metaLocations.size(); } catch (Exception e) { - LOG.error("Failed to get table {}'s region replication, ", connection.getMetaTableName(), + LOG.error("Failed to get table {}'s region replication, ", TEST_UTIL.getConnection().getMetaTableName(), e); } return numOfReplicas; @@ -117,20 +117,20 @@ public void testMetaChangeFromReplicaNoReplica() throws IOException, Interrupted IntStream.range(0, numOfMetaReplica).forEach(i -> assertNotEquals(replicaIdCount[i], 0)); // Change to No meta replica - HBaseTestingUtil.setReplicas(admin, connection.getMetaTableName(), 1); + HBaseTestingUtil.setReplicas(admin, TEST_UTIL.getConnection().getMetaTableName(), 1); TEST_UTIL.waitFor(30000, - () -> TEST_UTIL.getMiniHBaseCluster().getRegions(connection.getMetaTableName()).size() == 1); + () -> TEST_UTIL.getMiniHBaseCluster().getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() == 1); CatalogReplicaLoadBalanceSelector metaSelectorWithNoReplica = CatalogReplicaLoadBalanceSelectorFactory.createSelector(replicaSelectorClass, - connection.getMetaTableName(), CONN, () -> { + TEST_UTIL.getConnection().getMetaTableName(), CONN, () -> { int numOfReplicas = CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS; try { RegionLocations metaLocations = CONN.registry.getMetaRegionLocations() .get(CONN.connConf.getMetaReadRpcTimeoutNs(), TimeUnit.NANOSECONDS); numOfReplicas = metaLocations.size(); } catch (Exception e) { - LOG.error("Failed to get table {}'s region replication, ", connection.getMetaTableName(), + LOG.error("Failed to get table {}'s region replication, ", TEST_UTIL.getConnection().getMetaTableName(), e); } return numOfReplicas; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java index 6f3121e9ea46..f118e2ce746f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java @@ -22,7 +22,7 @@ import java.util.List; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; @@ -48,7 +48,7 @@ public void testReplicaCleanup() throws Exception { ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher(); List metaReplicaZnodes = zkw.getMetaReplicaNodes(); assertEquals(3, metaReplicaZnodes.size()); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), connection.getMetaTableName(), 1); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), RegionInfoBuilder.FIRST_META_REGIONINFO.getTable(), 1); metaReplicaZnodes = zkw.getMetaReplicaNodes(); assertEquals(1, metaReplicaZnodes.size()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java index 66de0abf673f..42caa1350427 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java @@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.SingleProcessHBaseCluster.MiniHBaseClusterRegionServer; import org.apache.hadoop.hbase.TableName; @@ -454,7 +454,7 @@ public ScanResponse scan(final RpcController controller, final ScanRequest reque } else { ScanResponse scanRes = super.scan(controller, request); String regionName = Bytes.toString(request.getRegion().getValue().toByteArray()); - if (!regionName.contains(connection.getMetaTableName().getNameAsString())) { + if (!regionName.contains(TEST_UTIL.getConnection().getMetaTableName().getNameAsString())) { tableScannerId = scanRes.getScannerId(); if (sleepOnOpen) { try { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java index ee75ad008406..7845a9cf89ff 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java @@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.client.metrics.ScanMetricsRegionInfo; @@ -90,8 +90,8 @@ public void setup() throws IOException { conf = TEST_UTIL.getConfiguration(); rootDir = TEST_UTIL.getDefaultRootDirPath(); fs = TEST_UTIL.getTestFileSystem(); - htd = TEST_UTIL.getAdmin().getDescriptor(connection.getMetaTableName()); - hri = TEST_UTIL.getAdmin().getRegions(connection.getMetaTableName()).get(0); + htd = TEST_UTIL.getAdmin().getDescriptor(TEST_UTIL.getConnection().getMetaTableName()); + hri = TEST_UTIL.getAdmin().getRegions(TEST_UTIL.getConnection().getMetaTableName()).get(0); scan = new Scan(); } @@ -201,7 +201,7 @@ private void testScanMetricsWithScanMetricsByRegionDisabled(ScanMetrics scanMetr Configuration copyConf = new Configuration(conf); Scan scan = new Scan(); scan.setScanMetricsEnabled(true); - TEST_UTIL.getAdmin().flush(connection.getMetaTableName()); + TEST_UTIL.getAdmin().flush(TEST_UTIL.getConnection().getMetaTableName()); try (ClientSideRegionScanner clientSideRegionScanner = new ClientSideRegionScanner(copyConf, fs, rootDir, htd, hri, scan, scanMetrics)) { clientSideRegionScanner.next(); @@ -230,7 +230,7 @@ private void testScanMetricByRegion(ScanMetrics scanMetrics) throws IOException Configuration copyConf = new Configuration(conf); Scan scan = new Scan(); scan.setEnableScanMetricsByRegion(true); - TEST_UTIL.getAdmin().flush(connection.getMetaTableName()); + TEST_UTIL.getAdmin().flush(TEST_UTIL.getConnection().getMetaTableName()); try (ClientSideRegionScanner clientSideRegionScanner = new ClientSideRegionScanner(copyConf, fs, rootDir, htd, hri, scan, scanMetrics)) { clientSideRegionScanner.next(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java index d5c4993db711..05ed229eeccb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java @@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor; @@ -93,7 +93,7 @@ public void testDeleteForSureClearsAllTableRowsFromMeta() } // Now I have a nice table, mangle it by removing the HConstants.REGIONINFO_QUALIFIER_STR // content from a few of the rows. - try (Table metaTable = TEST_UTIL.getConnection().getTable(connection.getMetaTableName())) { + try (Table metaTable = TEST_UTIL.getConnection().getMetaTable()) { try (ResultScanner scanner = metaTable.getScanner( MetaTableAccessor.getScanForTableName(TEST_UTIL.getConfiguration(), tableName))) { for (Result result : scanner) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java index 7af27446a028..cc35cd52dba6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java @@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -2533,7 +2533,7 @@ public void testFilterAllRecords() throws IOException { scan.setCaching(1); // Filter out any records scan.setFilter(new FilterList(new FirstKeyOnlyFilter(), new InclusiveStopFilter(new byte[0]))); - try (Table table = TEST_UTIL.getConnection().getTable(connection.getMetaTableName())) { + try (Table table = TEST_UTIL.getConnection().getMetaTable()) { try (ResultScanner s = table.getScanner(scan)) { assertNull(s.next()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java index cf2ea4190a7e..e73baee5dfc2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java @@ -21,7 +21,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -50,7 +50,7 @@ public static void setUp() throws Exception { public void testUpgradeAndIncreaseReplicaCount() throws Exception { HMaster oldMaster = TEST_UTIL.getMiniHBaseCluster().getMaster(); TableDescriptors oldTds = oldMaster.getTableDescriptors(); - TableDescriptor oldMetaTd = oldTds.get(connection.getMetaTableName()); + TableDescriptor oldMetaTd = oldTds.get(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); assertEquals(3, oldMetaTd.getRegionReplication()); // force update the replica count to 1 and then kill the master, to simulate that hen upgrading, // we have no region replication in meta table descriptor but we actually have meta region diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java index 1ce4df6c5de8..e0a681182c94 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java @@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.Waiter; @@ -61,7 +61,7 @@ public static void setUp() throws Exception { StartTestingClusterOption.Builder builder = StartTestingClusterOption.builder(); builder.numMasters(3).numRegionServers(3); TEST_UTIL.startMiniCluster(builder.build()); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), connection.getMetaTableName(), 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TEST_UTIL.getConnection().getMetaTableName(), 3); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java index 8994ba58b0c2..3b39bee7ec85 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.MultiActionResultTooLarge; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionTooBusyException; @@ -88,7 +88,7 @@ public static void setUpBeforeClass() throws Exception { conf.setStrings(HConstants.REGION_SERVER_IMPL, RegionServerWithFakeRpcServices.class.getName()); TEST_UTIL.startMiniCluster(1); TEST_UTIL.getHBaseCluster().waitForActiveAndReadyMaster(); - TEST_UTIL.waitUntilAllRegionsAssigned(connection.getMetaTableName()); + TEST_UTIL.waitUntilAllRegionsAssigned(TEST_UTIL.getConnection().getMetaTableName()); badRS = TEST_UTIL.getHBaseCluster().getRegionServer(0); assertTrue(badRS.getRSRpcServices() instanceof FakeRSRpcServices); TableDescriptor desc = TableDescriptorBuilder.newBuilder(TABLE_NAME) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java index d910778cffe6..588324487acb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java @@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaRegionLocationCache; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.MultithreadedTestUtil; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.master.HMaster; @@ -63,7 +63,7 @@ public class TestMetaRegionLocationCache { @BeforeClass public static void setUp() throws Exception { TEST_UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), connection.getMetaTableName(), 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TEST_UTIL.getConnection().getMetaTableName(), 3); REGISTRY = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, REGISTRY); TEST_UTIL.getAdmin().balancerSwitch(false, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java index 836a3dc451b4..0cf228c6065e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java @@ -24,7 +24,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -54,7 +54,7 @@ public static void setUp() throws Exception { @Test public void testMetaHTDReplicaCount() throws Exception { assertEquals(3, - TEST_UTIL.getAdmin().getDescriptor(connection.getMetaTableName()).getRegionReplication()); + TEST_UTIL.getAdmin().getDescriptor(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).getRegionReplication()); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java index 26faaa431a43..3fc3fe62698e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore; @@ -98,7 +98,7 @@ public static void shutdownMetaAndDoValidations(HBaseTestingUtil util) throws Ex ServerName master = null; try (Connection c = ConnectionFactory.createConnection(util.getConfiguration())) { try (Table htable = util.createTable(TABLE, FAMILIES)) { - util.getAdmin().flush(connection.getMetaTableName()); + util.getAdmin().flush(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); Thread.sleep( conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 30000) * 6); List regions = MetaTableAccessor.getTableRegions(c, TABLE); @@ -115,7 +115,7 @@ public static void shutdownMetaAndDoValidations(HBaseTestingUtil util) throws Ex Thread.sleep(10); hrl = MetaTableAccessor.getRegionLocation(c, regions.get(0)); } while (primary.equals(hrl.getServerName())); - util.getAdmin().flush(connection.getMetaTableName()); + util.getAdmin().flush(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); Thread.sleep( conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD, 30000) * 3); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java index 6d774cd366f6..c631fa11c65e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java @@ -22,7 +22,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -49,7 +49,7 @@ public class TestMultiActionMetricsFromClient { public static void setUpBeforeClass() throws Exception { TEST_UTIL.startMiniCluster(1); TEST_UTIL.getHBaseCluster().waitForActiveAndReadyMaster(); - TEST_UTIL.waitUntilAllRegionsAssigned(connection.getMetaTableName()); + TEST_UTIL.waitUntilAllRegionsAssigned(TEST_UTIL.getConnection().getMetaTableName()); TEST_UTIL.createTable(TABLE_NAME, FAMILY); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java index 67107441cae8..04938eb2c2c9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java @@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor; @@ -262,7 +262,7 @@ public static void beforeClass() throws Exception { HTU.startMiniCluster(NB_SERVERS); // Enable meta replica at server side - HBaseTestingUtil.setReplicas(HTU.getAdmin(), connection.getMetaTableName(), 2); + HBaseTestingUtil.setReplicas(HTU.getAdmin(), TEST_HTU.getConnection().getMetaTableName(), 2); HTU.getHBaseCluster().startMaster(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java index 8870fcc26171..eaeabeeeac46 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.regionserver.BootstrapNodeManager; @@ -72,7 +72,7 @@ public static void setUpBeforeClass() throws Exception { UTIL.getConfiguration().setLong(RpcConnectionRegistry.MIN_SECS_BETWEEN_REFRESHES, 0); UTIL.getConfiguration().setLong(BootstrapNodeManager.REQUEST_MASTER_MIN_INTERVAL_SECS, 1); UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), connection.getMetaTableName(), 3); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), UTIL.getConnection().getMetaTableName(), 3); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java index eb36db653ab6..5e94df783d54 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.StartTestingClusterOption; @@ -281,13 +281,13 @@ public void testAsyncTable() throws Exception { public void testChangeMetaReplicaCount() throws Exception { Admin admin = TEST_UTIL.getAdmin(); try (RegionLocator locator = - TEST_UTIL.getConnection().getRegionLocator(connection.getMetaTableName())) { + TEST_UTIL.getConnection().getRegionLocator(TEST_UTIL.getConnection().getMetaTableName())) { assertEquals(1, locator.getAllRegionLocations().size()); - HBaseTestingUtil.setReplicas(admin, connection.getMetaTableName(), 3); + HBaseTestingUtil.setReplicas(admin, TEST_UTIL.getConnection().getMetaTableName(), 3); TEST_UTIL.waitFor(30000, () -> locator.getAllRegionLocations().size() == 3); - HBaseTestingUtil.setReplicas(admin, connection.getMetaTableName(), 2); + HBaseTestingUtil.setReplicas(admin, TEST_UTIL.getConnection().getMetaTableName(), 2); TEST_UTIL.waitFor(30000, () -> locator.getAllRegionLocations().size() == 2); - HBaseTestingUtil.setReplicas(admin, connection.getMetaTableName(), 1); + HBaseTestingUtil.setReplicas(admin, TEST_UTIL.getConnection().getMetaTableName(), 1); TEST_UTIL.waitFor(30000, () -> locator.getAllRegionLocations().size() == 1); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java index 6fa2167d9a2b..b0cc3dc20cf3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java @@ -20,7 +20,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -50,7 +50,7 @@ public void testShutdownOfReplicaHolder() throws Exception { // checks that the when the server holding meta replica is shut down, the meta replica // can be recovered try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); - RegionLocator locator = conn.getRegionLocator(connection.getMetaTableName())) { + RegionLocator locator = conn.getRegionLocator(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable())) { HRegionLocation hrl = locator.getRegionLocations(HConstants.EMPTY_START_ROW, true).get(1); ServerName oldServer = hrl.getServerName(); TEST_UTIL.getHBaseClusterInterface().killRegionServer(oldServer); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java index 45af730962ab..17eac52a7378 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java @@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.TableNotFoundException; @@ -164,7 +164,7 @@ public static void cleanupTest() throws Exception { */ @Test(expected = IllegalArgumentException.class) public void testMetaTablesSnapshot() throws Exception { - UTIL.getAdmin().snapshot("metaSnapshot", connection.getMetaTableName()); + UTIL.getAdmin().snapshot("metaSnapshot", TEST_UTIL.getConnection().getMetaTableName()); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java index 63bfacf4e43b..1095eb5159ae 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -64,7 +64,7 @@ public class TestZKConnectionRegistry { @BeforeClass public static void setUp() throws Exception { TEST_UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), connection.getMetaTableName(), 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TEST_UTIL.getConnection().getMetaTableName(), 3); REGISTRY = new ZKConnectionRegistry(TEST_UTIL.getConfiguration(), null); } @@ -89,7 +89,7 @@ public void test() throws InterruptedException, ExecutionException, IOException IntStream.range(0, 3).forEach(i -> { HRegionLocation loc = locs.getRegionLocation(i); assertNotNull("Replica " + i + " doesn't have location", loc); - assertEquals(connection.getMetaTableName(), loc.getRegion().getTable()); + assertEquals(TEST_UTIL.getConnection().getMetaTableName(), loc.getRegion().getTable()); assertEquals(i, loc.getRegion().getReplicaId()); }); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java index c96b29ff7a39..0fdc2a68591a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.LocalHBaseCluster; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; @@ -201,7 +201,7 @@ public void testTableActionsAvailableForAdmins() throws Exception { @Override public Void run() throws Exception { // Check the expected content is present in the http response - Pair pair = getTablePage(connection.getMetaTableName()); + Pair pair = getTablePage(TEST_UTIL.getConnection().getMetaTableName()); assertEquals(HttpURLConnection.HTTP_OK, pair.getFirst().intValue()); assertTrue("expected=" + expectedAuthorizedContent + ", content=" + pair.getSecond(), pair.getSecond().contains(expectedAuthorizedContent)); @@ -214,7 +214,7 @@ public Void run() throws Exception { nonAdmin.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - Pair pair = getTablePage(connection.getMetaTableName()); + Pair pair = getTablePage(TEST_UTIL.getConnection().getMetaTableName()); assertEquals(HttpURLConnection.HTTP_OK, pair.getFirst().intValue()); assertFalse( "should not find=" + expectedAuthorizedContent + ", content=" + pair.getSecond(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java index 14f4ed153709..6e6e5f81cb60 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.PleaseHoldException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; @@ -244,7 +244,7 @@ public void testFlushedSequenceIdPersistLoad() throws Exception { .setColumnFamily(ColumnFamilyDescriptorBuilder.of(Bytes.toBytes("cf"))).build(); Table table = TEST_UTIL.createTable(tableDescriptor, null); // flush META region - TEST_UTIL.flush(connection.getMetaTableName()); + TEST_UTIL.flush(TEST_UTIL.getConnection().getMetaTableName()); // wait for regionserver report Threads.sleep(msgInterval * 2); // record flush seqid before cluster shutdown diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java index 9f981b015da6..8cfceb28919e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java @@ -23,7 +23,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -64,7 +64,7 @@ public static void teardownTest() throws Exception { @Test public void tesMetaDescriptorHasSFTConfig() throws Exception { - TableDescriptor descriptor = UTIL.getAdmin().getDescriptor(connection.getMetaTableName()); + TableDescriptor descriptor = UTIL.getAdmin().getDescriptor(UTIL.getConnection().getMetaTableName()); assertEquals(FILE.name(), descriptor.getValue(TRACKER_IMPL)); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java index eeabc0e48818..7957502a96d3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java @@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; @@ -308,7 +308,7 @@ public void testIncompleteMetaTableReplicaInformation() throws Exception { ADMIN.disableTable(tableName); // now delete one replica info from all the rows // this is to make the meta appear to be only partially updated - Table metaTable = ADMIN.getConnection().getTable(connection.getMetaTableName()); + Table metaTable = ADMIN.getConnection().getMetaTable(); for (byte[] row : tableRows) { Delete deleteOneReplicaLocation = new Delete(row); deleteOneReplicaLocation.addColumns(HConstants.CATALOG_FAMILY, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java index 87bda4fbaf61..91398b310425 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java @@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AsyncTable; @@ -94,7 +94,7 @@ public void testNewCluster() throws Exception { Connection conn = TEST_UTIL.getConnection(); assertTrue(conn.getAdmin().isMasterInMaintenanceMode()); - try (Table table = conn.getTable(connection.getMetaTableName()); + try (Table table = conn.getTable(TEST_UTIL.getConnection().getMetaTableName()); ResultScanner scanner = table.getScanner(new Scan())) { assertNotNull("Could not read meta.", scanner.next()); } @@ -121,7 +121,7 @@ public void testExistingCluster() throws Exception { Connection conn = TEST_UTIL.getConnection(); assertTrue(conn.getAdmin().isMasterInMaintenanceMode()); - try (Table table = conn.getTable(connection.getMetaTableName()); + try (Table table = conn.getTable(TEST_UTIL.getConnection().getMetaTableName()); ResultScanner scanner = table.getScanner(HConstants.TABLE_FAMILY); Stream results = StreamSupport.stream(scanner.spliterator(), false)) { assertTrue("Did not find user table records while reading hbase:meta", diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java index 5b1fb7a0518d..0fe4b499ad4c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java @@ -22,7 +22,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; @@ -302,7 +302,7 @@ public void testKillRSWithOpeningRegion2482() throws Exception { */ private static int addToEachStartKey(final int expected) throws IOException { Table t = TEST_UTIL.getConnection().getTable(TABLENAME); - Table meta = TEST_UTIL.getConnection().getTable(connection.getMetaTableName()); + Table meta = TEST_UTIL.getConnection().getMetaTable(); int rows = 0; Scan scan = new Scan(); scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java index d98a57eea38f..8ddc6f2a52d4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java @@ -22,7 +22,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.client.Connection; @@ -66,7 +66,7 @@ public static void tearDownAfterClass() throws Exception { @Test public void testStopActiveMaster() throws Exception { try (Connection conn = ConnectionFactory.createConnection(UTIL.getConfiguration()); - RegionLocator locator = conn.getRegionLocator(connection.getMetaTableName())) { + RegionLocator locator = conn.getRegionLocator(TEST_UTIL.getConnection().getMetaTableName())) { ServerName oldMetaServer = locator.getAllRegionLocations().get(0).getServerName(); ServerName oldMaster = UTIL.getMiniHBaseCluster().getMaster().getServerName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java index 443ce15e8545..7f42783f3d28 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; @@ -68,7 +67,7 @@ public class TestMigrateAndMirrorMetaLocations { @BeforeClass public static void setUp() throws Exception { UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), connection.getMetaTableName(), 2); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), UTIL.getConnection().getMetaTableName(), 2); } @AfterClass @@ -143,20 +142,20 @@ public void test() throws Exception { } // wait until all meta regions have been assigned UTIL.waitFor(30000, - () -> UTIL.getMiniHBaseCluster().getRegions(connection.getMetaTableName()).size() == 2); + () -> UTIL.getMiniHBaseCluster().getRegions(UTIL.getConnection().getMetaTableName()).size() == 2); // make sure all the SCPs are finished waitUntilNoSCP(); checkMirrorLocation(2); // increase replica count to 3 - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), connection.getMetaTableName(), 3); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), UTIL.getConnection().getMetaTableName(), 3); checkMirrorLocation(3); byte[] replica2Data = ZKUtil.getData(UTIL.getZooKeeperWatcher(), UTIL.getZooKeeperWatcher().getZNodePaths().getZNodeForReplica(2)); // decrease replica count to 1 - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), connection.getMetaTableName(), 1); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), UTIL.getConnection().getMetaTableName(), 1); checkMirrorLocation(1); // restart the whole cluster, put an extra replica znode on zookeeper, to see if we will remove diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java index 72b9be888eda..bb4e9dd63fa2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; @@ -78,7 +78,7 @@ public static final class SuspendProcedure extends Procedure @Override public TableName getTableName() { - return connection.getMetaTableName(); + return TEST_UTIL.getConnection().getMetaTableName(); } @Override @@ -155,7 +155,7 @@ public static void tearDown() throws Exception { private void removeNamespaceFamily() throws IOException { FileSystem fs = UTIL.getTestFileSystem(); Path rootDir = CommonFSUtils.getRootDir(UTIL.getConfiguration()); - Path tableDir = CommonFSUtils.getTableDir(rootDir, connection.getMetaTableName()); + Path tableDir = CommonFSUtils.getTableDir(rootDir, TEST_UTIL.getConnection().getMetaTableName()); TableDescriptor metaTableDesc = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir); TableDescriptor noNsMetaTableDesc = TableDescriptorBuilder.newBuilder(metaTableDesc) .removeColumnFamily(HConstants.NAMESPACE_FAMILY).build(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java index 1d0d6f85cc6d..6e2c41648e91 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.StartTestingClusterOption; @@ -127,7 +127,7 @@ private void validateRecreateClusterWithUserTableEnabled(boolean cleanupWALs, private void restartHBaseCluster(boolean cleanUpWALs, boolean cleanUpZnodes) throws Exception { // flush cache so that everything is on disk - TEST_UTIL.getMiniHBaseCluster().flushcache(connection.getMetaTableName()); + TEST_UTIL.getMiniHBaseCluster().flushcache(TEST_UTIL.getConnection().getMetaTableName()); TEST_UTIL.getMiniHBaseCluster().flushcache(); List oldServers = @@ -178,7 +178,7 @@ private void prepareDataBeforeRecreate(HBaseTestingUtil testUtil, TableName tabl put.addColumn(Bytes.toBytes("f"), Bytes.toBytes("c"), Bytes.toBytes("v")); table.put(put); - ensureTableNotColocatedWithSystemTable(tableName, connection.getMetaTableName()); + ensureTableNotColocatedWithSystemTable(tableName, TEST_UTIL.getConnection().getMetaTableName()); } private void ensureTableNotColocatedWithSystemTable(TableName userTable, TableName systemTable) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java index 2c8ef58a7774..8aed78d71013 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java @@ -24,7 +24,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -81,7 +81,7 @@ public void testRestart() throws IOException, InterruptedException { table.put(new Put(row).addColumn(FAMILY, QUALIFIER, row)); } // flush all in memory data - UTIL.flush(connection.getMetaTableName()); + UTIL.flush(TEST_UTIL.getConnection().getMetaTableName()); UTIL.flush(NAME); // stop master first, so when stopping region server, we will not schedule a SCP. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java index cabc9bba5904..15a4047a911f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java @@ -21,7 +21,7 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.client.AsyncAdmin; import org.apache.hadoop.hbase.client.AsyncConnection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -64,13 +64,13 @@ public static void tearDown() throws Exception { public void test() throws Exception { RegionServerThread rsThread = null; for (RegionServerThread t : UTIL.getMiniHBaseCluster().getRegionServerThreads()) { - if (!t.getRegionServer().getRegions(connection.getMetaTableName()).isEmpty()) { + if (!t.getRegionServer().getRegions(TEST_UTIL.getConnection().getMetaTableName()).isEmpty()) { rsThread = t; break; } } HRegionServer rs = rsThread.getRegionServer(); - RegionInfo hri = rs.getRegions(connection.getMetaTableName()).get(0).getRegionInfo(); + RegionInfo hri = rs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).get(0).getRegionInfo(); HMaster master = UTIL.getMiniHBaseCluster().getMaster(); ProcedureExecutor executor = master.getMasterProcedureExecutor(); DummyRegionProcedure proc = new DummyRegionProcedure(executor.getEnvironment(), hri); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java index d52a7458743c..2a5b7e29a594 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java @@ -21,7 +21,7 @@ import java.util.concurrent.CountDownLatch; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ProcedureTestUtil; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -151,11 +151,11 @@ public static void setUp() throws Exception { UTIL.createTable(TABLE_NAME, CF); UTIL.getAdmin().balancerSwitch(false, true); HRegionServer srcRs = UTIL.getRSForFirstRegionInTable(TABLE_NAME); - if (!srcRs.getRegions(connection.getMetaTableName()).isEmpty()) { - RegionInfo metaRegion = srcRs.getRegions(connection.getMetaTableName()).get(0).getRegionInfo(); + if (!srcRs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).isEmpty()) { + RegionInfo metaRegion = srcRs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).get(0).getRegionInfo(); HRegionServer dstRs = UTIL.getOtherRegionServer(srcRs); UTIL.getAdmin().move(metaRegion.getEncodedNameAsBytes(), dstRs.getServerName()); - UTIL.waitFor(30000, () -> !dstRs.getRegions(connection.getMetaTableName()).isEmpty()); + UTIL.waitFor(30000, () -> !dstRs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).isEmpty()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java index 0a21143bb2e0..e5d2f55af6d4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java @@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ProcedureTestUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AsyncAdmin; @@ -95,7 +95,7 @@ public static void setUp() throws Exception { Configuration conf = UTIL.getConfiguration(); conf.setClass(HConstants.MASTER_IMPL, HMasterForTest.class, HMaster.class); UTIL.startMiniCluster(1); - UTIL.waitTableAvailable(connection.getMetaTableName()); + UTIL.waitTableAvailable(UTIL.getConnection().getMetaTableName()); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java index 7af43fd41336..b78a32ddc951 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; @@ -122,7 +122,7 @@ public void testVisitMetaForBadRegionState() throws Exception { put.addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER, Bytes.toBytes("BAD_STATE")); - try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { + try (Table table = UTIL.getConnection().getMetaTable()) { table.put(put); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java index 7f9f63883445..e006accb2303 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java @@ -29,7 +29,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; @@ -168,7 +168,7 @@ public void describeTo(Description description) { @Test public void testFailAndRollback() throws Exception { - HRegionServer rsWithMeta = UTIL.getRSForFirstRegionInTable(connection.getMetaTableName()); + HRegionServer rsWithMeta = UTIL.getRSForFirstRegionInTable(TEST_UTIL.getConnection().getMetaTableName()); UTIL.getMiniHBaseCluster().killRegionServer(rsWithMeta.getServerName()); UTIL.waitFor(15000, () -> getSCPForServer(rsWithMeta.getServerName()) != null); ServerCrashProcedure scp = getSCPForServer(rsWithMeta.getServerName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java index 2311d13c17f8..088567e24f0a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java @@ -29,7 +29,7 @@ import java.util.stream.Collectors; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -147,7 +147,7 @@ static void validateReplicaDistribution(Connection connection, TableName tableNa static void validateRegionLocations(Map> tableToServers, TableName productTableName, boolean shouldBeBalanced) { ServerName metaServer = - tableToServers.get(connection.getMetaTableName()).stream().findFirst().orElseThrow(); + tableToServers.get(TableName.valueOf("hbase", "meta")).stream().findFirst().orElseThrow(); ServerName quotaServer = tableToServers.get(QuotaUtil.QUOTA_TABLE_NAME).stream().findFirst().orElseThrow(); Set productServers = tableToServers.get(productTableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java index e2f063cdbec0..9ed292445c1a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; @@ -162,7 +162,7 @@ public void testRoundRobinAssignment() throws Exception { LoadBalancer balancer = master.getLoadBalancer(); List regions = admin.getRegions(tableName); - regions.addAll(admin.getRegions(connection.getMetaTableName())); + regions.addAll(admin.getRegions(TEST_UTIL.getConnection().getMetaTableName())); List servers = Lists.newArrayList( admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet()); Map> map = balancer.roundRobinAssignment(regions, servers); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java index c0833525c091..dd7927618df9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java @@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -94,7 +94,7 @@ public void testTableIsolation() throws Exception { BalancerConditionalsTestUtil.generateSplits(2 * NUM_SERVERS)); Set tablesToBeSeparated = - ImmutableSet. builder().add(connection.getMetaTableName()) + ImmutableSet. builder().add(TEST_UTIL.getConnection().getMetaTableName()) .add(QuotaUtil.QUOTA_TABLE_NAME).add(productTableName).build(); // Pause the balancer @@ -149,7 +149,7 @@ private static void validateRegionLocations(Map> tabl TableName productTableName, boolean shouldBeBalanced) { // Validate that the region assignments ServerName metaServer = - tableToServers.get(connection.getMetaTableName()).stream().findFirst().orElseThrow(); + tableToServers.get(TEST_UTIL.getConnection().getMetaTableName()).stream().findFirst().orElseThrow(); ServerName quotaServer = tableToServers.get(QuotaUtil.QUOTA_TABLE_NAME).stream().findFirst().orElseThrow(); Set productServers = tableToServers.get(productTableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java index 7824114cecce..d6e4b962f3bf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java @@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Delete; @@ -95,7 +95,7 @@ public static void tearDownAfterClass() throws Exception { @After public void tearDown() throws IOException { - try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName()); + try (Table table = UTIL.getConnection().getMetaTable(); ResultScanner scanner = table.getScanner(new Scan().addFamily(HConstants.CATALOG_FAMILY) .addFamily(HConstants.REPLICATION_BARRIER_FAMILY).setFilter(new FirstKeyOnlyFilter()))) { for (;;) { @@ -149,20 +149,20 @@ private void addBarrier(RegionInfo region, long... barriers) throws IOException put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER, put.getTimestamp() - barriers.length + i, Bytes.toBytes(barriers[i])); } - try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { + try (Table table = UTIL.getConnection().getMetaTable()) { table.put(put); } } private void fillCatalogFamily(RegionInfo region) throws IOException { - try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { + try (Table table = UTIL.getConnection().getMetaTable()) { table.put(new Put(region.getRegionName()).addColumn(HConstants.CATALOG_FAMILY, Bytes.toBytes("whatever"), Bytes.toBytes("whatever"))); } } private void clearCatalogFamily(RegionInfo region) throws IOException { - try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { + try (Table table = UTIL.getConnection().getMetaTable()) { table.delete(new Delete(region.getRegionName()).addFamily(HConstants.CATALOG_FAMILY)); } } @@ -282,7 +282,7 @@ public void testDeleteRowForDeletedRegion() throws IOException, ReplicationExcep // No catalog family, then we should remove the whole row clearCatalogFamily(region); cleaner.chore(); - try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { + try (Table table = UTIL.getConnection().getMetaTable()) { assertFalse(table .exists(new Get(region.getRegionName()).addFamily(HConstants.REPLICATION_BARRIER_FAMILY))); } @@ -304,7 +304,7 @@ public void testDeleteRowForDeletedRegionNoPeers() throws IOException { // There are no peers, and no catalog family for this region either, so we should remove the // barriers. And since there is no catalog family, after we delete the barrier family, the whole // row is deleted. - try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { + try (Table table = UTIL.getConnection().getMetaTable()) { assertFalse(table.exists(new Get(region.getRegionName()))); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java index 0bc050343e7e..406ac78e2c03 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java @@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaMockingUtil; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.Waiter; @@ -183,7 +183,7 @@ private PairOfSameType waitOnDaughters(final RegionInfo r) throws IO long start = EnvironmentEdgeManager.currentTime(); PairOfSameType pair = null; try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); - Table metaTable = conn.getTable(connection.getMetaTableName())) { + Table metaTable = conn.getTable(TEST_UTIL.getConnection().getMetaTableName())) { Result result = null; RegionInfo region = null; while ((EnvironmentEdgeManager.currentTime() - start) < 60000) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java index 8fc1d7cb6987..fab54d8d0d91 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java @@ -25,7 +25,7 @@ import java.util.List; import java.util.SortedSet; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.testclassification.MasterTests; @@ -50,21 +50,21 @@ public class TestMetaFixerNoCluster { private static byte[] D = Bytes.toBytes("d"); private static RegionInfo ALL = RegionInfoBuilder.FIRST_META_REGIONINFO; private static RegionInfo _ARI = - RegionInfoBuilder.newBuilder(connection.getMetaTableName()).setEndKey(A).build(); + RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setEndKey(A).build(); private static RegionInfo _BRI = - RegionInfoBuilder.newBuilder(connection.getMetaTableName()).setEndKey(B).build(); + RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setEndKey(B).build(); private static RegionInfo ABRI = - RegionInfoBuilder.newBuilder(connection.getMetaTableName()).setStartKey(A).setEndKey(B).build(); + RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(A).setEndKey(B).build(); private static RegionInfo ACRI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(connection.getMetaTableName()).setStartKey(A).setEndKey(C).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(A).setEndKey(C).build(); private static RegionInfo CDRI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(connection.getMetaTableName()).setStartKey(C).setEndKey(D).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(C).setEndKey(D).build(); private static RegionInfo ADRI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(connection.getMetaTableName()).setStartKey(A).setEndKey(D).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(A).setEndKey(D).build(); private static RegionInfo D_RI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(connection.getMetaTableName()).setStartKey(D).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(D).build(); private static RegionInfo C_RI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(connection.getMetaTableName()).setStartKey(C).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(C).build(); @Test public void testGetRegionInfoWithLargestEndKey() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java index eb97f2f2a1f3..27a75cc08d23 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java @@ -56,7 +56,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.RegionMetrics; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.Size; @@ -105,7 +104,7 @@ public void before() { @Test public void testNoNormalizationForMetaTable() { - TableName testTable = connection.getMetaTableName(); + TableName testTable = masterServices.getConnection().getMetaTableName(); TableDescriptor testMetaTd = TableDescriptorBuilder.newBuilder(testTable).build(); List RegionInfo = new ArrayList<>(); Map regionSizes = new HashMap<>(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java index 73d467cd8ecc..d71948aff71d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java @@ -34,9 +34,9 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.Result; @@ -262,7 +262,7 @@ public String toString() { private static class PrimaryNotMetaRegionSelector extends RegionSelector { @Override boolean regionFilter(final RegionInfo info) { - return !Objects.equals(connection.getMetaTableName(), info.getTable()) + return !Objects.equals(TableName.valueOf("hbase:meta"), info.getTable()) && Objects.equals(RegionInfo.DEFAULT_REPLICA_ID, info.getReplicaId()); } @@ -278,7 +278,7 @@ Exception regionFilterFailure() { private static class ReplicaNonMetaRegionSelector extends RegionSelector { @Override boolean regionFilter(RegionInfo info) { - return !Objects.equals(connection.getMetaTableName(), info.getTable()) + return !Objects.equals(TableName.valueOf("hbase:meta"), info.getTable()) && !Objects.equals(RegionInfo.DEFAULT_REPLICA_ID, info.getReplicaId()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java index 314ff36368a1..c650dd6511f6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java @@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.client.AsyncAdmin; @@ -148,7 +148,13 @@ public static void tearDown() throws Exception { @Test public void test() throws Exception { RegionServerThread rsWithMetaThread = UTIL.getMiniHBaseCluster().getRegionServerThreads() - .stream().filter(t -> !t.getRegionServer().getRegions(connection.getMetaTableName()).isEmpty()) + .stream().filter(t -> { + try { + return !t.getRegionServer().getRegions(UTIL.getConnection().getMetaTableName()).isEmpty(); + } catch (IOException e) { + throw new RuntimeException(e); + } + }) .findAny().get(); HRegionServer rsNoMeta = UTIL.getOtherRegionServer(rsWithMetaThread.getRegionServer()); FAIL = true; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java index 13a30cf559de..abeacfbc4f46 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java @@ -22,7 +22,7 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; @@ -123,7 +123,7 @@ public static class MetaTableProcedure extends Procedure @Override public TableName getTableName() { - return connection.getMetaTableName(); + return TEST_UTIL.getConnection().getMetaTableName(); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableWithMasterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableWithMasterFailover.java index e14075176169..6b269db477a0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableWithMasterFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableWithMasterFailover.java @@ -64,7 +64,7 @@ private void testTruncateWithFailoverAtStep(final boolean preserveSplits, final RegionInfo[] regions = MasterProcedureTestingUtility.createTable(getMasterProcedureExecutor(), tableName, splitKeys, families); // load and verify that there are rows in the table - MasterProcedureTestingUtility.loadData(UTIL.getConnection(), tableName, 100, splitKeys, + MasterProcedureTestingUtility.loadData(TEST_UTIL.getConnection(), tableName, 100, splitKeys, families); assertEquals(100, UTIL.countRows(tableName)); // disable the table @@ -95,7 +95,7 @@ private void testTruncateWithFailoverAtStep(final boolean preserveSplits, final assertEquals(0, UTIL.countRows(tableName)); // verify that the table is read/writable - MasterProcedureTestingUtility.loadData(UTIL.getConnection(), tableName, 50, splitKeys, + MasterProcedureTestingUtility.loadData(TEST_UTIL.getConnection(), tableName, 50, splitKeys, families); assertEquals(50, UTIL.countRows(tableName)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java index e2518790e4ae..dbf3c6011fea 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java @@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.YouAreDeadException; @@ -132,7 +132,7 @@ public void test() throws Exception { HRegionServer regionSvr = UTIL.getRSForFirstRegionInTable(TABLE_NAME); HRegion region = regionSvr.getRegions(TABLE_NAME).get(0); String regName = region.getRegionInfo().getEncodedName(); - List metaRegs = regionSvr.getRegions(connection.getMetaTableName()); + List metaRegs = regionSvr.getRegions(TEST_UTIL.getConnection().getMetaTableName()); if (metaRegs != null && !metaRegs.isEmpty()) { LOG.info("meta is on the same server: " + regionSvr); // when region is on same server as hbase:meta, reassigning meta would abort the server diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java index e639cd3a1ed0..e979fe1201fa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java @@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueTestUtil; import org.apache.hadoop.hbase.KeyValueUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -1003,7 +1003,7 @@ public void testShouldFlushMeta() throws Exception { TableDescriptors tds = new FSTableDescriptors(conf); FSTableDescriptors.tryUpdateMetaTableDescriptor(conf); HRegion meta = HRegion.createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, testDir, conf, - tds.get(connection.getMetaTableName()), + tds.get(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()), wFactory.getWAL(RegionInfoBuilder.FIRST_META_REGIONINFO)); // parameterized tests add [#] suffix get rid of [ and ]. TableDescriptor desc = TableDescriptorBuilder diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java index 89f33e8d4b22..7eae181ea0f7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java @@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.Stoppable; @@ -430,7 +430,7 @@ public static void blockUntilRegionSplit(Configuration conf, long timeout, log("blocking until region is split:" + Bytes.toStringBinary(regionName)); RegionInfo daughterA = null, daughterB = null; try (Connection conn = ConnectionFactory.createConnection(conf); - Table metaTable = conn.getTable(connection.getMetaTableName())) { + Table metaTable = conn.getTable(TEST_UTIL.getConnection().getMetaTableName())) { Result result = null; RegionInfo region = null; while ((EnvironmentEdgeManager.currentTime() - start) < timeout) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java index ed110896c03f..a5b7a583c793 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java @@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Delete; @@ -93,7 +93,7 @@ public void testUsingMetaAndBinary() throws IOException { // Up flush size else we bind up when we use default catalog flush of 16k. TableDescriptors tds = new FSTableDescriptors(UTIL.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(UTIL.getConfiguration()); - TableDescriptor td = tds.get(connection.getMetaTableName()); + TableDescriptor td = tds.get(UTIL.getConnection().getMetaTableName()); td = TableDescriptorBuilder.newBuilder(td).setMemStoreFlushSize(64 * 1024 * 1024).build(); HRegion mr = HBaseTestingUtil.createRegionAndWAL(RegionInfoBuilder.FIRST_META_REGIONINFO, rootdir, conf, td); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java index 907b666b74fd..1f3232dccffb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtil; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -73,12 +73,12 @@ public void testReadAndWriteRegionInfoFile() throws IOException, InterruptedExce FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(FS, ROOT_DIR); FSTableDescriptors.tryUpdateAndGetMetaTableDescriptor(CONF, FS, ROOT_DIR); HRegion r = HBaseTestingUtil.createRegionAndWAL(ri, ROOT_DIR, CONF, - fsTableDescriptors.get(connection.getMetaTableName())); + fsTableDescriptors.get(ri.getTable())); // Get modtime on the file. long modtime = getModTime(r); HBaseTestingUtil.closeRegionAndWAL(r); Thread.sleep(1001); - r = HRegion.openHRegion(ROOT_DIR, ri, fsTableDescriptors.get(connection.getMetaTableName()), null, + r = HRegion.openHRegion(ROOT_DIR, ri, fsTableDescriptors.get(ri.getTable()), null, CONF); // Ensure the file is not written for a second time. long modtime2 = getModTime(r); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java index 563d8c3ba7ed..3f8c2f13886f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java @@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -72,7 +72,7 @@ public class TestRegionInfo { public void testIsStart() { assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isFirst()); org.apache.hadoop.hbase.client.RegionInfo ri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(connection.getMetaTableName()).setStartKey(Bytes.toBytes("not_start")).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(Bytes.toBytes("not_start")).build(); assertFalse(ri.isFirst()); } @@ -80,7 +80,7 @@ public void testIsStart() { public void testIsEnd() { assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isFirst()); org.apache.hadoop.hbase.client.RegionInfo ri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(connection.getMetaTableName()).setEndKey(Bytes.toBytes("not_end")).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setEndKey(Bytes.toBytes("not_end")).build(); assertFalse(ri.isLast()); } @@ -88,9 +88,9 @@ public void testIsEnd() { public void testIsNext() { byte[] bytes = Bytes.toBytes("row"); org.apache.hadoop.hbase.client.RegionInfo ri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(connection.getMetaTableName()).setEndKey(bytes).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setEndKey(bytes).build(); org.apache.hadoop.hbase.client.RegionInfo ri2 = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(connection.getMetaTableName()).setStartKey(bytes).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(bytes).build(); assertFalse(ri.isNext(RegionInfoBuilder.FIRST_META_REGIONINFO)); assertTrue(ri.isNext(ri2)); } @@ -103,18 +103,18 @@ public void testIsOverlap() { byte[] d = Bytes.toBytes("d"); org.apache.hadoop.hbase.client.RegionInfo all = RegionInfoBuilder.FIRST_META_REGIONINFO; org.apache.hadoop.hbase.client.RegionInfo ari = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(connection.getMetaTableName()).setEndKey(a).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setEndKey(a).build(); org.apache.hadoop.hbase.client.RegionInfo abri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(connection.getMetaTableName()) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()) .setStartKey(a).setEndKey(b).build(); org.apache.hadoop.hbase.client.RegionInfo adri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(connection.getMetaTableName()) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()) .setStartKey(a).setEndKey(d).build(); org.apache.hadoop.hbase.client.RegionInfo cdri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(connection.getMetaTableName()) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()) .setStartKey(c).setEndKey(d).build(); org.apache.hadoop.hbase.client.RegionInfo dri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(connection.getMetaTableName()).setStartKey(d).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(d).build(); assertTrue(all.isOverlap(all)); assertTrue(all.isOverlap(abri)); assertFalse(abri.isOverlap(cdri)); @@ -141,17 +141,17 @@ public void testIsOverlaps() { byte[] e = Bytes.toBytes("e"); byte[] f = Bytes.toBytes("f"); org.apache.hadoop.hbase.client.RegionInfo ari = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(connection.getMetaTableName()).setEndKey(a).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setEndKey(a).build(); org.apache.hadoop.hbase.client.RegionInfo abri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(connection.getMetaTableName()) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()) .setStartKey(a).setEndKey(b).build(); org.apache.hadoop.hbase.client.RegionInfo eri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(connection.getMetaTableName()).setEndKey(e).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setEndKey(e).build(); org.apache.hadoop.hbase.client.RegionInfo cdri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(connection.getMetaTableName()) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()) .setStartKey(c).setEndKey(d).build(); org.apache.hadoop.hbase.client.RegionInfo efri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(connection.getMetaTableName()) + org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()) .setStartKey(e).setEndKey(f).build(); assertFalse(ari.isOverlap(abri)); assertTrue(abri.isOverlap(eri)); @@ -176,12 +176,12 @@ public void testReadAndWriteHRegionInfoFile() throws IOException, InterruptedExc FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(htu.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(htu.getConfiguration()); HRegion r = HBaseTestingUtil.createRegionAndWAL(hri, basedir, htu.getConfiguration(), - fsTableDescriptors.get(connection.getMetaTableName())); + fsTableDescriptors.get(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable())); // Get modtime on the file. long modtime = getModTime(r); HBaseTestingUtil.closeRegionAndWAL(r); Thread.sleep(1001); - r = HRegion.openHRegion(basedir, hri, fsTableDescriptors.get(connection.getMetaTableName()), null, + r = HRegion.openHRegion(basedir, hri, fsTableDescriptors.get(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()), null, htu.getConfiguration()); // Ensure the file is not written for a second time. long modtime2 = getModTime(r); @@ -255,7 +255,7 @@ public void testContainsRange() { @Test public void testContainsRangeForMetaTable() { TableDescriptor tableDesc = - TableDescriptorBuilder.newBuilder(connection.getMetaTableName()).build(); + TableDescriptorBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).build(); RegionInfo hri = RegionInfoBuilder.newBuilder(tableDesc.getTableName()).build(); byte[] startRow = HConstants.EMPTY_START_ROW; byte[] row1 = Bytes.toBytes("a,a,0"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java index c3f6f5bc16f9..77a2bb71e988 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TestMetaTableAccessor; import org.apache.hadoop.hbase.client.Consistency; @@ -148,7 +148,7 @@ public void testRegionReplicaUpdatesMetaLocation() throws Exception { openRegion(HTU, getRS(), hriSecondary); Table meta = null; try { - meta = HTU.getConnection().getTable(connection.getMetaTableName()); + meta = HTU.getConnection().getMetaTable(); TestMetaTableAccessor.assertMetaLocation(meta, hriPrimary.getRegionName(), getRS().getServerName(), -1, 1, false); } finally { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java index cd1bb0866036..1faa97c70f95 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java @@ -22,7 +22,7 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -66,7 +66,7 @@ public static void setUp() throws Exception { UTIL.createTable(TABLE_NAME, CF); UTIL.waitTableAvailable(TABLE_NAME); HRegionServer rs = UTIL.getRSForFirstRegionInTable(TABLE_NAME); - if (!rs.getRegions(connection.getMetaTableName()).isEmpty()) { + if (!rs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).isEmpty()) { HRegionServer rs1 = UTIL.getOtherRegionServer(rs); UTIL.moveRegionAndWait( UTIL.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0).getRegionInfo(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java index 8da716fe6a9a..22b4d025ce38 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java @@ -23,7 +23,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -98,12 +98,12 @@ public static void stopMasterAndCacheMetaLocation(HBaseTestingUtil HTU) // cache meta location, so we will not go to master to lookup meta region location for (JVMClusterUtil.RegionServerThread t : HTU.getMiniHBaseCluster().getRegionServerThreads()) { try (RegionLocator locator = - t.getRegionServer().getConnection().getRegionLocator(connection.getMetaTableName())) { + t.getRegionServer().getConnection().getRegionLocator(HTU.getConnection().getMetaTableName())) { locator.getAllRegionLocations(); } } try ( - RegionLocator locator = HTU.getConnection().getRegionLocator(connection.getMetaTableName())) { + RegionLocator locator = HTU.getConnection().getRegionLocator(HTU.getConnection().getMetaTableName())) { locator.getAllRegionLocations(); } // Stop master diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java index f3be3bcd9192..97532aadae54 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; @@ -136,7 +136,7 @@ public void testRejectRequestsOnAbort() throws Exception { .getRegionServerThreads()) { HRegionServer regionServer = regionServerThread.getRegionServer(); if ( - regionServer.getRegions(connection.getMetaTableName()).isEmpty() + regionServer.getRegions(UTIL.getConnection().getMetaTableName()).isEmpty() && !regionServer.getRegions(TABLE_NAME).isEmpty() ) { serverWithoutMeta = regionServer; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java index 49037259d09a..baff35b8ecbb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java @@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.YouAreDeadException; @@ -136,7 +136,7 @@ public void test() throws Exception { RegionServerThread rst1 = UTIL.getMiniHBaseCluster().getRegionServerThreads().get(1); HRegionServer liveRS; RegionServerThread toKillRSThread; - if (rst1.getRegionServer().getRegions(connection.getMetaTableName()).isEmpty()) { + if (rst1.getRegionServer().getRegions(TEST_UTIL.getConnection().getMetaTableName()).isEmpty()) { liveRS = rst0.getRegionServer(); toKillRSThread = rst1; } else { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java index 859708769050..c3750ba66e39 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java @@ -31,7 +31,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.StartTestingClusterOption; @@ -339,7 +339,7 @@ void validateData(Table table, int rownum) throws IOException { public void testCompactionRecordDoesntBlockRolling() throws Exception { // When the hbase:meta table can be opened, the region servers are running - try (Table t = TEST_UTIL.getConnection().getTable(connection.getMetaTableName()); + try (Table t = TEST_UTIL.getConnection().getMetaTable(); Table table = createTestTable(getName())) { server = TEST_UTIL.getRSForFirstRegionInTable(table.getName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java index c1c2585774d3..1d1ed79fbdf6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; @@ -163,7 +163,7 @@ public void testRSAbortWithUnflushedEdits() throws Exception { LOG.info("Starting testRSAbortWithUnflushedEdits()"); // When the hbase:meta table can be opened, the region servers are running - TEST_UTIL.getConnection().getTable(connection.getMetaTableName()).close(); + TEST_UTIL.getConnection().getMetaTable().close(); // Create the test table and open it TableName tableName = TableName.valueOf(this.getClass().getSimpleName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java index 97c3354f35ee..48764dc2213c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java @@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; @@ -323,7 +323,7 @@ public void testLogRollOnPipelineRestart() throws Exception { fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS()) > 1); LOG.info("Replication=" + fs.getDefaultReplication(TEST_UTIL.getDataTestDirOnTestFS())); // When the hbase:meta table can be opened, the region servers are running - Table t = TEST_UTIL.getConnection().getTable(connection.getMetaTableName()); + Table t = TEST_UTIL.getConnection().getMetaTable(); try { this.server = cluster.getRegionServer(0); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java index c849b57378a0..91c65907bfa9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java @@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -95,7 +95,7 @@ public void testContendedLogRolling() throws Exception { CommonFSUtils.setRootDir(conf, dir); FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(TEST_UTIL.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration()); - TableDescriptor metaTableDescriptor = fsTableDescriptors.get(connection.getMetaTableName()); + TableDescriptor metaTableDescriptor = fsTableDescriptors.get(TEST_UTIL.getConnection().getMetaTableName()); conf.set(FSHLogProvider.WRITER_IMPL, HighLatencySyncWriter.class.getName()); final WALFactory wals = new WALFactory(conf, TestLogRollingNoCluster.class.getName()); final WAL wal = wals.getWAL(null); @@ -159,7 +159,7 @@ public void run() { try { TableDescriptors tds = new FSTableDescriptors(TEST_UTIL.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration()); - TableDescriptor htd = tds.get(connection.getMetaTableName()); + TableDescriptor htd = tds.get(TEST_UTIL.getConnection().getMetaTableName()); for (int i = 0; i < this.count; i++) { long now = EnvironmentEdgeManager.currentTime(); // Roll every ten edits @@ -176,7 +176,7 @@ public void run() { scopes.put(fam, 0); } final long txid = wal.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), - connection.getMetaTableName(), now, mvcc, scopes), edit); + TEST_UTIL.getConnection().getMetaTableName(), now, mvcc, scopes), edit); Threads.sleep(ThreadLocalRandom.current().nextInt(5)); wal.sync(txid); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java index 5e764ebb0ef5..9cf4db32343e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java @@ -94,7 +94,7 @@ public String explainFailure() throws Exception { @Test public void testAddPeer() throws Exception { TableName tableName = createTable(); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -103,7 +103,7 @@ public void testAddPeer() throws Exception { HRegionServer rs = UTIL.getOtherRegionServer(UTIL.getRSForFirstRegionInTable(tableName)); moveRegionAndArchiveOldWals(region, rs); addPeer(true); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -120,7 +120,7 @@ public void testChangeToSerial() throws Exception { UTIL.getAdmin().addReplicationPeer(PEER_ID, peerConfig, true); TableName tableName = createTable(); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -142,7 +142,7 @@ public void testChangeToSerial() throws Exception { ReplicationPeerConfig.newBuilder(peerConfig).setSerial(true).build()); UTIL.getAdmin().enableReplicationPeer(PEER_ID); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -160,7 +160,7 @@ public void testAddToSerialPeer() throws Exception { UTIL.getAdmin().addReplicationPeer(PEER_ID, peerConfig, true); TableName tableName = createTable(); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -183,7 +183,7 @@ public void testAddToSerialPeer() throws Exception { ReplicationPeerConfig.newBuilder(peerConfig) .setTableCFsMap(ImmutableMap.of(tableName, Collections.emptyList())).build()); UTIL.getAdmin().enableReplicationPeer(PEER_ID); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -195,7 +195,7 @@ public void testAddToSerialPeer() throws Exception { @Test public void testDisabledTable() throws Exception { TableName tableName = createTable(); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -204,7 +204,7 @@ public void testDisabledTable() throws Exception { rollAllWALs(); addPeer(true); UTIL.getAdmin().enableTable(tableName); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -216,7 +216,7 @@ public void testDisabledTable() throws Exception { @Test public void testDisablingTable() throws Exception { TableName tableName = createTable(); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -239,7 +239,7 @@ public void testDisablingTable() throws Exception { tsm.setTableState(tableName, TableState.State.DISABLED); t.join(); UTIL.getAdmin().enableTable(tableName); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -251,7 +251,7 @@ public void testDisablingTable() throws Exception { @Test public void testEnablingTable() throws Exception { TableName tableName = createTable(); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -274,7 +274,7 @@ public void testEnablingTable() throws Exception { assertTrue(t.isAlive()); tsm.setTableState(tableName, TableState.State.ENABLED); t.join(); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestRemoveFromSerialReplicationPeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestRemoveFromSerialReplicationPeer.java index 23f4dca33447..2dc52eb90a3a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestRemoveFromSerialReplicationPeer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestRemoveFromSerialReplicationPeer.java @@ -80,7 +80,7 @@ public void testRemoveTable() throws Exception { .setReplicateAllUserTables(false) .setTableCFsMap(ImmutableMap.of(tableName, Collections.emptyList())).setSerial(true).build(); UTIL.getAdmin().addReplicationPeer(PEER_ID, peerConfig, true); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -101,7 +101,7 @@ public void testRemoveTable() throws Exception { public void testRemoveSerialFlag() throws Exception { TableName tableName = createTable(); addPeer(true); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java index fe09c1c6aa83..79e40861f7ed 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.security.access.PermissionStorage; @@ -73,7 +73,7 @@ public void testSystemTableWALEntryFilter() { // meta WALKeyImpl key1 = new WALKeyImpl(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(), - connection.getMetaTableName(), EnvironmentEdgeManager.currentTime()); + RegionInfoBuilder.FIRST_META_REGIONINFO.getTable(), EnvironmentEdgeManager.currentTime()); Entry metaEntry = new Entry(key1, null); assertNull(filter.filter(metaEntry)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java index 7ab82b60cec9..b1ae3e57b1fb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java @@ -66,7 +66,7 @@ public void setUp() throws IOException, StreamLacksCapabilityException { @Test public void testRegionMove() throws Exception { TableName tableName = createTable(); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -74,7 +74,7 @@ public void testRegionMove() throws Exception { RegionInfo region = UTIL.getAdmin().getRegions(tableName).get(0); HRegionServer rs = UTIL.getOtherRegionServer(UTIL.getRSForFirstRegionInTable(tableName)); moveRegion(region, rs); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 100; i < 200; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -86,7 +86,7 @@ public void testRegionMove() throws Exception { @Test public void testRegionSplit() throws Exception { TableName tableName = createTable(); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -98,7 +98,7 @@ public void testRegionSplit() throws Exception { UTIL.waitUntilNoRegionsInTransition(30000); List regions = UTIL.getAdmin().getRegions(tableName); assertEquals(2, regions.size()); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -145,7 +145,7 @@ public void testRegionMerge() throws Exception { .build(), new byte[][] { splitKey }); UTIL.waitTableAvailable(tableName); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -158,7 +158,7 @@ public void testRegionMerge() throws Exception { UTIL.waitUntilNoRegionsInTransition(30000); List regionsAfterMerge = UTIL.getAdmin().getRegions(tableName); assertEquals(1, regionsAfterMerge.size()); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -212,7 +212,7 @@ public void testRemovePeerNothingReplicated() throws Exception { @Test public void testRemovePeer() throws Exception { TableName tableName = createTable(); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -232,7 +232,7 @@ public void testRemovePeer() throws Exception { @Test public void testRemoveSerialFlag() throws Exception { TableName tableName = createTable(); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplicationFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplicationFailover.java index 1295ea14abcd..9a0df321029a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplicationFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplicationFailover.java @@ -56,7 +56,7 @@ public void testKillRS() throws Exception { TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build()); UTIL.waitTableAvailable(tableName); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -65,7 +65,7 @@ public void testKillRS() throws Exception { .filter(t -> !t.getRegionServer().getRegions(tableName).isEmpty()).findFirst().get(); thread.getRegionServer().abort("for testing"); thread.join(); - try (Table table = UTIL.getConnection().getTable(tableName)) { + try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { for (int i = 100; i < 200; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java index 07247363689e..a50241c93e25 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; @@ -99,10 +99,10 @@ public void before() throws Exception { // conf.setInt(HConstants.META_REPLICAS_NUM, numOfMetaReplica); HTU.startMiniCluster(NB_SERVERS); // Enable hbase:meta replication. - HBaseTestingUtil.setReplicas(HTU.getAdmin(), connection.getMetaTableName(), numOfMetaReplica); + HBaseTestingUtil.setReplicas(HTU.getAdmin(), HTU.getConnection().getMetaTableName(), numOfMetaReplica); HTU.waitFor(30000, - () -> HTU.getMiniHBaseCluster().getRegions(connection.getMetaTableName()).size() + () -> HTU.getMiniHBaseCluster().getRegions(HTU.getConnection().getMetaTableName()).size() >= numOfMetaReplica); } @@ -120,37 +120,37 @@ public void testHBaseMetaReplicates() throws Exception { try (Table table = HTU.createTable(TableName.valueOf(this.name.getMethodName() + "_0"), HConstants.CATALOG_FAMILY, Arrays.copyOfRange(HBaseTestingUtil.KEYS, 1, HBaseTestingUtil.KEYS.length))) { - verifyReplication(connection.getMetaTableName(), numOfMetaReplica, + verifyReplication(HTU.getConnection().getMetaTableName(), numOfMetaReplica, getMetaCells(table.getName())); } try (Table table = HTU.createTable(TableName.valueOf(this.name.getMethodName() + "_1"), HConstants.CATALOG_FAMILY, Arrays.copyOfRange(HBaseTestingUtil.KEYS, 1, HBaseTestingUtil.KEYS.length))) { - verifyReplication(connection.getMetaTableName(), numOfMetaReplica, + verifyReplication(HTU.getConnection().getMetaTableName(), numOfMetaReplica, getMetaCells(table.getName())); // Try delete. HTU.deleteTableIfAny(table.getName()); - verifyDeletedReplication(connection.getMetaTableName(), numOfMetaReplica, table.getName()); + verifyDeletedReplication(HTU.getConnection().getMetaTableName(), numOfMetaReplica, table.getName()); } } @Test public void testCatalogReplicaReplicationWithFlushAndCompaction() throws Exception { try (Connection connection = ConnectionFactory.createConnection(HTU.getConfiguration()); - Table table = connection.getTable(connection.getMetaTableName())) { + Table table = connection.getTable(HTU.getConnection().getMetaTableName())) { // load the data to the table for (int i = 0; i < 5; i++) { LOG.info("Writing data from " + i * 1000 + " to " + (i * 1000 + 1000)); HTU.loadNumericRows(table, HConstants.CATALOG_FAMILY, i * 1000, i * 1000 + 1000); LOG.info("flushing table"); - HTU.flush(connection.getMetaTableName()); + HTU.flush(HTU.getConnection().getMetaTableName()); LOG.info("compacting table"); if (i < 4) { - HTU.compact(connection.getMetaTableName(), false); + HTU.compact(HTU.getConnection().getMetaTableName(), false); } } - verifyReplication(connection.getMetaTableName(), numOfMetaReplica, 0, 5000, + verifyReplication(HTU.getConnection().getMetaTableName(), numOfMetaReplica, 0, 5000, HConstants.CATALOG_FAMILY); } } @@ -185,7 +185,7 @@ public void testCatalogReplicaReplicationWithReplicaMoved() throws Exception { } } try (Connection connection = ConnectionFactory.createConnection(HTU.getConfiguration()); - Table table = connection.getTable(connection.getMetaTableName())) { + Table table = connection.getTable(HTU.getConnection().getMetaTableName())) { // load the data to the table for (int i = 0; i < 5; i++) { LOG.info("Writing data from " + i * 1000 + " to " + (i * 1000 + 1000)); @@ -195,7 +195,7 @@ public void testCatalogReplicaReplicationWithReplicaMoved() throws Exception { } } - verifyReplication(connection.getMetaTableName(), numOfMetaReplica, 0, 5000, + verifyReplication(HTU.getConnection().getMetaTableName(), numOfMetaReplica, 0, 5000, HConstants.CATALOG_FAMILY); } } @@ -417,7 +417,7 @@ private void getMetaReplicaReadRequests(final Region[] metaRegions, final long[] @Test public void testHBaseMetaReplicaGets() throws Exception { TableName tn = TableName.valueOf(this.name.getMethodName()); - final Region[] metaRegions = getAllRegions(connection.getMetaTableName(), numOfMetaReplica); + final Region[] metaRegions = getAllRegions(HTU.getConnection().getMetaTableName(), numOfMetaReplica); long[] readReqsForMetaReplicas = new long[numOfMetaReplica]; long[] readReqsForMetaReplicasAfterGet = new long[numOfMetaReplica]; long[] readReqsForMetaReplicasAfterGetAllLocations = new long[numOfMetaReplica]; @@ -430,7 +430,7 @@ public void testHBaseMetaReplicaGets() throws Exception { try (Table table = HTU.createTable(tn, HConstants.CATALOG_FAMILY, Arrays.copyOfRange(HBaseTestingUtil.KEYS, 1, HBaseTestingUtil.KEYS.length))) { - verifyReplication(connection.getMetaTableName(), numOfMetaReplica, + verifyReplication(HTU.getConnection().getMetaTableName(), numOfMetaReplica, getMetaCells(table.getName())); // load different values HTU.loadTable(table, new byte[][] { HConstants.CATALOG_FAMILY }, VALUE); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java index cfd430fdbc76..1da006fc0055 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java @@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; @@ -197,7 +197,7 @@ public void testWALEntryFilter() throws IOException { assertTrue(wef.filter(e) == e); // Test system WAL edit. e = new WAL.Entry( - new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, connection.getMetaTableName(), -1, -1, uuid), we); + new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, TEST_UTIL.getConnection().getMetaTableName(), -1, -1, uuid), we); assertNull(wef.filter(e)); } finally { rs.terminate("Done"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java index 65edfcc07dda..31e433aac380 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; @@ -163,7 +163,7 @@ private void addStateAndBarrier(RegionInfo region, RegionState.State state, long put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER, put.getTimestamp() - barriers.length + i, Bytes.toBytes(barriers[i])); } - try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { + try (Table table = UTIL.getConnection().getMetaTable()) { table.put(put); } } @@ -172,7 +172,7 @@ private void setState(RegionInfo region, RegionState.State state) throws IOExcep Put put = new Put(region.getRegionName(), EnvironmentEdgeManager.currentTime()); put.addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER, Bytes.toBytes(state.name())); - try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { + try (Table table = UTIL.getConnection().getMetaTable()) { table.put(put); } } @@ -189,7 +189,7 @@ private void addParents(RegionInfo region, List parents) throws IOEx put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, ReplicationBarrierFamilyFormat.REPLICATION_PARENT_QUALIFIER, ReplicationBarrierFamilyFormat.getParentsBytes(parents)); - try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { + try (Table table = UTIL.getConnection().getMetaTable()) { table.put(put); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsCPHookCalled.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsCPHookCalled.java index 8288784c522a..f861948211e3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsCPHookCalled.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsCPHookCalled.java @@ -20,7 +20,6 @@ import static org.junit.Assert.assertTrue; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -69,7 +68,7 @@ public void testGetRSGroupInfoCPHookCalled() throws Exception { @Test public void testGetRSGroupInfoOfTableCPHookCalled() throws Exception { - ADMIN.getRSGroup(connection.getMetaTableName()); + ADMIN.getRSGroup(TEST_UTIL.getConnection().getMetaTableName()); assertTrue(OBSERVER.preGetRSGroupInfoOfTableCalled); assertTrue(OBSERVER.postGetRSGroupInfoOfTableCalled); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java index 883fb8d3a843..398fd317f415 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java @@ -29,7 +29,7 @@ import java.util.Set; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -247,10 +247,10 @@ public void testLowerMetaGroupVersion() throws Exception { // move hbase:meta to meta_group Set toAddTables = new HashSet<>(); - toAddTables.add(connection.getMetaTableName()); + toAddTables.add(TEST_UTIL.getConnection().getMetaTableName()); ADMIN.setRSGroup(toAddTables, groupName); assertTrue(ADMIN.getConfiguredNamespacesAndTablesInRSGroup(groupName).getSecond() - .contains(connection.getMetaTableName())); + .contains(TEST_UTIL.getConnection().getMetaTableName())); // restart the regionserver in meta_group, and lower its version String originVersion = ""; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java index fb1ec46740f3..69d2c63985c7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -237,7 +237,7 @@ public void testRunCleanerChore() throws Exception { public void testExecProcedure() throws Exception { verifyAdminCheckForAction((admin) -> { // Using existing table instead of creating a new one. - admin.execProcedure("flush-table-proc", connection.getMetaTableName().getNameAsString(), + admin.execProcedure("flush-table-proc", TEST_UTIL.getConnection().getMetaTableName().getNameAsString(), new HashMap<>()); }); } @@ -261,7 +261,7 @@ public void testExecProcedureWithRet() throws Exception { verifyAdminCheckForAction((admin) -> { // Using existing table instead of creating a new one. admin.execProcedureWithReturn("flush-table-proc", - connection.getMetaTableName().getNameAsString(), new HashMap<>()); + TEST_UTIL.getConnection().getMetaTableName().getNameAsString(), new HashMap<>()); }); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java index 6f973320a1eb..0dc9c76914f6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java @@ -29,7 +29,7 @@ import java.util.Collection; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.client.AsyncConnection; import org.apache.hadoop.hbase.client.AsyncTable; import org.apache.hadoop.hbase.client.Connection; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java index 98b042849644..c77e6afad56f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java @@ -33,7 +33,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; @@ -87,7 +87,7 @@ public static void setupBeforeClass() throws Exception { TEST_UTIL.startMiniCluster(1); TEST_UTIL.getHBaseCluster().waitForActiveAndReadyMaster(); - TEST_UTIL.waitUntilAllRegionsAssigned(connection.getMetaTableName()); + TEST_UTIL.waitUntilAllRegionsAssigned(TEST_UTIL.getConnection().getMetaTableName()); rootDir = CommonFSUtils.getRootDir(conf); fs = TEST_UTIL.getTestFileSystem(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java index 22ac5fc57658..3e3784835f81 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java @@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -200,7 +200,7 @@ protected void deleteRegion(Configuration conf, final TableDescriptor htd, byte[ if (metaRow) { try ( - Table meta = connection.getTable(connection.getMetaTableName(), tableExecutorService)) { + Table meta = connection.getTable(TEST_UTIL.getConnection().getMetaTableName(), tableExecutorService)) { Delete delete = new Delete(deleteRow); meta.delete(delete); } @@ -512,7 +512,7 @@ public boolean tableHasErrors(HbckTableInfo table) { protected void deleteMetaRegion(Configuration conf, boolean unassign, boolean hdfs, boolean regionInfoOnly) throws IOException, InterruptedException { - HRegionLocation metaLocation = connection.getRegionLocator(connection.getMetaTableName()) + HRegionLocation metaLocation = connection.getRegionLocator(TEST_UTIL.getConnection().getMetaTableName()) .getRegionLocation(HConstants.EMPTY_START_ROW); ServerName hsa = metaLocation.getServerName(); RegionInfo hri = metaLocation.getRegion(); @@ -527,7 +527,7 @@ protected void deleteMetaRegion(Configuration conf, boolean unassign, boolean hd LOG.info("deleting hdfs .regioninfo data: " + hri.toString() + hsa.toString()); Path rootDir = CommonFSUtils.getRootDir(conf); FileSystem fs = rootDir.getFileSystem(conf); - Path p = new Path(rootDir + "/" + connection.getMetaTableName().getNameAsString(), + Path p = new Path(rootDir + "/" + TEST_UTIL.getConnection().getMetaTableName().getNameAsString(), hri.getEncodedName()); Path hriPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE); fs.delete(hriPath, true); @@ -537,7 +537,7 @@ protected void deleteMetaRegion(Configuration conf, boolean unassign, boolean hd LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString()); Path rootDir = CommonFSUtils.getRootDir(conf); FileSystem fs = rootDir.getFileSystem(conf); - Path p = new Path(rootDir + "/" + connection.getMetaTableName().getNameAsString(), + Path p = new Path(rootDir + "/" + TEST_UTIL.getConnection().getMetaTableName().getNameAsString(), hri.getEncodedName()); HBaseFsck.debugLsr(conf, p); boolean success = fs.delete(p, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java index d9795110d029..a158ebdfc4e4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -281,7 +281,7 @@ public void testGetAll() throws IOException, InterruptedException { } // add hbase:meta htds.createTableDescriptor( - TableDescriptorBuilder.newBuilder(connection.getMetaTableName()).build()); + TableDescriptorBuilder.newBuilder(TableName.valueOf("hbase", "meta")).build()); assertEquals("getAll() didn't return all TableDescriptors, expected: " + (count + 1) + " got: " + htds.getAll().size(), count + 1, htds.getAll().size()); } @@ -300,7 +300,7 @@ public void testParallelGetAll() throws IOException, InterruptedException { } // add hbase:meta htds.createTableDescriptor( - TableDescriptorBuilder.newBuilder(connection.getMetaTableName()).build()); + TableDescriptorBuilder.newBuilder(TableName.valueOf("hbase", "meta")).build()); int getTableDescriptorSize = htds.getAll().size(); assertEquals("getAll() didn't return all TableDescriptors, expected: " + (count + 1) + " got: " @@ -327,7 +327,7 @@ public void testGetAllOrdering() throws Exception { // Remove hbase:meta from list. It shows up now since we made it dynamic. The schema // is written into the fs by the FSTableDescriptors constructor now where before it // didn't. - tables.remove(connection.getMetaTableName().getNameAsString()); + tables.remove(TableName.valueOf("hbase", "meta").getNameAsString()); assertEquals(4, tables.size()); String[] tableNamesOrdered = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java index a6590e982fcc..ad43a13eef49 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java @@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; @@ -199,7 +199,7 @@ private void addStateAndBarrier(RegionInfo region, RegionState.State state, long put.addColumn(HConstants.REPLICATION_BARRIER_FAMILY, HConstants.SEQNUM_QUALIFIER, put.getTimestamp() - barriers.length + i, Bytes.toBytes(barriers[i])); } - try (Table table = UTIL.getConnection().getTable(connection.getMetaTableName())) { + try (Table table = UTIL.getConnection().getMetaTable()) { table.put(put); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java index 9f72f45b73e2..349decee59d2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckMOB.java @@ -27,7 +27,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.io.hfile.TestHFile; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java index 0267fe2051c3..28d982076fc7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java @@ -30,7 +30,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; @@ -288,7 +288,7 @@ public void testRegionServerPort() throws Exception { public void testLoadMetaRegion() throws Exception { HRegionServer rsWithMeta = TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads().stream() .map(t -> t.getRegionServer()) - .filter(rs -> rs.getRegions(connection.getMetaTableName()).size() > 0).findFirst().get(); + .filter(rs -> rs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() > 0).findFirst().get(); int onlineRegions = rsWithMeta.getNumberOfOnlineRegions(); String rsName = rsWithMeta.getServerName().getAddress().toString(); try (RegionMover rm = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java index 81c0196dfbf9..4977f5912259 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java @@ -26,7 +26,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -87,7 +87,7 @@ public void setUp() throws Exception { // Remove rs contains hbase:meta, otherwise test looks unstable and buggy in test env. ServerName rsContainMeta = TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads().stream() .map(t -> t.getRegionServer()) - .filter(rs -> rs.getRegions(connection.getMetaTableName()).size() > 0).findFirst().get() + .filter(rs -> rs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() > 0).findFirst().get() .getServerName(); LOG.info("{} contains hbase:meta", rsContainMeta); List modifiable = new ArrayList<>(allServers); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java index baf86fdc39ce..5d7c06e80e8d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java @@ -59,7 +59,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -457,7 +457,7 @@ public void testRecoveredEditsPathForMeta() throws IOException { @Test public void testOldRecoveredEditsFileSidelined() throws IOException { Path p = createRecoveredEditsPathForRegion(); - Path tdir = CommonFSUtils.getTableDir(HBASEDIR, connection.getMetaTableName()); + Path tdir = CommonFSUtils.getTableDir(HBASEDIR, TEST_UTIL.getConnection().getMetaTableName()); Path regiondir = new Path(tdir, RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName()); fs.mkdirs(regiondir); Path parent = WALSplitUtil.getRegionDirRecoveredEditsDir(regiondir); @@ -470,7 +470,7 @@ public void testOldRecoveredEditsFileSidelined() throws IOException { private Path createRecoveredEditsPathForRegion() throws IOException { byte[] encoded = RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(); - Path p = WALSplitUtil.getRegionSplitEditsPath(connection.getMetaTableName(), encoded, 1, + Path p = WALSplitUtil.getRegionSplitEditsPath(TEST_UTIL.getConnection().getMetaTableName(), encoded, 1, FILENAME_BEING_SPLIT, TMPDIRNAME, conf, ""); return p; } diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java index 19aa72487d5c..cd5466d18754 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java @@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; @@ -1074,10 +1074,12 @@ public TRegionInfo getRegionInfo(ByteBuffer searchRow) throws IOError { try { byte[] row = getBytes(searchRow); Result startRowResult = - getReverseScanResult(MetaTableName.getInstance().getName(), row, HConstants.CATALOG_FAMILY); + // TODO(HBASE-XXXXX - Phase 6): Get dynamic name from connection + // For now, hardcode default. Future: use getConnection(user).getMetaTableName().getName() + getReverseScanResult(TableName.valueOf("hbase", "meta").getName(), row, HConstants.CATALOG_FAMILY); if (startRowResult == null) { - throw new IOException("Cannot find row in " + MetaTableName.getInstance() + ", row=" + throw new IOException("Cannot find row in hbase:meta, row=" + Bytes.toStringBinary(row)); } diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java index a9fe61d1d4b8..f44a3de32c71 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java @@ -19,9 +19,10 @@ import com.google.errorprone.annotations.RestrictedApi; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.master.RegionState; @@ -167,11 +168,11 @@ public static void setMetaLocation(ZKWatcher zookeeper, ServerName serverName, i RegionState.State state) throws KeeperException { if (serverName == null) { LOG.warn("Tried to set null ServerName in {}; skipping -- ServerName required", - MetaTableName.getInstance()); + TableName.valueOf("hbase", "meta")); return; } LOG.info("Setting {} replicaId={} location in ZooKeeper as {}, state={}", - MetaTableName.getInstance(), replicaId, serverName, state); + TableName.valueOf("hbase", "meta"), replicaId, serverName, state); // Make the MetaRegionServer pb and then get its bytes and save this as // the znode content. MetaRegionServer pbrsr = @@ -182,10 +183,10 @@ public static void setMetaLocation(ZKWatcher zookeeper, ServerName serverName, i ZKUtil.setData(zookeeper, zookeeper.getZNodePaths().getZNodeForReplica(replicaId), data); } catch (KeeperException.NoNodeException nne) { if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) { - LOG.debug("{} region location doesn't exist, create it", MetaTableName.getInstance()); + LOG.debug("{} region location doesn't exist, create it", TableName.valueOf("hbase", "meta")); } else { LOG.debug("{} region location doesn't exist for replicaId={}, create it", - MetaTableName.getInstance(), replicaId); + TableName.valueOf("hbase", "meta"), replicaId); } ZKUtil.createAndWatch(zookeeper, zookeeper.getZNodePaths().getZNodeForReplica(replicaId), data); @@ -235,9 +236,9 @@ public static void deleteMetaLocation(ZKWatcher zookeeper) throws KeeperExceptio public static void deleteMetaLocation(ZKWatcher zookeeper, int replicaId) throws KeeperException { if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) { - LOG.info("Deleting {} region location in ZooKeeper", MetaTableName.getInstance()); + LOG.info("Deleting {} region location in ZooKeeper", TableName.valueOf("hbase", "meta")); } else { - LOG.info("Deleting {} for {} region location in ZooKeeper", MetaTableName.getInstance(), + LOG.info("Deleting {} for {} region location in ZooKeeper", TableName.valueOf("hbase", "meta"), replicaId); } try { diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java index f5f250ff1354..64023d0efe86 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java @@ -32,7 +32,8 @@ import java.util.LinkedList; import java.util.List; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MetaTableName; + +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; @@ -75,7 +76,7 @@ public static String dump(final ZKWatcher zkWatcher) { sb.append("\n ").append(child); } } - sb.append("\nRegion server holding ").append(MetaTableName.getInstance()).append(":"); + sb.append("\nRegion server holding ").append(TableName.valueOf("hbase", "meta")).append(":"); sb.append("\n ").append(MetaTableLocator.getMetaRegionLocation(zkWatcher)); int numMetaReplicas = zkWatcher.getMetaReplicaNodes().size(); for (int i = 1; i < numMetaReplicas; i++) { From 2f046702df091f0a182c53ac842ef9f5ec5e1fd3 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Tue, 20 Jan 2026 11:11:15 -0500 Subject: [PATCH 10/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: Add meta table name in master region --- .../apache/hadoop/hbase/master/HMaster.java | 46 ++++++ .../hbase/master/MetaTableNameStore.java | 139 ++++++++++++++++++ .../master/procedure/InitMetaProcedure.java | 23 ++- 3 files changed, 205 insertions(+), 3 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 3238a3d9cc8d..52432697eaf5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -510,6 +510,12 @@ public class HMaster extends HBaseServerBase implements Maste */ private ReplicationPeerModificationStateStore replicationPeerModificationStateStore; + /** + * Store for the meta table name in the Master Local Region. + * This provides cluster-specific storage for dynamic meta table name discovery. + */ + private MetaTableNameStore metaTableNameStore; + /** * Initializes the HMaster. The steps are as follows: *

@@ -1017,6 +1023,19 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE masterRegion = MasterRegionFactory.create(this); rsListStorage = new MasterRegionServerList(masterRegion, this); + // Initialize meta table name store after master region is ready + metaTableNameStore = new MetaTableNameStore(masterRegion); + try { + if (metaTableNameStore.isStored()) { + TableName metaName = metaTableNameStore.load(); + LOG.info("Loaded meta table name from Master Local Region: {}", metaName); + } else { + LOG.info("Meta table name not yet stored (will be set during InitMetaProcedure)"); + } + } catch (IOException e) { + LOG.warn("Failed to load meta table name from Master Local Region, will use default", e); + } + // Initialize the ServerManager and register it as a configuration observer this.serverManager = createServerManager(this, rsListStorage); this.configurationManager.registerObserver(this.serverManager); @@ -1657,6 +1676,33 @@ public TableStateManager getTableStateManager() { return tableStateManager; } + /** + * Override base implementation to read from Master Local Region storage. + * This allows the master to return the cluster-specific meta table name. + */ + @Override + public TableName getMetaTableName() { + if (metaTableNameStore != null) { + try { + return metaTableNameStore.load(); + } catch (IOException e) { + LOG.warn("Failed to load meta table name from Master Local Region, using default", e); + } + } + // Fallback to base implementation (returns default "hbase:meta") + return super.getMetaTableName(); + } + + /** + * Internal accessor for procedures to get the meta table name store. + * This is not exposed via MasterServices interface to avoid interface pollution. + * Package visibility is insufficient as procedures are in a sub-package. + * @return the meta table name store, or null if not yet initialized + */ + public MetaTableNameStore getMetaTableNameStoreInternal() { + return metaTableNameStore; + } + /* * Start up all services. If any of these threads gets an unhandled exception then they just die * with a logged message. This should be fine because in general, we do not expect the master to diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java new file mode 100644 index 000000000000..473f9130646d --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java @@ -0,0 +1,139 @@ +/* + * 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 + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.master; + +import java.io.IOException; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.master.region.MasterRegion; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Stores and retrieves the meta table name for this cluster in the Master Local Region. + *

+ * This provides cluster-specific storage for the meta table name, which is essential for + * read replica clusters where each cluster needs its own meta table name. + *

+ * Currently stores the default "hbase:meta" to establish the storage pattern. + * Future enhancements will add replica-specific naming based on configuration. + */ +@InterfaceAudience.Private +public class MetaTableNameStore { + private static final Logger LOG = LoggerFactory.getLogger(MetaTableNameStore.class); + + // Storage keys for Master Local Region + private static final byte[] META_TABLE_NAME_ROW = Bytes.toBytes("meta_table_name"); + private static final byte[] INFO_FAMILY = Bytes.toBytes("info"); + private static final byte[] NAME_QUALIFIER = Bytes.toBytes("name"); + + private final MasterRegion masterRegion; + + // Cache to avoid repeated reads from Master Local Region + private volatile TableName cachedMetaTableName; + + public MetaTableNameStore(MasterRegion masterRegion) { + this.masterRegion = masterRegion; + } + + /** + * Store the meta table name in the Master Local Region. + *

+ * This should be called once during cluster initialization (InitMetaProcedure). + * The stored value is cluster-specific and will not conflict with other clusters + * sharing the same HDFS. + * @param metaTableName the meta table name to store + * @throws IOException if the storage operation fails + */ + public void store(TableName metaTableName) throws IOException { + LOG.info("Storing meta table name in Master Local Region: {}", metaTableName); + + Put put = new Put(META_TABLE_NAME_ROW); + put.addColumn(INFO_FAMILY, NAME_QUALIFIER, + Bytes.toBytes(metaTableName.getNameAsString())); + masterRegion.update(r -> r.put(put)); + + // Update cache + cachedMetaTableName = metaTableName; + + LOG.info("Successfully stored meta table name: {}", metaTableName); + } + + /** + * Load the meta table name from the Master Local Region. + *

+ * Returns the cached value if available, otherwise reads from Master Local Region. + * If no stored value is found (e.g., first bootstrap before InitMetaProcedure runs), + * returns the default "hbase:meta". + * @return the meta table name for this cluster + * @throws IOException if the load operation fails + */ + public TableName load() throws IOException { + // Return cached value if available + if (cachedMetaTableName != null) { + return cachedMetaTableName; + } + + synchronized (this) { + // Double-check after acquiring lock + if (cachedMetaTableName != null) { + return cachedMetaTableName; + } + + // Read from Master Local Region + Get get = new Get(META_TABLE_NAME_ROW); + get.addColumn(INFO_FAMILY, NAME_QUALIFIER); + Result result = masterRegion.get(get); + + if (!result.isEmpty()) { + byte[] value = result.getValue(INFO_FAMILY, NAME_QUALIFIER); + cachedMetaTableName = TableName.valueOf(Bytes.toString(value)); + LOG.debug("Loaded meta table name from Master Local Region: {}", cachedMetaTableName); + return cachedMetaTableName; + } + return cachedMetaTableName; + } + } + + /** + * Check if a meta table name has been stored in the Master Local Region. + * @return true if a meta table name is stored, false otherwise + * @throws IOException if the check operation fails + */ + public boolean isStored() throws IOException { + Get get = new Get(META_TABLE_NAME_ROW); + get.addColumn(INFO_FAMILY, NAME_QUALIFIER); + Result result = masterRegion.get(get); + return !result.isEmpty(); + } + + /** + * Clear the cached meta table name. + *

+ * This forces the next call to {@link #load()} to read from Master Local Region. + * Primarily used for testing. + */ + void clearCache() { + cachedMetaTableName = null; + } +} + diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java index 045249e746d5..ad544eb20ab9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java @@ -35,6 +35,8 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.io.hfile.HFile; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.master.MetaTableNameStore; import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; @@ -68,9 +70,9 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure Date: Thu, 22 Jan 2026 10:45:52 -0500 Subject: [PATCH 11/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: Format code part 1 --- .../SnapshotOfRegionAssignmentFromMeta.java | 1 - ...rgeClusterBalancingMetaTableIsolation.java | 4 +-- ...gTableIsolationAndReplicaDistribution.java | 3 +- .../AbstractRpcBasedConnectionRegistry.java | 1 - .../hadoop/hbase/client/AsyncConnection.java | 6 ++-- .../hadoop/hbase/client/Connection.java | 4 --- .../hbase/client/ConnectionRegistry.java | 5 +-- .../hbase/client/RegionInfoBuilder.java | 3 +- .../client/DoNothingConnectionRegistry.java | 1 - .../client/TestAsyncAdminRpcPriority.java | 2 +- .../client/TestAsyncConnectionTracing.java | 4 +-- .../client/TestAsyncRegionLocatorTracing.java | 3 +- .../client/TestAsyncTableRpcPriority.java | 2 +- .../hbase/client/TestAsyncTableTracing.java | 2 +- .../hbase/client/TestMetricsConnection.java | 6 ++-- .../hadoop/hbase/CellComparatorImpl.java | 7 +--- .../hbase/InnerStoreCellComparator.java | 7 +--- .../org/apache/hadoop/hbase/TableName.java | 3 +- .../hbase/master/MetaTableNameStore.java | 35 +++---------------- .../master/RegionPlacementMaintainer.java | 1 - .../client/TestAsyncNonMetaRegionLocator.java | 2 +- ...ncNonMetaRegionLocatorConcurrenyLimit.java | 2 +- .../hbase/client/TestAsyncRegionLocator.java | 2 +- ...stAsyncSingleRequestRpcRetryingCaller.java | 4 +-- ...talogReplicaLoadBalanceSimpleSelector.java | 2 +- .../hbase/client/TestFallbackToUseReplay.java | 2 +- ...TestTableProcedureWaitingQueueCleanup.java | 6 +++- .../TestRegionServerCrashDisableWAL.java | 2 +- .../TestShutdownWhileWALBroken.java | 2 +- .../TestAddToSerialReplicationPeer.java | 24 ++++++------- .../TestRemoveFromSerialReplicationPeer.java | 4 +-- 31 files changed, 52 insertions(+), 100 deletions(-) diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java index 8cf62ac41398..2a2512b066a3 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/SnapshotOfRegionAssignmentFromMeta.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.CatalogFamilyFormat; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java index b12df5a1fb54..34d0f9331291 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java @@ -71,9 +71,8 @@ public static void setup() { // Create regions List allRegions = new ArrayList<>(); for (int i = 0; i < NUM_REGIONS; i++) { - // TODO(HBASE-XXXXX - Phase 6): Make configurable for replica testing TableName tableName = i < 3 ? - TableName.valueOf("hbase", "meta") : NON_META_TABLE_NAME; + TableName.valueOf("hbase:meta") : NON_META_TABLE_NAME; byte[] startKey = new byte[1]; startKey[0] = (byte) i; byte[] endKey = new byte[1]; @@ -101,7 +100,6 @@ public void testMetaTableIsolation() { } private boolean isMetaTableIsolated(BalancerClusterState cluster) { - // TODO(HBASE-XXXXX - Phase 6): Make configurable for replica testing return isTableIsolated(cluster, TableName.valueOf("hbase", "meta"), "Meta"); } diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java index 61d2ac2efabe..a4ca304e0409 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java @@ -75,7 +75,6 @@ public static void setup() { for (int i = 0; i < NUM_REGIONS; i++) { TableName tableName; if (i < 1) { - // TODO(HBASE-XXXXX - Phase 6): Make configurable for replica testing tableName = TableName.valueOf("hbase", "meta"); } else if (i < 10) { tableName = SYSTEM_TABLE_NAME; @@ -121,7 +120,7 @@ public void testTableIsolationAndReplicaDistribution() { * Validates whether all meta table regions are isolated. */ private boolean isMetaTableIsolated(BalancerClusterState cluster) { - // TODO(HBASE-XXXXX - Phase 6): Make configurable for replica testing + return isTableIsolated(cluster, TableName.valueOf("hbase", "meta"), "Meta"); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java index c4778056c6c3..dc598c4be40f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java @@ -262,7 +262,6 @@ public CompletableFuture getMetaTableName() { (c, s, d) -> s.getMetaTableName(c, GetMetaTableNameRequest.getDefaultInstance(), d), GetMetaTableNameResponse::hasTableName, "getMetaTableName()") .thenApply(resp -> { - // If the response has a table name, parse it. Otherwise, return default. if (resp.hasTableName() && !resp.getTableName().isEmpty()) { return TableName.valueOf(resp.getTableName()); } else { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java index 79087d5f5294..bcdd5b01ec5d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java @@ -48,8 +48,6 @@ public interface AsyncConnection extends Closeable { * lifetime of this connection. For most clusters, this will be "hbase:meta". For read replica * clusters or other specialized configurations, this may return a different table name. *

- * This method should be used instead of static references to meta table name to ensure - * compatibility with clusters that use custom meta table names. * @return The meta table name for this cluster */ TableName getMetaTableName(); @@ -118,8 +116,8 @@ default AsyncTable getTable(TableName tableName, ExecutorSer } /** - * Retrieve an AsyncTable implementation for accessing the meta table. - * This method returns the correct meta table for this connection (hbase:meta or hbase:meta_suffix). + * Retrieve an {@link AsyncTable} implementation for accessing the meta table. + * This method returns the correct meta table for this connection * @return An AsyncTable to use for interactions with the meta table */ default AsyncTable getMetaTable() { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java index 107d488f4c4c..e6a936f9882e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java @@ -69,8 +69,6 @@ public interface Connection extends Abortable, Closeable { * lifetime of this connection. For most clusters, this will be "hbase:meta". For read replica * clusters or other specialized configurations, this may return a different table name. *

- * This method should be used instead of static references to meta table name to ensure - * compatibility with clusters that use custom meta table names. * @return The meta table name for this cluster */ TableName getMetaTableName(); @@ -111,8 +109,6 @@ default Table getTable(TableName tableName, ExecutorService pool) throws IOExcep /** * Retrieve a Table implementation for accessing the meta table. * This method returns the correct meta table for this connection (hbase:meta or hbase:meta_suffix). - * The returned Table is not thread safe, a new instance should be created for each using thread. - * This is a lightweight operation, pooling or caching of the returned Table is neither required nor desired. * @return A Table to use for interactions with the meta table */ default Table getMetaTable() throws IOException { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java index a15778fbcb3e..b80f05ba2081 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionRegistry.java @@ -52,11 +52,8 @@ public interface ConnectionRegistry extends Closeable { /** * Get the name of the meta table for this cluster. *

- * Should only be called once, similar to {@link #getClusterId()}. The upper layer should cache - * this value as it will not change during the connection lifetime. + * Should only be called once, similar to {@link #getClusterId()}. *

- * For most clusters, this will return the default "hbase:meta". For read replica clusters or - * other specialized configurations, this may return a different table name. * @return CompletableFuture containing the meta table name */ CompletableFuture getMetaTableName(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java index 30b564cff2af..2ca992d24f09 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java @@ -43,8 +43,7 @@ public class RegionInfoBuilder { */ // TODO: How come Meta regions still do not have encoded region names? Fix. // hbase:meta,,1.1588230740 should be the hbase:meta first region name. - // TODO(HBASE-XXXXX - Phase 6): Make this configurable for read replica clusters - // For now, hardcode to default. Future: lazy initialization based on config + // TODO: For now, hardcode to default. Future: lazy initialization based on config or make it use conenction public static final RegionInfo FIRST_META_REGIONINFO = new MutableRegionInfo(1L, TableName.valueOf("hbase", "meta"), RegionInfo.DEFAULT_REPLICA_ID); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java index 7f87e7e29b72..1be0c0bc9cf0 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java @@ -51,7 +51,6 @@ public CompletableFuture getActiveMaster() { @Override public CompletableFuture getMetaTableName() { return CompletableFuture.completedFuture(null); - } @Override diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java index ea17a187ad5d..833cb39076e8 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java @@ -145,7 +145,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { any()); User user = UserProvider.instantiate(CONF).getCurrent(); conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF, user), "test", - org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, user) { + TableName.valueOf("hbase:meta"), null, user) { @Override CompletableFuture getMasterStub() { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java index 24e30098984f..df27e49ee23a 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; @@ -72,8 +73,7 @@ public CompletableFuture getActiveMaster() { return CompletableFuture.completedFuture(masterServer); } }; - conn = new AsyncConnectionImpl(CONF, registry, "test", - org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, user); + conn = new AsyncConnectionImpl(CONF, registry, "test", TableName.valueOf("hbase:meta"), null, user); } @After diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java index 53ae717275d9..0b4ef2a1ebde 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; @@ -100,7 +101,7 @@ public void setUp() throws IOException { public CompletableFuture getMetaRegionLocations() { return CompletableFuture.completedFuture(locs); } - }, "test", org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, user); + }, "test", TableName.valueOf("hbase:meta"), null, user); } @After diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java index 13ad29312f2c..175f5f446e5a 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java @@ -164,7 +164,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(stub).get(any(HBaseRpcController.class), any(GetRequest.class), any()); User user = UserProvider.instantiate(CONF).getCurrent(); conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF, user), "test", - org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, user) { + TableName.valueOf("hbase:meta"), null, user) { @Override AsyncRegionLocator getLocator() { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java index 6a1fc45737fd..b1ffab020fbe 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java @@ -210,7 +210,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(stub).get(any(HBaseRpcController.class), any(GetRequest.class), any()); final User user = UserProvider.instantiate(CONF).getCurrent(); conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF, user), "test", - org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, user) { + TableName.valueOf("hbase:meta"), null, user) { @Override AsyncRegionLocator getLocator() { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java index 40ec35649e4e..d8d0aea73f95 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java @@ -107,14 +107,14 @@ public void testMetricsConnectionScope() throws IOException { conf.setBoolean(MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY, true); AsyncConnectionImpl impl = new AsyncConnectionImpl(conf, null, "foo", - org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()); + TableName.valueOf("hbase:meta"), null, User.getCurrent()); Optional metrics = impl.getConnectionMetrics(); assertTrue("Metrics should be present", metrics.isPresent()); assertEquals(clusterId + "@" + Integer.toHexString(impl.hashCode()), metrics.get().getMetricScope()); conf.set(MetricsConnection.METRICS_SCOPE_KEY, scope); impl = new AsyncConnectionImpl(conf, null, "foo", - org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()); + TableName.valueOf("hbase:meta"), null, User.getCurrent()); metrics = impl.getConnectionMetrics(); assertTrue("Metrics should be present", metrics.isPresent()); @@ -135,7 +135,7 @@ public void testMetricsWithMultiConnections() throws IOException { List connList = new ArrayList(); for (int i = 0; i < num; i++) { impl = new AsyncConnectionImpl(conf, null, null, - org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, user); + TableName.valueOf("hbase:meta"), null, user); connList.add(impl); } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java index 6a85f13406e9..15c1cd204f84 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java @@ -794,11 +794,6 @@ public static CellComparator getCellComparator(byte[] tableName) { } private static boolean isMetaTable(byte[] tableName) { - // Quick check: meta tables are in hbase namespace and start with "meta" - return tableName != null && tableName.length > 11 && - tableName[0] == 'h' && tableName[1] == 'b' && tableName[2] == 'a' && - tableName[3] == 's' && tableName[4] == 'e' && tableName[5] == ':' && - tableName[6] == 'm' && tableName[7] == 'e' && tableName[8] == 't' && - tableName[9] == 'a'; + return Bytes.startsWith(tableName, Bytes.toBytes("hbase:meta")); } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java index 29fdc51adffc..88a1ec2561c7 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java @@ -82,11 +82,6 @@ public static CellComparator getInnerStoreCellComparator(byte[] tableName) { } private static boolean isMetaTable(byte[] tableName) { - // Quick check: meta tables are in hbase namespace and start with "meta" - return tableName != null && tableName.length > 11 && - tableName[0] == 'h' && tableName[1] == 'b' && tableName[2] == 'a' && - tableName[3] == 's' && tableName[4] == 'e' && tableName[5] == ':' && - tableName[6] == 'm' && tableName[7] == 'e' && tableName[8] == 't' && - tableName[9] == 'a'; + return Bytes.startsWith(tableName, Bytes.toBytes("hbase:meta")); } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java index 658263ac747b..e2cd2242ceb0 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java @@ -46,7 +46,6 @@ *

*/ @InterfaceAudience.Public -@InterfaceStability.Stable public final class TableName implements Comparable { private static final Logger LOG = LoggerFactory.getLogger(TableName.class); @@ -304,7 +303,7 @@ private TableName(ByteBuffer namespace, ByteBuffer qualifier) throws IllegalArgu } if (qualifierAsString.equals(OLD_META_STR)) { throw new IllegalArgumentException(OLD_META_STR + " no longer exists. The table has been " - + "renamed to hbase:meta"); + + "renamed to hbase:meta or hbase:meta_suffix in conf"); } if (Bytes.equals(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME, namespace)) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java index 473f9130646d..524a64d79bb8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java @@ -31,24 +31,17 @@ /** * Stores and retrieves the meta table name for this cluster in the Master Local Region. *

- * This provides cluster-specific storage for the meta table name, which is essential for - * read replica clusters where each cluster needs its own meta table name. + * This provides cluster-specific storage for the meta table name. *

- * Currently stores the default "hbase:meta" to establish the storage pattern. - * Future enhancements will add replica-specific naming based on configuration. */ @InterfaceAudience.Private public class MetaTableNameStore { private static final Logger LOG = LoggerFactory.getLogger(MetaTableNameStore.class); - - // Storage keys for Master Local Region private static final byte[] META_TABLE_NAME_ROW = Bytes.toBytes("meta_table_name"); private static final byte[] INFO_FAMILY = Bytes.toBytes("info"); private static final byte[] NAME_QUALIFIER = Bytes.toBytes("name"); private final MasterRegion masterRegion; - - // Cache to avoid repeated reads from Master Local Region private volatile TableName cachedMetaTableName; public MetaTableNameStore(MasterRegion masterRegion) { @@ -59,47 +52,36 @@ public MetaTableNameStore(MasterRegion masterRegion) { * Store the meta table name in the Master Local Region. *

* This should be called once during cluster initialization (InitMetaProcedure). - * The stored value is cluster-specific and will not conflict with other clusters + * The stored value is cluster-specific and should not conflict with other clusters * sharing the same HDFS. * @param metaTableName the meta table name to store * @throws IOException if the storage operation fails */ public void store(TableName metaTableName) throws IOException { LOG.info("Storing meta table name in Master Local Region: {}", metaTableName); - Put put = new Put(META_TABLE_NAME_ROW); put.addColumn(INFO_FAMILY, NAME_QUALIFIER, Bytes.toBytes(metaTableName.getNameAsString())); masterRegion.update(r -> r.put(put)); - - // Update cache cachedMetaTableName = metaTableName; - LOG.info("Successfully stored meta table name: {}", metaTableName); } /** * Load the meta table name from the Master Local Region. *

- * Returns the cached value if available, otherwise reads from Master Local Region. - * If no stored value is found (e.g., first bootstrap before InitMetaProcedure runs), - * returns the default "hbase:meta". * @return the meta table name for this cluster * @throws IOException if the load operation fails */ public TableName load() throws IOException { - // Return cached value if available if (cachedMetaTableName != null) { return cachedMetaTableName; } synchronized (this) { - // Double-check after acquiring lock if (cachedMetaTableName != null) { return cachedMetaTableName; } - - // Read from Master Local Region Get get = new Get(META_TABLE_NAME_ROW); get.addColumn(INFO_FAMILY, NAME_QUALIFIER); Result result = masterRegion.get(get); @@ -110,6 +92,9 @@ public TableName load() throws IOException { LOG.debug("Loaded meta table name from Master Local Region: {}", cachedMetaTableName); return cachedMetaTableName; } + cachedMetaTableName = TableName.valueOf("hbase", "meta"); + LOG.info("No stored meta table name found in Master Local Region, using default: {}", + cachedMetaTableName); return cachedMetaTableName; } } @@ -125,15 +110,5 @@ public boolean isStored() throws IOException { Result result = masterRegion.get(get); return !result.isEmpty(); } - - /** - * Clear the cached meta table name. - *

- * This forces the next call to {@link #load()} to read from Master Local Region. - * Primarily used for testing. - */ - void clearCache() { - cachedMetaTableName = null; - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java index e44b149820db..a9beb21364ec 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java @@ -620,7 +620,6 @@ public void updateAssignmentPlanToMeta(FavoredNodesPlan plan) throws IOException } catch (Exception e) { LOG.error( "Failed to update hbase:meta with the new assignment" + "plan because " + e.getMessage()); - LOG.info("Updated {} with the new assignment plan", connection.getMetaTableName()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java index c72d5bc1c5b2..33bf046ee00f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java @@ -131,7 +131,7 @@ public void setUpBeforeTest() throws InterruptedException, ExecutionException, I ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); conn = - new AsyncConnectionImpl(c, registry, registry.getClusterId().get(), org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()); + new AsyncConnectionImpl(c, registry, registry.getClusterId().get(), TableName.valueOf("hbase:meta"), null, User.getCurrent()); locator = new AsyncNonMetaRegionLocator(conn, AsyncConnectionImpl.RETRY_TIMER); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java index 70073cae8ad5..5a94c3feaa91 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java @@ -127,7 +127,7 @@ public static void setUp() throws Exception { ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, - registry.getClusterId().get(), org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()); + registry.getClusterId().get(), TableName.valueOf("hbase:meta"), null, User.getCurrent()); LOCATOR = new AsyncNonMetaRegionLocator(CONN, AsyncConnectionImpl.RETRY_TIMER); SPLIT_KEYS = IntStream.range(1, 256).mapToObj(i -> Bytes.toBytes(String.format("%02x", i))) .toArray(byte[][]::new); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java index cce2c4fa735b..83c826fcfbf9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java @@ -102,7 +102,7 @@ public static void setUp() throws Exception { ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, - registry.getClusterId().get(), org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()); + registry.getClusterId().get(), TableName.valueOf("hbase:meta"), null, User.getCurrent()); LOCATOR = CONN.getLocator(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java index c243901f4497..8f82df838af7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java @@ -75,7 +75,7 @@ public static void setUpBeforeClass() throws Exception { ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, - registry.getClusterId().get(), org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()); + registry.getClusterId().get(), TableName.valueOf("hbase:meta"), null, User.getCurrent()); } @AfterClass @@ -166,7 +166,7 @@ void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) { }; try (AsyncConnectionImpl mockedConn = new AsyncConnectionImpl(CONN.getConfiguration(), CONN.registry, CONN.registry.getClusterId().get(), - org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()) { + TableName.valueOf("hbase:meta"), null, User.getCurrent()) { @Override AsyncRegionLocator getLocator() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java index 12dc36223479..6f581d66fb1c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java @@ -79,7 +79,7 @@ public static void setUp() throws Exception { registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); CONN = new AsyncConnectionImpl(conf, registry, registry.getClusterId().get(), - org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()); + TableName.valueOf("hbase:meta"), null, User.getCurrent()); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFallbackToUseReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFallbackToUseReplay.java index b35bbc7ceba0..cda9104d4a0c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFallbackToUseReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFallbackToUseReplay.java @@ -99,7 +99,7 @@ public static void setUpBeforeClass() throws IOException { return null; }).when(stub).replay(any(), any(), any()); CONN = new AsyncClusterConnectionImpl(CONF, mock(ConnectionRegistry.class), "test", - org.apache.hadoop.hbase.TableName.META_TABLE_NAME, null, User.getCurrent()) { + TableName.valueOf("hbase:meta"), null, User.getCurrent()) { @Override AsyncRegionLocator getLocator() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java index abeacfbc4f46..14f3b0cd5152 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java @@ -123,7 +123,11 @@ public static class MetaTableProcedure extends Procedure @Override public TableName getTableName() { - return TEST_UTIL.getConnection().getMetaTableName(); + try { + return UTIL.getConnection().getMetaTableName(); + } catch (IOException e) { + throw new RuntimeException(e); + } } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java index 1faa97c70f95..18e8cb69d66d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java @@ -66,7 +66,7 @@ public static void setUp() throws Exception { UTIL.createTable(TABLE_NAME, CF); UTIL.waitTableAvailable(TABLE_NAME); HRegionServer rs = UTIL.getRSForFirstRegionInTable(TABLE_NAME); - if (!rs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).isEmpty()) { + if (!rs.getRegions(UTIL.getConnection().getMetaTableName()).isEmpty()) { HRegionServer rs1 = UTIL.getOtherRegionServer(rs); UTIL.moveRegionAndWait( UTIL.getMiniHBaseCluster().getRegions(TABLE_NAME).get(0).getRegionInfo(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java index baff35b8ecbb..924b0bc98c85 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java @@ -136,7 +136,7 @@ public void test() throws Exception { RegionServerThread rst1 = UTIL.getMiniHBaseCluster().getRegionServerThreads().get(1); HRegionServer liveRS; RegionServerThread toKillRSThread; - if (rst1.getRegionServer().getRegions(TEST_UTIL.getConnection().getMetaTableName()).isEmpty()) { + if (rst1.getRegionServer().getRegions(UTIL.getConnection().getMetaTableName()).isEmpty()) { liveRS = rst0.getRegionServer(); toKillRSThread = rst1; } else { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java index 9cf4db32343e..5e764ebb0ef5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestAddToSerialReplicationPeer.java @@ -94,7 +94,7 @@ public String explainFailure() throws Exception { @Test public void testAddPeer() throws Exception { TableName tableName = createTable(); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -103,7 +103,7 @@ public void testAddPeer() throws Exception { HRegionServer rs = UTIL.getOtherRegionServer(UTIL.getRSForFirstRegionInTable(tableName)); moveRegionAndArchiveOldWals(region, rs); addPeer(true); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -120,7 +120,7 @@ public void testChangeToSerial() throws Exception { UTIL.getAdmin().addReplicationPeer(PEER_ID, peerConfig, true); TableName tableName = createTable(); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -142,7 +142,7 @@ public void testChangeToSerial() throws Exception { ReplicationPeerConfig.newBuilder(peerConfig).setSerial(true).build()); UTIL.getAdmin().enableReplicationPeer(PEER_ID); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -160,7 +160,7 @@ public void testAddToSerialPeer() throws Exception { UTIL.getAdmin().addReplicationPeer(PEER_ID, peerConfig, true); TableName tableName = createTable(); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -183,7 +183,7 @@ public void testAddToSerialPeer() throws Exception { ReplicationPeerConfig.newBuilder(peerConfig) .setTableCFsMap(ImmutableMap.of(tableName, Collections.emptyList())).build()); UTIL.getAdmin().enableReplicationPeer(PEER_ID); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -195,7 +195,7 @@ public void testAddToSerialPeer() throws Exception { @Test public void testDisabledTable() throws Exception { TableName tableName = createTable(); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -204,7 +204,7 @@ public void testDisabledTable() throws Exception { rollAllWALs(); addPeer(true); UTIL.getAdmin().enableTable(tableName); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -216,7 +216,7 @@ public void testDisabledTable() throws Exception { @Test public void testDisablingTable() throws Exception { TableName tableName = createTable(); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -239,7 +239,7 @@ public void testDisablingTable() throws Exception { tsm.setTableState(tableName, TableState.State.DISABLED); t.join(); UTIL.getAdmin().enableTable(tableName); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -251,7 +251,7 @@ public void testDisablingTable() throws Exception { @Test public void testEnablingTable() throws Exception { TableName tableName = createTable(); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -274,7 +274,7 @@ public void testEnablingTable() throws Exception { assertTrue(t.isAlive()); tsm.setTableState(tableName, TableState.State.ENABLED); t.join(); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestRemoveFromSerialReplicationPeer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestRemoveFromSerialReplicationPeer.java index 2dc52eb90a3a..23f4dca33447 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestRemoveFromSerialReplicationPeer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestRemoveFromSerialReplicationPeer.java @@ -80,7 +80,7 @@ public void testRemoveTable() throws Exception { .setReplicateAllUserTables(false) .setTableCFsMap(ImmutableMap.of(tableName, Collections.emptyList())).setSerial(true).build(); UTIL.getAdmin().addReplicationPeer(PEER_ID, peerConfig, true); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -101,7 +101,7 @@ public void testRemoveTable() throws Exception { public void testRemoveSerialFlag() throws Exception { TableName tableName = createTable(); addPeer(true); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } From 5a6c4f0ae6bd1d2ca60af0822c9121c6402a0b2a Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Thu, 22 Jan 2026 12:06:38 -0500 Subject: [PATCH 12/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: Format code part 2 --- .../TestMetaUpdatesGoToPriorityQueue.java | 2 +- .../client/AbstractTestRegionLocator.java | 25 +++++++++++-------- .../client/TestAsyncTableUseMetaReplicas.java | 4 +-- ...talogReplicaLoadBalanceSimpleSelector.java | 4 +-- .../client/TestClientScannerTimeouts.java | 18 +++++++------ .../hbase/client/TestReplicaWithCluster.java | 2 +- .../hbase/client/TestSnapshotFromClient.java | 2 +- .../client/TestZKConnectionRegistry.java | 6 ++++- .../hadoop/hbase/http/TestInfoServersACL.java | 4 +-- .../TestMetaAssignmentWithStopMaster.java | 2 +- .../master/TestMigrateNamespaceTable.java | 8 ++++-- .../TestRestartWithEmptyWALDirectory.java | 2 +- ...ServerCrashProcedureCarryingMetaStuck.java | 4 +-- .../TestCloseRegionWhileRSCrash.java | 6 ++--- .../master/assignment/TestRollbackSCP.java | 2 +- ...MetaTableIsolationBalancerConditional.java | 9 +++++-- .../TestTruncateTableWithMasterFailover.java | 4 +-- .../TestCompactionInDeadRegionServer.java | 2 +- .../replication/TestSerialReplication.java | 16 ++++++------ .../TestSerialReplicationFailover.java | 4 +-- .../hadoop/hbase/util/TestRegionMover1.java | 8 +++++- .../TestRegionMoverWithRSGroupEnable.java | 9 ++++++- 22 files changed, 89 insertions(+), 54 deletions(-) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java index 99e59c73a440..5aa7cde8fb36 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java @@ -87,7 +87,7 @@ private void multiMutate(byte[] row, List mutations) throws IOExceptio } } MutateRowsRequest request = builder.build(); - AsyncTable table = UTIL.getAsyncConnection().getTable(TEST_UTIL.getConnection().getMetaTableName()); + AsyncTable table = UTIL.getAsyncConnection().getTable(UTIL.getConnection().getMetaTableName()); CompletableFuture future = table. coprocessorService(MultiRowMutationService::newStub, (stub, controller, done) -> stub.mutateRows(controller, request, done), row); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java index 66e86cf031bf..e5bdb4ca0ca4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java @@ -50,7 +50,7 @@ public abstract class AbstractTestRegionLocator { protected static void startClusterAndCreateTable() throws Exception { UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), TEST_UTIL.getConnection().getMetaTableName(), REGION_REPLICATION); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), UTIL.getConnection().getMetaTableName(), REGION_REPLICATION); TableDescriptor td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRegionReplication(REGION_REPLICATION) .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build(); @@ -70,7 +70,7 @@ protected static void startClusterAndCreateTable() throws Exception { @After public void tearDownAfterTest() throws IOException { clearCache(TABLE_NAME); - clearCache(TEST_UTIL.getConnection().getMetaTableName()); + clearCache(UTIL.getConnection().getMetaTableName()); } private byte[] getStartKey(int index) { @@ -171,8 +171,13 @@ private void assertMetaRegionLocation(HRegionLocation loc, int replicaId) { assertArrayEquals(HConstants.EMPTY_START_ROW, region.getStartKey()); assertArrayEquals(HConstants.EMPTY_END_ROW, region.getEndKey()); assertEquals(replicaId, region.getReplicaId()); - ServerName expected = - findRegionLocation(TEST_UTIL.getConnection().getMetaTableName(), region.getStartKey(), replicaId); + ServerName expected; + try { + expected = + findRegionLocation(UTIL.getConnection().getMetaTableName(), region.getStartKey(), replicaId); + } catch (IOException e) { + throw new RuntimeException(e); + } assertEquals(expected, loc.getServerName()); } @@ -185,19 +190,19 @@ private void assertMetaRegionLocations(List locs) { @Test public void testMeta() throws IOException { - assertMetaStartOrEndKeys(getStartKeys(TEST_UTIL.getConnection().getMetaTableName())); - assertMetaStartOrEndKeys(getEndKeys(TEST_UTIL.getConnection().getMetaTableName())); - Pair startEndKeys = getStartEndKeys(TEST_UTIL.getConnection().getMetaTableName()); + assertMetaStartOrEndKeys(getStartKeys(UTIL.getConnection().getMetaTableName())); + assertMetaStartOrEndKeys(getEndKeys(UTIL.getConnection().getMetaTableName())); + Pair startEndKeys = getStartEndKeys(UTIL.getConnection().getMetaTableName()); assertMetaStartOrEndKeys(startEndKeys.getFirst()); assertMetaStartOrEndKeys(startEndKeys.getSecond()); for (int replicaId = 0; replicaId < REGION_REPLICATION; replicaId++) { assertMetaRegionLocation( - getRegionLocation(TEST_UTIL.getConnection().getMetaTableName(), HConstants.EMPTY_START_ROW, replicaId), + getRegionLocation(UTIL.getConnection().getMetaTableName(), HConstants.EMPTY_START_ROW, replicaId), replicaId); } assertMetaRegionLocations( - getRegionLocations(TEST_UTIL.getConnection().getMetaTableName(), HConstants.EMPTY_START_ROW)); - assertMetaRegionLocations(getAllRegionLocations(TEST_UTIL.getConnection().getMetaTableName())); + getRegionLocations(UTIL.getConnection().getMetaTableName(), HConstants.EMPTY_START_ROW)); + assertMetaRegionLocations(getAllRegionLocations(UTIL.getConnection().getMetaTableName())); } protected abstract byte[][] getStartKeys(TableName tableName) throws IOException; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java index ac21c6ea8dcc..65a0dc22f69d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java @@ -95,14 +95,14 @@ public static void setUp() throws Exception { conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, FailPrimaryMetaScanCp.class.getName()); UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), TEST_UTIL.getConnection().getMetaTableName(), 3); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), UTIL.getConnection().getMetaTableName(), 3); try (ConnectionRegistry registry = ConnectionRegistryFactory.create(conf, User.getCurrent())) { RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(UTIL, registry); } try (Table table = UTIL.createTable(TABLE_NAME, FAMILY)) { table.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)); } - UTIL.flush(TEST_UTIL.getConnection().getMetaTableName()); + UTIL.flush(UTIL.getConnection().getMetaTableName()); // wait for the store file refresh so we can read the region location from secondary meta // replicas Thread.sleep(2000); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java index 6f581d66fb1c..41035c757388 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java @@ -102,7 +102,7 @@ public void testMetaChangeFromReplicaNoReplica() throws IOException, Interrupted .get(CONN.connConf.getMetaReadRpcTimeoutNs(), TimeUnit.NANOSECONDS); numOfReplicas = metaLocations.size(); } catch (Exception e) { - LOG.error("Failed to get table {}'s region replication, ", TEST_UTIL.getConnection().getMetaTableName(), + LOG.error("Failed to get table meta table's region replication, ", e); } return numOfReplicas; @@ -130,7 +130,7 @@ public void testMetaChangeFromReplicaNoReplica() throws IOException, Interrupted .get(CONN.connConf.getMetaReadRpcTimeoutNs(), TimeUnit.NANOSECONDS); numOfReplicas = metaLocations.size(); } catch (Exception e) { - LOG.error("Failed to get table {}'s region replication, ", TEST_UTIL.getConnection().getMetaTableName(), + LOG.error("Failed to get table meta table's region replication, ", e); } return numOfReplicas; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java index 42caa1350427..b6f386cd93b2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java @@ -454,15 +454,19 @@ public ScanResponse scan(final RpcController controller, final ScanRequest reque } else { ScanResponse scanRes = super.scan(controller, request); String regionName = Bytes.toString(request.getRegion().getValue().toByteArray()); - if (!regionName.contains(TEST_UTIL.getConnection().getMetaTableName().getNameAsString())) { - tableScannerId = scanRes.getScannerId(); - if (sleepOnOpen) { - try { - LOG.info("openScanner SLEEPING " + sleepTime); - Thread.sleep(sleepTime); - } catch (InterruptedException e) { + try { + if (!regionName.contains(TEST_UTIL.getConnection().getMetaTableName().getNameAsString())) { + tableScannerId = scanRes.getScannerId(); + if (sleepOnOpen) { + try { + LOG.info("openScanner SLEEPING " + sleepTime); + Thread.sleep(sleepTime); + } catch (InterruptedException e) { + } } } + } catch (IOException e) { + throw new RuntimeException(e); } return scanRes; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java index 04938eb2c2c9..6af93a59ec39 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java @@ -262,7 +262,7 @@ public static void beforeClass() throws Exception { HTU.startMiniCluster(NB_SERVERS); // Enable meta replica at server side - HBaseTestingUtil.setReplicas(HTU.getAdmin(), TEST_HTU.getConnection().getMetaTableName(), 2); + HBaseTestingUtil.setReplicas(HTU.getAdmin(), HTU.getConnection().getMetaTableName(), 2); HTU.getHBaseCluster().startMaster(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java index 17eac52a7378..da685d24a14e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java @@ -164,7 +164,7 @@ public static void cleanupTest() throws Exception { */ @Test(expected = IllegalArgumentException.class) public void testMetaTablesSnapshot() throws Exception { - UTIL.getAdmin().snapshot("metaSnapshot", TEST_UTIL.getConnection().getMetaTableName()); + UTIL.getAdmin().snapshot("metaSnapshot", UTIL.getConnection().getMetaTableName()); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java index 1095eb5159ae..354811d41d3a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java @@ -89,7 +89,11 @@ public void test() throws InterruptedException, ExecutionException, IOException IntStream.range(0, 3).forEach(i -> { HRegionLocation loc = locs.getRegionLocation(i); assertNotNull("Replica " + i + " doesn't have location", loc); - assertEquals(TEST_UTIL.getConnection().getMetaTableName(), loc.getRegion().getTable()); + try { + assertEquals(TEST_UTIL.getConnection().getMetaTableName(), loc.getRegion().getTable()); + } catch (IOException e) { + throw new RuntimeException(e); + } assertEquals(i, loc.getRegion().getReplicaId()); }); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java index 0fdc2a68591a..ce86a3e31e59 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java @@ -201,7 +201,7 @@ public void testTableActionsAvailableForAdmins() throws Exception { @Override public Void run() throws Exception { // Check the expected content is present in the http response - Pair pair = getTablePage(TEST_UTIL.getConnection().getMetaTableName()); + Pair pair = getTablePage(UTIL.getConnection().getMetaTableName()); assertEquals(HttpURLConnection.HTTP_OK, pair.getFirst().intValue()); assertTrue("expected=" + expectedAuthorizedContent + ", content=" + pair.getSecond(), pair.getSecond().contains(expectedAuthorizedContent)); @@ -214,7 +214,7 @@ public Void run() throws Exception { nonAdmin.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - Pair pair = getTablePage(TEST_UTIL.getConnection().getMetaTableName()); + Pair pair = getTablePage(UTIL.getConnection().getMetaTableName()); assertEquals(HttpURLConnection.HTTP_OK, pair.getFirst().intValue()); assertFalse( "should not find=" + expectedAuthorizedContent + ", content=" + pair.getSecond(), diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java index 8ddc6f2a52d4..13821417282c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java @@ -66,7 +66,7 @@ public static void tearDownAfterClass() throws Exception { @Test public void testStopActiveMaster() throws Exception { try (Connection conn = ConnectionFactory.createConnection(UTIL.getConfiguration()); - RegionLocator locator = conn.getRegionLocator(TEST_UTIL.getConnection().getMetaTableName())) { + RegionLocator locator = conn.getRegionLocator(UTIL.getConnection().getMetaTableName())) { ServerName oldMetaServer = locator.getAllRegionLocations().get(0).getServerName(); ServerName oldMaster = UTIL.getMiniHBaseCluster().getMaster().getServerName(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java index bb4e9dd63fa2..268791a1a294 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java @@ -78,7 +78,11 @@ public static final class SuspendProcedure extends Procedure @Override public TableName getTableName() { - return TEST_UTIL.getConnection().getMetaTableName(); + try { + return UTIL.getConnection().getMetaTableName(); + } catch (IOException e) { + throw new RuntimeException(e); + } } @Override @@ -155,7 +159,7 @@ public static void tearDown() throws Exception { private void removeNamespaceFamily() throws IOException { FileSystem fs = UTIL.getTestFileSystem(); Path rootDir = CommonFSUtils.getRootDir(UTIL.getConfiguration()); - Path tableDir = CommonFSUtils.getTableDir(rootDir, TEST_UTIL.getConnection().getMetaTableName()); + Path tableDir = CommonFSUtils.getTableDir(rootDir, UTIL.getConnection().getMetaTableName()); TableDescriptor metaTableDesc = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir); TableDescriptor noNsMetaTableDesc = TableDescriptorBuilder.newBuilder(metaTableDesc) .removeColumnFamily(HConstants.NAMESPACE_FAMILY).build(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java index 8aed78d71013..8f1df662f7b9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java @@ -81,7 +81,7 @@ public void testRestart() throws IOException, InterruptedException { table.put(new Put(row).addColumn(FAMILY, QUALIFIER, row)); } // flush all in memory data - UTIL.flush(TEST_UTIL.getConnection().getMetaTableName()); + UTIL.flush(UTIL.getConnection().getMetaTableName()); UTIL.flush(NAME); // stop master first, so when stopping region server, we will not schedule a SCP. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java index 15a4047a911f..a3f0f3940cdc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java @@ -64,13 +64,13 @@ public static void tearDown() throws Exception { public void test() throws Exception { RegionServerThread rsThread = null; for (RegionServerThread t : UTIL.getMiniHBaseCluster().getRegionServerThreads()) { - if (!t.getRegionServer().getRegions(TEST_UTIL.getConnection().getMetaTableName()).isEmpty()) { + if (!t.getRegionServer().getRegions(UTIL.getConnection().getMetaTableName()).isEmpty()) { rsThread = t; break; } } HRegionServer rs = rsThread.getRegionServer(); - RegionInfo hri = rs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).get(0).getRegionInfo(); + RegionInfo hri = rs.getRegions(UTIL.getConnection().getMetaTableName()).get(0).getRegionInfo(); HMaster master = UTIL.getMiniHBaseCluster().getMaster(); ProcedureExecutor executor = master.getMasterProcedureExecutor(); DummyRegionProcedure proc = new DummyRegionProcedure(executor.getEnvironment(), hri); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java index 2a5b7e29a594..578767d86348 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java @@ -151,11 +151,11 @@ public static void setUp() throws Exception { UTIL.createTable(TABLE_NAME, CF); UTIL.getAdmin().balancerSwitch(false, true); HRegionServer srcRs = UTIL.getRSForFirstRegionInTable(TABLE_NAME); - if (!srcRs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).isEmpty()) { - RegionInfo metaRegion = srcRs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).get(0).getRegionInfo(); + if (!srcRs.getRegions(UTIL.getConnection().getMetaTableName()).isEmpty()) { + RegionInfo metaRegion = srcRs.getRegions(UTIL.getConnection().getMetaTableName()).get(0).getRegionInfo(); HRegionServer dstRs = UTIL.getOtherRegionServer(srcRs); UTIL.getAdmin().move(metaRegion.getEncodedNameAsBytes(), dstRs.getServerName()); - UTIL.waitFor(30000, () -> !dstRs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).isEmpty()); + UTIL.waitFor(30000, () -> !dstRs.getRegions(UTIL.getConnection().getMetaTableName()).isEmpty()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java index e006accb2303..b2e21486a388 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java @@ -168,7 +168,7 @@ public void describeTo(Description description) { @Test public void testFailAndRollback() throws Exception { - HRegionServer rsWithMeta = UTIL.getRSForFirstRegionInTable(TEST_UTIL.getConnection().getMetaTableName()); + HRegionServer rsWithMeta = UTIL.getRSForFirstRegionInTable(UTIL.getConnection().getMetaTableName()); UTIL.getMiniHBaseCluster().killRegionServer(rsWithMeta.getServerName()); UTIL.waitFor(15000, () -> getSCPForServer(rsWithMeta.getServerName()) != null); ServerCrashProcedure scp = getSCPForServer(rsWithMeta.getServerName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java index dd7927618df9..e85165bca4ac 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java @@ -148,8 +148,13 @@ private static void validateRegionLocationsWithRetry(Connection connection, private static void validateRegionLocations(Map> tableToServers, TableName productTableName, boolean shouldBeBalanced) { // Validate that the region assignments - ServerName metaServer = - tableToServers.get(TEST_UTIL.getConnection().getMetaTableName()).stream().findFirst().orElseThrow(); + ServerName metaServer; + try { + metaServer = + tableToServers.get(TEST_UTIL.getConnection().getMetaTableName()).stream().findFirst().orElseThrow(); + } catch (IOException e) { + throw new RuntimeException(e); + } ServerName quotaServer = tableToServers.get(QuotaUtil.QUOTA_TABLE_NAME).stream().findFirst().orElseThrow(); Set productServers = tableToServers.get(productTableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableWithMasterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableWithMasterFailover.java index 6b269db477a0..e14075176169 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableWithMasterFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTruncateTableWithMasterFailover.java @@ -64,7 +64,7 @@ private void testTruncateWithFailoverAtStep(final boolean preserveSplits, final RegionInfo[] regions = MasterProcedureTestingUtility.createTable(getMasterProcedureExecutor(), tableName, splitKeys, families); // load and verify that there are rows in the table - MasterProcedureTestingUtility.loadData(TEST_UTIL.getConnection(), tableName, 100, splitKeys, + MasterProcedureTestingUtility.loadData(UTIL.getConnection(), tableName, 100, splitKeys, families); assertEquals(100, UTIL.countRows(tableName)); // disable the table @@ -95,7 +95,7 @@ private void testTruncateWithFailoverAtStep(final boolean preserveSplits, final assertEquals(0, UTIL.countRows(tableName)); // verify that the table is read/writable - MasterProcedureTestingUtility.loadData(TEST_UTIL.getConnection(), tableName, 50, splitKeys, + MasterProcedureTestingUtility.loadData(UTIL.getConnection(), tableName, 50, splitKeys, families); assertEquals(50, UTIL.countRows(tableName)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java index dbf3c6011fea..7f76dda62f0d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java @@ -132,7 +132,7 @@ public void test() throws Exception { HRegionServer regionSvr = UTIL.getRSForFirstRegionInTable(TABLE_NAME); HRegion region = regionSvr.getRegions(TABLE_NAME).get(0); String regName = region.getRegionInfo().getEncodedName(); - List metaRegs = regionSvr.getRegions(TEST_UTIL.getConnection().getMetaTableName()); + List metaRegs = regionSvr.getRegions(UTIL.getConnection().getMetaTableName()); if (metaRegs != null && !metaRegs.isEmpty()) { LOG.info("meta is on the same server: " + regionSvr); // when region is on same server as hbase:meta, reassigning meta would abort the server diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java index b1ae3e57b1fb..7ab82b60cec9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplication.java @@ -66,7 +66,7 @@ public void setUp() throws IOException, StreamLacksCapabilityException { @Test public void testRegionMove() throws Exception { TableName tableName = createTable(); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -74,7 +74,7 @@ public void testRegionMove() throws Exception { RegionInfo region = UTIL.getAdmin().getRegions(tableName).get(0); HRegionServer rs = UTIL.getOtherRegionServer(UTIL.getRSForFirstRegionInTable(tableName)); moveRegion(region, rs); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 100; i < 200; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -86,7 +86,7 @@ public void testRegionMove() throws Exception { @Test public void testRegionSplit() throws Exception { TableName tableName = createTable(); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -98,7 +98,7 @@ public void testRegionSplit() throws Exception { UTIL.waitUntilNoRegionsInTransition(30000); List regions = UTIL.getAdmin().getRegions(tableName); assertEquals(2, regions.size()); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -145,7 +145,7 @@ public void testRegionMerge() throws Exception { .build(), new byte[][] { splitKey }); UTIL.waitTableAvailable(tableName); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -158,7 +158,7 @@ public void testRegionMerge() throws Exception { UTIL.waitUntilNoRegionsInTransition(30000); List regionsAfterMerge = UTIL.getAdmin().getRegions(tableName); assertEquals(1, regionsAfterMerge.size()); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -212,7 +212,7 @@ public void testRemovePeerNothingReplicated() throws Exception { @Test public void testRemovePeer() throws Exception { TableName tableName = createTable(); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -232,7 +232,7 @@ public void testRemovePeer() throws Exception { @Test public void testRemoveSerialFlag() throws Exception { TableName tableName = createTable(); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplicationFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplicationFailover.java index 9a0df321029a..1295ea14abcd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplicationFailover.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestSerialReplicationFailover.java @@ -56,7 +56,7 @@ public void testKillRS() throws Exception { TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder .newBuilder(CF).setScope(HConstants.REPLICATION_SCOPE_GLOBAL).build()).build()); UTIL.waitTableAvailable(tableName); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 0; i < 100; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } @@ -65,7 +65,7 @@ public void testKillRS() throws Exception { .filter(t -> !t.getRegionServer().getRegions(tableName).isEmpty()).findFirst().get(); thread.getRegionServer().abort("for testing"); thread.join(); - try (Table table = TEST_UTIL.getConnection().getTable(tableName)) { + try (Table table = UTIL.getConnection().getTable(tableName)) { for (int i = 100; i < 200; i++) { table.put(new Put(Bytes.toBytes(i)).addColumn(CF, CQ, Bytes.toBytes(i))); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java index 28d982076fc7..d550dafaac7e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java @@ -288,7 +288,13 @@ public void testRegionServerPort() throws Exception { public void testLoadMetaRegion() throws Exception { HRegionServer rsWithMeta = TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads().stream() .map(t -> t.getRegionServer()) - .filter(rs -> rs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() > 0).findFirst().get(); + .filter(rs -> { + try { + return rs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() > 0; + } catch (IOException e) { + throw new RuntimeException(e); + } + }).findFirst().get(); int onlineRegions = rsWithMeta.getNumberOfOnlineRegions(); String rsName = rsWithMeta.getServerName().getAddress().toString(); try (RegionMover rm = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java index 4977f5912259..71c5c2de8ce4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertEquals; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; @@ -87,7 +88,13 @@ public void setUp() throws Exception { // Remove rs contains hbase:meta, otherwise test looks unstable and buggy in test env. ServerName rsContainMeta = TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads().stream() .map(t -> t.getRegionServer()) - .filter(rs -> rs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() > 0).findFirst().get() + .filter(rs -> { + try { + return rs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() > 0; + } catch (IOException e) { + throw new RuntimeException(e); + } + }).findFirst().get() .getServerName(); LOG.info("{} contains hbase:meta", rsContainMeta); List modifiable = new ArrayList<>(allServers); From 14fa2bf5fa4847257961f97d27979d2df3e3d699 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Thu, 22 Jan 2026 12:08:25 -0500 Subject: [PATCH 13/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: Format code part 3: spotless --- .../backup/impl/IncrementalBackupManager.java | 1 - .../favored/FavoredNodeAssignmentHelper.java | 1 - ...rgeClusterBalancingMetaTableIsolation.java | 4 +- ...gTableIsolationAndReplicaDistribution.java | 3 +- .../AbstractRpcBasedConnectionRegistry.java | 22 ++++------ .../hadoop/hbase/client/AsyncConnection.java | 4 +- .../client/AsyncNonMetaRegionLocator.java | 4 +- .../hbase/client/AsyncRegionLocator.java | 1 - .../hadoop/hbase/client/Connection.java | 4 +- .../hbase/client/ConnectionFactory.java | 18 ++++---- .../hbase/client/MutableRegionInfo.java | 1 - .../hbase/client/RawAsyncHBaseAdmin.java | 1 - .../hbase/client/RegionInfoBuilder.java | 4 +- .../hbase/client/TableDescriptorBuilder.java | 5 +-- .../hbase/client/ZKConnectionRegistry.java | 4 +- .../hbase/security/token/ClientTokenUtil.java | 1 - .../hbase/shaded/protobuf/ProtobufUtil.java | 1 - .../client/DoNothingConnectionRegistry.java | 3 +- .../client/TestAsyncAdminRpcPriority.java | 1 - .../client/TestAsyncConnectionTracing.java | 3 +- .../client/TestAsyncRegionLocatorTracing.java | 15 +++---- .../client/TestAsyncTableRpcPriority.java | 1 - .../hbase/client/TestCompactFromClient.java | 1 - .../hbase/client/TestMetricsConnection.java | 7 ++-- .../hbase/client/TestRegionInfoBuilder.java | 1 - .../hadoop/hbase/CellComparatorImpl.java | 2 +- .../org/apache/hadoop/hbase/HConstants.java | 3 +- .../hbase/InnerStoreCellComparator.java | 2 +- .../org/apache/hadoop/hbase/TableName.java | 7 ++-- .../hadoop/hbase/TestCellComparator.java | 32 ++++++-------- .../org/apache/hadoop/hbase/TestKeyValue.java | 30 +++++-------- .../hbase/TestClientClusterMetrics.java | 15 +++---- .../util/ProcessBasedLocalHBaseCluster.java | 1 - .../hadoop/hbase/util/RestartMetaTest.java | 1 - .../hbase/mttr/IntegrationTestMTTR.java | 1 - .../hbase/mapreduce/TestImportExport.java | 5 +-- .../hadoop/hbase/rest/TestStatusResource.java | 4 +- .../model/TestStorageClusterStatusModel.java | 8 ++-- .../hadoop/hbase/HBaseRpcServicesBase.java | 2 +- .../apache/hadoop/hbase/HBaseServerBase.java | 4 +- .../client/ClusterConnectionFactory.java | 3 +- .../hbase/coprocessor/MetaTableMetrics.java | 3 +- .../apache/hadoop/hbase/master/HMaster.java | 25 +++++------ ...sterAnnotationReadingPriorityFunction.java | 1 - .../hbase/master/MetaTableNameStore.java | 9 ++-- .../master/RegionPlacementMaintainer.java | 1 - .../hbase/master/TableNamespaceManager.java | 8 ++-- .../hbase/master/TableStateManager.java | 1 - .../master/assignment/AssignmentManager.java | 5 +-- .../MergeTableRegionsProcedure.java | 1 - .../master/assignment/RegionStateStore.java | 7 +--- .../assignment/SplitTableRegionProcedure.java | 1 - .../cleaner/ReplicationBarrierCleaner.java | 6 +-- .../hadoop/hbase/master/http/MetaBrowser.java | 1 - .../hbase/master/janitor/CatalogJanitor.java | 5 +-- .../hbase/master/janitor/MetaFixer.java | 5 +-- .../master/janitor/ReportMakingVisitor.java | 1 - .../procedure/DeleteTableProcedure.java | 5 +-- .../procedure/DisableTableProcedure.java | 1 - .../procedure/HBCKServerCrashProcedure.java | 10 ++--- .../master/procedure/InitMetaProcedure.java | 5 +-- .../procedure/MasterProcedureScheduler.java | 1 - .../hbase/master/procedure/MetaQueue.java | 3 +- .../MigrateNamespaceTableProcedure.java | 8 ++-- .../procedure/ModifyTableProcedure.java | 6 +-- .../hbase/master/procedure/SchemaLocking.java | 9 ++-- .../hbase/regionserver/RSRpcServices.java | 1 - .../ReplicationBarrierFamilyFormat.java | 1 - .../hbase/security/access/AccessChecker.java | 1 - .../hadoop/hbase/tool/BulkLoadHFilesTool.java | 1 - .../hadoop/hbase/util/FSTableDescriptors.java | 5 +-- .../apache/hadoop/hbase/util/HBaseFsck.java | 4 +- .../hadoop/hbase/util/HBaseFsckRepair.java | 1 - .../apache/hadoop/hbase/util/RegionMover.java | 1 - .../wal/BoundedRecoveredHFilesOutputSink.java | 1 - .../apache/hadoop/hbase/HBaseTestingUtil.java | 3 +- .../hadoop/hbase/TestHBaseMetaEdit.java | 6 ++- .../TestMetaUpdatesGoToPriorityQueue.java | 3 +- .../client/AbstractTestRegionLocator.java | 16 +++---- .../client/MetaWithReplicasTestBase.java | 1 - .../hbase/client/RegionReplicaTestHelper.java | 5 +-- .../apache/hadoop/hbase/client/TestAdmin.java | 1 - .../hadoop/hbase/client/TestAdmin2.java | 4 +- .../TestAsyncAdminWithRegionReplicas.java | 7 ++-- .../client/TestAsyncMetaRegionLocator.java | 5 +-- .../client/TestAsyncNonMetaRegionLocator.java | 13 +++--- .../client/TestAsyncRegionAdminApi2.java | 4 +- ...stAsyncSingleRequestRpcRetryingCaller.java | 4 +- .../hbase/client/TestAsyncTableAdminApi.java | 4 +- .../hbase/client/TestAsyncTableAdminApi2.java | 1 - .../hbase/client/TestAsyncTableAdminApi3.java | 4 +- .../client/TestAsyncTableUseMetaReplicas.java | 1 - ...talogReplicaLoadBalanceSimpleSelector.java | 42 +++++++++---------- .../hbase/client/TestCleanupMetaReplica.java | 4 +- .../client/TestClientScannerTimeouts.java | 5 ++- .../client/TestClientSideRegionScanner.java | 1 - .../hadoop/hbase/client/TestEnableTable.java | 1 - .../hbase/client/TestFromClientSide5.java | 1 - .../TestIncreaseMetaReplicaThroughConfig.java | 1 - .../hbase/client/TestMasterRegistry.java | 4 +- .../hadoop/hbase/client/TestMetaCache.java | 1 - .../client/TestMetaRegionLocationCache.java | 4 +- .../client/TestMetaWithReplicasBasic.java | 5 +-- .../TestMetaWithReplicasShutdownHandling.java | 1 - .../TestMultiActionMetricsFromClient.java | 1 - .../hbase/client/TestReplicaWithCluster.java | 1 - .../client/TestRpcConnectionRegistry.java | 1 - .../client/TestSeparateClientZKCluster.java | 1 - .../TestShutdownOfMetaReplicaHolder.java | 4 +- .../hbase/client/TestSnapshotFromClient.java | 1 - .../client/TestZKConnectionRegistry.java | 4 +- .../hadoop/hbase/http/TestInfoServersACL.java | 1 - .../hadoop/hbase/master/TestMaster.java | 1 - ...MasterFileSystemWithStoreFileTracking.java | 4 +- ...TestMasterOperationsForRegionReplicas.java | 1 - .../hbase/master/TestMasterRepairMode.java | 1 - .../hbase/master/TestMasterTransitions.java | 1 - .../TestMetaAssignmentWithStopMaster.java | 1 - .../TestMigrateAndMirrorMetaLocations.java | 3 +- .../master/TestMigrateNamespaceTable.java | 1 - .../hbase/master/TestRecreateCluster.java | 1 - .../TestRestartWithEmptyWALDirectory.java | 1 - ...ServerCrashProcedureCarryingMetaStuck.java | 1 - .../TestCloseRegionWhileRSCrash.java | 7 ++-- .../TestOpenRegionProcedureBackoff.java | 1 - .../assignment/TestRegionStateStore.java | 1 - .../master/assignment/TestRollbackSCP.java | 4 +- .../BalancerConditionalsTestUtil.java | 1 - .../TestFavoredStochasticLoadBalancer.java | 1 - ...MetaTableIsolationBalancerConditional.java | 5 +-- .../TestReplicationBarrierCleaner.java | 1 - .../TestCatalogJanitorInMemoryStates.java | 1 - .../janitor/TestMetaFixerNoCluster.java | 21 ++++++---- .../procedure/TestProcedurePriority.java | 8 ++-- ...TestTableProcedureWaitingQueueCleanup.java | 1 - .../TestCompactionInDeadRegionServer.java | 1 - .../regionserver/TestDefaultMemStore.java | 1 - .../TestEndToEndSplitTransaction.java | 1 - .../TestGetClosestAtOrBefore.java | 1 - .../TestReadAndWriteRegionInfoFile.java | 4 +- .../hbase/regionserver/TestRegionInfo.java | 40 +++++++++++------- .../regionserver/TestRegionReplicas.java | 1 - .../TestRegionServerCrashDisableWAL.java | 1 - .../TestRegionServerNoMaster.java | 9 ++-- .../TestRegionServerRejectDuringAbort.java | 1 - .../TestShutdownWhileWALBroken.java | 1 - .../wal/AbstractTestLogRolling.java | 1 - .../regionserver/wal/TestLogRollAbort.java | 1 - .../regionserver/wal/TestLogRolling.java | 1 - .../wal/TestLogRollingNoCluster.java | 4 +- .../TestReplicationWALEntryFilters.java | 1 - .../TestMetaRegionReplicaReplication.java | 10 +++-- .../regionserver/TestReplicationSource.java | 5 +-- .../TestSerialReplicationChecker.java | 1 - .../hbase/rsgroup/TestRSGroupsKillRS.java | 1 - .../security/access/TestRpcAccessChecks.java | 5 +-- .../token/TestGenerateDelegationToken.java | 1 - .../snapshot/TestRegionSnapshotTask.java | 1 - .../hadoop/hbase/util/BaseTestHBaseFsck.java | 20 +++++---- .../hbase/util/TestFSTableDescriptors.java | 1 - ...TestHBaseFsckCleanReplicationBarriers.java | 1 - .../hadoop/hbase/util/TestRegionMover1.java | 4 +- .../TestRegionMoverWithRSGroupEnable.java | 7 +--- .../apache/hadoop/hbase/wal/TestWALSplit.java | 5 +-- .../thrift/ThriftHBaseServiceHandler.java | 7 ++-- .../hbase/zookeeper/MetaTableLocator.java | 8 ++-- .../apache/hadoop/hbase/zookeeper/ZKDump.java | 1 - 167 files changed, 316 insertions(+), 440 deletions(-) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java index 25805c4635c9..ba494d0010b3 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java @@ -27,7 +27,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.backup.util.BackupUtils; import org.apache.hadoop.hbase.client.Connection; diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java index 393eac965f34..78f2f1ea36b8 100644 --- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java +++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.CellBuilderType; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java index 34d0f9331291..ccd28c486d8a 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java @@ -27,7 +27,6 @@ import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -71,8 +70,7 @@ public static void setup() { // Create regions List allRegions = new ArrayList<>(); for (int i = 0; i < NUM_REGIONS; i++) { - TableName tableName = i < 3 ? - TableName.valueOf("hbase:meta") : NON_META_TABLE_NAME; + TableName tableName = i < 3 ? TableName.valueOf("hbase:meta") : NON_META_TABLE_NAME; byte[] startKey = new byte[1]; startKey[0] = (byte) i; byte[] endKey = new byte[1]; diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java index a4ca304e0409..34f090b32b7b 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java @@ -28,7 +28,6 @@ import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -120,7 +119,7 @@ public void testTableIsolationAndReplicaDistribution() { * Validates whether all meta table regions are isolated. */ private boolean isMetaTableIsolated(BalancerClusterState cluster) { - + return isTableIsolated(cluster, TableName.valueOf("hbase", "meta"), "Meta"); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java index dc598c4be40f..2cd9b78309c2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java @@ -256,19 +256,15 @@ public CompletableFuture getActiveMaster() { @Override public CompletableFuture getMetaTableName() { - return tracedFuture( - () -> this - . call( - (c, s, d) -> s.getMetaTableName(c, GetMetaTableNameRequest.getDefaultInstance(), d), - GetMetaTableNameResponse::hasTableName, "getMetaTableName()") - .thenApply(resp -> { - if (resp.hasTableName() && !resp.getTableName().isEmpty()) { - return TableName.valueOf(resp.getTableName()); - } else { - return TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); - } - }), - getClass().getSimpleName() + ".getMetaTableName"); + return tracedFuture(() -> this. call( + (c, s, d) -> s.getMetaTableName(c, GetMetaTableNameRequest.getDefaultInstance(), d), + GetMetaTableNameResponse::hasTableName, "getMetaTableName()").thenApply(resp -> { + if (resp.hasTableName() && !resp.getTableName().isEmpty()) { + return TableName.valueOf(resp.getTableName()); + } else { + return TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); + } + }), getClass().getSimpleName() + ".getMetaTableName"); } @Override diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java index bcdd5b01ec5d..64151fa6ad5c 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java @@ -116,8 +116,8 @@ default AsyncTable getTable(TableName tableName, ExecutorSer } /** - * Retrieve an {@link AsyncTable} implementation for accessing the meta table. - * This method returns the correct meta table for this connection + * Retrieve an {@link AsyncTable} implementation for accessing the meta table. This method returns + * the correct meta table for this connection * @return An AsyncTable to use for interactions with the meta table */ default AsyncTable getMetaTable() { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java index 016e5bbac799..eef82a3ebac6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java @@ -244,8 +244,8 @@ private boolean tryComplete(LocateRequest req, CompletableFuture createAsyncConnection(URI conne future.completeExceptionally(new IOException("meta table name came back null")); return; } - Class clazz = appliedConf.getClass( - HBASE_CLIENT_ASYNC_CONNECTION_IMPL, AsyncConnectionImpl.class, AsyncConnection.class); - try { - future.complete(user.runAs((PrivilegedExceptionAction< - ? extends AsyncConnection>) () -> ReflectionUtils.newInstance(clazz, appliedConf, + Class clazz = appliedConf.getClass( + HBASE_CLIENT_ASYNC_CONNECTION_IMPL, AsyncConnectionImpl.class, AsyncConnection.class); + try { + future.complete(user.runAs((PrivilegedExceptionAction< + ? extends AsyncConnection>) () -> ReflectionUtils.newInstance(clazz, appliedConf, registry, clusterId, metaTableName, null, user, connectionAttributes))); - } catch (Exception e) { - registry.close(); - future.completeExceptionally(e); - } + } catch (Exception e) { + registry.close(); + future.completeExceptionally(e); + } }); }); return future; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java index 909d10c6f0a4..18805bce3400 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java @@ -21,7 +21,6 @@ import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparatorImpl; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java index bc51758dd291..631d5e42cf70 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java @@ -56,7 +56,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.RegionMetrics; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java index 2ca992d24f09..ec3dd84db39d 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.client; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.yetus.audience.InterfaceAudience; @@ -43,7 +42,8 @@ public class RegionInfoBuilder { */ // TODO: How come Meta regions still do not have encoded region names? Fix. // hbase:meta,,1.1588230740 should be the hbase:meta first region name. - // TODO: For now, hardcode to default. Future: lazy initialization based on config or make it use conenction + // TODO: For now, hardcode to default. Future: lazy initialization based on config or make it use + // conenction public static final RegionInfo FIRST_META_REGIONINFO = new MutableRegionInfo(1L, TableName.valueOf("hbase", "meta"), RegionInfo.DEFAULT_REPLICA_ID); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java index a9d47460b776..f158a72b9c23 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java @@ -38,7 +38,6 @@ import java.util.stream.Collectors; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.HBaseException; @@ -616,8 +615,8 @@ private ModifyableTableDescriptor(final TableName name, this.name = name; families.forEach(c -> this.families.put(c.getName(), ColumnFamilyDescriptorBuilder.copy(c))); this.values.putAll(values); - this.values.put(IS_META_KEY, - new Bytes(Bytes.toBytes(Boolean.toString(TableName.isMetaTableName(name))))); + this.values.put(IS_META_KEY, + new Bytes(Bytes.toBytes(Boolean.toString(TableName.isMetaTableName(name))))); } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java index 36734ba3c97f..72b3c388f716 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java @@ -267,8 +267,8 @@ public CompletableFuture getActiveMaster() { @Override public CompletableFuture getMetaTableName() { return tracedFuture( - () -> CompletableFuture.completedFuture( - TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta")), + () -> CompletableFuture + .completedFuture(TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta")), "ZKConnectionRegistry.getMetaTableName"); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java index 1a01e1506fe6..47fb4594c3cf 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/security/token/ClientTokenUtil.java @@ -22,7 +22,6 @@ import java.security.PrivilegedExceptionAction; import java.util.concurrent.CompletableFuture; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.client.AsyncConnection; import org.apache.hadoop.hbase.client.AsyncTable; import org.apache.hadoop.hbase.client.Connection; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java index 823e72959645..0373edb34570 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java @@ -65,7 +65,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.KeyValue; - import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerTask; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java index 1be0c0bc9cf0..1f7cb679d92a 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java @@ -49,7 +49,8 @@ public CompletableFuture getActiveMaster() { return CompletableFuture.completedFuture(null); } - @Override public CompletableFuture getMetaTableName() { + @Override + public CompletableFuture getMetaTableName() { return CompletableFuture.completedFuture(null); } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java index 833cb39076e8..575f7f69a94b 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java @@ -33,7 +33,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.HBaseRpcController; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java index df27e49ee23a..29f9543cc67c 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java @@ -73,7 +73,8 @@ public CompletableFuture getActiveMaster() { return CompletableFuture.completedFuture(masterServer); } }; - conn = new AsyncConnectionImpl(CONF, registry, "test", TableName.valueOf("hbase:meta"), null, user); + conn = + new AsyncConnectionImpl(CONF, registry, "test", TableName.valueOf("hbase:meta"), null, user); } @After diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java index 0b4ef2a1ebde..5c533d2522d7 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java @@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MatcherPredicate; - import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -161,12 +160,10 @@ public void testGetRegionLocation() { conn.getLocator().getRegionLocation(conn.getMetaTableName(), HConstants.EMPTY_START_ROW, RegionLocateType.CURRENT, TimeUnit.SECONDS.toNanos(1)).join(); SpanData span = waitSpan("AsyncRegionLocator.getRegionLocation"); - assertThat(span, - allOf(hasStatusWithCode(StatusCode.OK), hasKind(SpanKind.INTERNAL), - buildConnectionAttributesMatcher(conn), - buildTableAttributesMatcher(conn.getMetaTableName()), - hasAttributes(containsEntryWithStringValuesOf("db.hbase.regions", - locs.getDefaultRegionLocation().getRegion().getRegionNameAsString())))); + assertThat(span, allOf(hasStatusWithCode(StatusCode.OK), hasKind(SpanKind.INTERNAL), + buildConnectionAttributesMatcher(conn), buildTableAttributesMatcher(conn.getMetaTableName()), + hasAttributes(containsEntryWithStringValuesOf("db.hbase.regions", + locs.getDefaultRegionLocation().getRegion().getRegionNameAsString())))); } @Test @@ -178,8 +175,8 @@ public void testGetRegionLocations() { Arrays.stream(locs.getRegionLocations()).map(HRegionLocation::getRegion) .map(RegionInfo::getRegionNameAsString).toArray(String[]::new); assertThat(span, allOf(hasStatusWithCode(StatusCode.OK), hasKind(SpanKind.INTERNAL), - buildConnectionAttributesMatcher(conn), - buildTableAttributesMatcher(conn.getMetaTableName()), hasAttributes( + buildConnectionAttributesMatcher(conn), buildTableAttributesMatcher(conn.getMetaTableName()), + hasAttributes( containsEntryWithStringValuesOf("db.hbase.regions", containsInAnyOrder(expectedRegions))))); } } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java index 175f5f446e5a..7e61b299e1ea 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java @@ -50,7 +50,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.HBaseRpcController; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java index 5461509379ec..f6093dfe9533 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java @@ -33,7 +33,6 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.hbase.ClientMetaTableAccessor; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.testclassification.ClientTests; diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java index d8d0aea73f95..7558e0056169 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java @@ -113,8 +113,8 @@ public void testMetricsConnectionScope() throws IOException { assertEquals(clusterId + "@" + Integer.toHexString(impl.hashCode()), metrics.get().getMetricScope()); conf.set(MetricsConnection.METRICS_SCOPE_KEY, scope); - impl = new AsyncConnectionImpl(conf, null, "foo", - TableName.valueOf("hbase:meta"), null, User.getCurrent()); + impl = new AsyncConnectionImpl(conf, null, "foo", TableName.valueOf("hbase:meta"), null, + User.getCurrent()); metrics = impl.getConnectionMetrics(); assertTrue("Metrics should be present", metrics.isPresent()); @@ -134,8 +134,7 @@ public void testMetricsWithMultiConnections() throws IOException { AsyncConnectionImpl impl; List connList = new ArrayList(); for (int i = 0; i < num; i++) { - impl = new AsyncConnectionImpl(conf, null, null, - TableName.valueOf("hbase:meta"), null, user); + impl = new AsyncConnectionImpl(conf, null, null, TableName.valueOf("hbase:meta"), null, user); connList.add(impl); } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java index 462559c8b1d4..f8e6b1533505 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java @@ -26,7 +26,6 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; -import org.apache.hadoop.hbase.HBaseCommonTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java index 15c1cd204f84..e2faf575975c 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java @@ -792,7 +792,7 @@ public static CellComparator getCellComparator(byte[] tableName) { ? MetaCellComparator.META_COMPARATOR : CellComparatorImpl.COMPARATOR; } - + private static boolean isMetaTable(byte[] tableName) { return Bytes.startsWith(tableName, Bytes.toBytes("hbase:meta")); } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 9ac69aaefb46..08b1fe06a7a2 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1209,8 +1209,7 @@ public enum OperationStatusCode { @Deprecated public static final List HBASE_NON_USER_TABLE_DIRS = Collections.unmodifiableList(Arrays.asList( - (String[]) ArrayUtils.addAll(new String[] { "hbase:meta" }, - HBASE_NON_TABLE_DIRS.toArray()))); + (String[]) ArrayUtils.addAll(new String[] { "hbase:meta" }, HBASE_NON_TABLE_DIRS.toArray()))); /** Health script related settings. */ public static final String HEALTH_SCRIPT_LOC = "hbase.node.health.script.location"; diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java index 88a1ec2561c7..fe95aa7195e8 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java @@ -80,7 +80,7 @@ public static CellComparator getInnerStoreCellComparator(byte[] tableName) { ? MetaCellComparator.META_COMPARATOR : InnerStoreCellComparator.INNER_STORE_COMPARATOR; } - + private static boolean isMetaTable(byte[] tableName) { return Bytes.startsWith(tableName, Bytes.toBytes("hbase:meta")); } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java index e2cd2242ceb0..eab8b9c9294e 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java @@ -25,7 +25,6 @@ import org.apache.commons.lang3.ArrayUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.yetus.audience.InterfaceStability; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -91,9 +90,9 @@ public final class TableName implements Comparable { /** One globally disallowed name */ public static final String DISALLOWED_TABLE_NAME = "zookeeper"; - /** - * Returns True if tn is a meta table (hbase:meta or hbase:meta_suffix). - * This handles both the default meta table and read replica meta tables. + /** + * Returns True if tn is a meta table (hbase:meta or hbase:meta_suffix). This handles + * both the default meta table and read replica meta tables. */ public static boolean isMetaTableName(final TableName tn) { if (tn == null) return false; diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java index 254dce9665b7..d4908903682a 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java @@ -196,37 +196,31 @@ public void testMetaComparisons2() { long now = EnvironmentEdgeManager.currentTime(); CellComparator c = MetaCellComparator.META_COMPARATOR; assertTrue(c.compare( - createByteBufferKeyValueFromKeyValue(new KeyValue( - Bytes.toBytes("hbase:meta" + ",a,,0,1"), now)), - createByteBufferKeyValueFromKeyValue(new KeyValue( - Bytes.toBytes("hbase:meta" + ",a,,0,1"), now))) == 0); + createByteBufferKeyValueFromKeyValue( + new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now)), + createByteBufferKeyValueFromKeyValue( + new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now))) == 0); Cell a = createByteBufferKeyValueFromKeyValue( new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now)); Cell b = createByteBufferKeyValueFromKeyValue( new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,2"), now)); assertTrue(c.compare(a, b) < 0); - assertTrue(c.compare( - createByteBufferKeyValueFromKeyValue(new KeyValue( - Bytes.toBytes("hbase:meta" + ",a,,0,2"), now)), - createByteBufferKeyValueFromKeyValue(new KeyValue( - Bytes.toBytes("hbase:meta" + ",a,,0,1"), now))) > 0); assertTrue(c.compare( createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now)), + new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,2"), now)), createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now))) + new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now))) > 0); + assertTrue(c.compare( + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now)), + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now))) == 0); assertTrue(c.compare( - createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now)), - createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes("hbase:meta" + ",,2"), now))) + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now)), + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta" + ",,2"), now))) < 0); assertTrue(c.compare( - createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes("hbase:meta" + ",,2"), now)), - createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now))) + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta" + ",,2"), now)), + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now))) > 0); } diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java index a55bcae490af..cfb02f44b1fd 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java @@ -197,32 +197,22 @@ public void testKeyValueBorderCases() { private void metacomparisons(final CellComparatorImpl c) { long now = EnvironmentEdgeManager.currentTime(); - assertTrue(c.compare( - new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now), - new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now)) - == 0); - KeyValue a = - new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now); - KeyValue b = - new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,2"), now); + assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now), + new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now)) == 0); + KeyValue a = new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now); + KeyValue b = new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,2"), now); assertTrue(c.compare(a, b) < 0); - assertTrue(c.compare( - new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,2"), now), - new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now)) - > 0); + assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,2"), now), + new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now)) > 0); } private void comparisons(final CellComparatorImpl c) { long now = EnvironmentEdgeManager.currentTime(); - assertTrue(c.compare( - new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now), - new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now)) - == 0); - assertTrue(c.compare( - new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now), + assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now), + new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now)) == 0); + assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now), new KeyValue(Bytes.toBytes("hbase:meta" + ",,2"), now)) < 0); - assertTrue(c.compare( - new KeyValue(Bytes.toBytes("hbase:meta" + ",,2"), now), + assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta" + ",,2"), now), new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now)) > 0); } diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java index 6daf0f94be17..2c66a0fc278a 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java @@ -221,9 +221,8 @@ public void testRegionStatesCount() throws Exception { ClusterMetrics metrics = ADMIN.getClusterMetrics(); Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2); - Assert.assertEquals( - metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getRegionsInTransition(), - 0); + Assert.assertEquals(metrics.getTableRegionStatesCount().get(connection.getMetaTableName()) + .getRegionsInTransition(), 0); Assert.assertEquals( metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getOpenRegions(), 1); Assert.assertEquals( @@ -252,9 +251,8 @@ public void testRegionStatesWithSplit() throws Exception { ClusterMetrics metrics = ADMIN.getClusterMetrics(); Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2); - Assert.assertEquals( - metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getRegionsInTransition(), - 0); + Assert.assertEquals(metrics.getTableRegionStatesCount().get(connection.getMetaTableName()) + .getRegionsInTransition(), 0); Assert.assertEquals( metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getOpenRegions(), 1); Assert.assertEquals( @@ -272,9 +270,8 @@ public void testRegionStatesWithSplit() throws Exception { metrics = ADMIN.getClusterMetrics(); Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2); - Assert.assertEquals( - metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getRegionsInTransition(), - 0); + Assert.assertEquals(metrics.getTableRegionStatesCount().get(connection.getMetaTableName()) + .getRegionsInTransition(), 0); Assert.assertEquals( metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getOpenRegions(), 1); Assert.assertEquals( diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java index c79de017f729..f40a4b35db13 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java @@ -42,7 +42,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hdfs.MiniDFSCluster; diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java index a6f10a477ce0..713b0133167b 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/RestartMetaTest.java @@ -20,7 +20,6 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java index 7c6c9f71c024..3fe0154ab095 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.IntegrationTestingUtility; import org.apache.hadoop.hbase.InvalidFamilyOperationException; - import org.apache.hadoop.hbase.NamespaceExistException; import org.apache.hadoop.hbase.NamespaceNotFoundException; import org.apache.hadoop.hbase.TableExistsException; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java index fb24172770e6..1cc83ac4ec26 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java @@ -50,7 +50,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.KeyValue; - import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; @@ -256,8 +255,8 @@ public void testSimpleCase() throws Throwable { */ @Test public void testMetaExport() throws Throwable { - String[] args = - new String[] { connection.getMetaTableName().getNameAsString(), FQ_OUTPUT_DIR, "1", "0", "0" }; + String[] args = new String[] { connection.getMetaTableName().getNameAsString(), FQ_OUTPUT_DIR, + "1", "0", "0" }; assertTrue(runExport(args)); } diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java index b3c77af967d4..ce3ec9e6ca26 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java @@ -29,7 +29,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.rest.client.Client; @@ -56,7 +55,8 @@ public class TestStatusResource { private static final Logger LOG = LoggerFactory.getLogger(TestStatusResource.class); - private static final byte[] META_REGION_NAME = Bytes.toBytes(connection.getMetaTableName() + ",,1"); + private static final byte[] META_REGION_NAME = + Bytes.toBytes(connection.getMetaTableName() + ",,1"); private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); private static final HBaseRESTTestingUtility REST_TEST_UTIL = new HBaseRESTTestingUtility(); diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java index 99069113c9a7..54a37a73310c 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java @@ -23,7 +23,6 @@ import java.util.Iterator; import org.apache.hadoop.hbase.HBaseClassTestRule; - import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -89,8 +88,8 @@ protected StorageClusterStatusModel buildTestModel() { model.addLiveNode("test1", 1245219839331L, 128, 1024).addRegion(Bytes.toBytes("hbase:root,,0"), 1, 1, 0, 0, 0, 1, 1, 2, 1, 1, 1, 1, 1); model.addLiveNode("test2", 1245239331198L, 512, 1024).addRegion( - Bytes.toBytes(connection.getMetaTableName() + ",,1246000043724"), 1, 1, 0, 0, 0, 1, 1, 2, 1, 1, - 1, 1, 1); + Bytes.toBytes(connection.getMetaTableName() + ",,1246000043724"), 1, 1, 0, 0, 0, 1, 1, 2, 1, + 1, 1, 1, 1); return model; } @@ -128,7 +127,8 @@ protected void checkModel(StorageClusterStatusModel model) { assertEquals(1024, node.getMaxHeapSizeMB()); regions = node.getRegions().iterator(); region = regions.next(); - assertEquals(Bytes.toString(region.getName()), connection.getMetaTableName() + ",,1246000043724"); + assertEquals(Bytes.toString(region.getName()), + connection.getMetaTableName() + ",,1246000043724"); assertEquals(1, region.getStores()); assertEquals(1, region.getStorefiles()); assertEquals(0, region.getStorefileSizeMB()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java index e2eb25d0f468..c644680dcbf4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseRpcServicesBase.java @@ -387,7 +387,7 @@ public final GetBootstrapNodesResponse getBootstrapNodes(RpcController controlle public final GetMetaTableNameResponse getMetaTableName(RpcController controller, GetMetaTableNameRequest request) throws ServiceException { GetMetaTableNameResponse.Builder builder = GetMetaTableNameResponse.newBuilder(); - + try { TableName metaTableName = server.getMetaTableName(); if (metaTableName != null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java index 9f2304f3844c..d5f481b63a6a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java @@ -679,8 +679,8 @@ public String toString() { public org.apache.hadoop.hbase.TableName getMetaTableName() { // For now, always return the default meta table name. // Future implementations may support custom meta table names from configuration or storage. - return org.apache.hadoop.hbase.TableName.valueOf( - org.apache.hadoop.hbase.NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); + return org.apache.hadoop.hbase.TableName + .valueOf(org.apache.hadoop.hbase.NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); } protected abstract CoprocessorHost getCoprocessorHost(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java index cd22d3ecd16d..b1b469110fcf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java @@ -42,8 +42,7 @@ private ClusterConnectionFactory() { private static AsyncClusterConnection createAsyncClusterConnection(Configuration conf, ConnectionRegistry registry, SocketAddress localAddress, User user) throws IOException { String clusterId = FutureUtils.get(registry.getClusterId()); - org.apache.hadoop.hbase.TableName metaTableName = - FutureUtils.get(registry.getMetaTableName()); + org.apache.hadoop.hbase.TableName metaTableName = FutureUtils.get(registry.getMetaTableName()); Class clazz = conf.getClass(HBASE_SERVER_CLUSTER_CONNECTION_IMPL, AsyncClusterConnectionImpl.class, AsyncClusterConnection.class); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java index da93a29423b4..17a314e0ae34 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java @@ -267,7 +267,8 @@ public void start(CoprocessorEnvironment env) throws IOException { if ( env instanceof RegionCoprocessorEnvironment && ((RegionCoprocessorEnvironment) env).getRegionInfo().getTable() != null - && TableName.isMetaTableName(((RegionCoprocessorEnvironment) env).getRegionInfo().getTable()) + && TableName + .isMetaTableName(((RegionCoprocessorEnvironment) env).getRegionInfo().getTable()) ) { RegionCoprocessorEnvironment regionCoprocessorEnv = (RegionCoprocessorEnvironment) env; registry = regionCoprocessorEnv.getMetricRegistryForRegionServer(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 52432697eaf5..90ef41e9231f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -83,7 +83,6 @@ import org.apache.hadoop.hbase.InvalidFamilyOperationException; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.PleaseHoldException; import org.apache.hadoop.hbase.PleaseRestartMasterException; @@ -511,8 +510,8 @@ public class HMaster extends HBaseServerBase implements Maste private ReplicationPeerModificationStateStore replicationPeerModificationStateStore; /** - * Store for the meta table name in the Master Local Region. - * This provides cluster-specific storage for dynamic meta table name discovery. + * Store for the meta table name in the Master Local Region. This provides cluster-specific + * storage for dynamic meta table name discovery. */ private MetaTableNameStore metaTableNameStore; @@ -1113,7 +1112,8 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE InitMetaProcedure initMetaProc = null; // Print out state of hbase:meta on startup; helps debugging. if ( - !this.assignmentManager.getRegionStates().hasTableRegionStates(getConnection().getMetaTableName()) + !this.assignmentManager.getRegionStates() + .hasTableRegionStates(getConnection().getMetaTableName()) ) { Optional optProc = procedureExecutor.getProcedures().stream() .filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny(); @@ -1201,12 +1201,13 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE // it is possible that we already have some replicas before upgrading, so we must set the // region replication number in meta TableDescriptor directly first, without creating a // ModifyTableProcedure, otherwise it may cause a double assign for the meta replicas. - int existingReplicasCount = - assignmentManager.getRegionStates().getRegionsOfTable(getConnection().getMetaTableName()).size(); + int existingReplicasCount = assignmentManager.getRegionStates() + .getRegionsOfTable(getConnection().getMetaTableName()).size(); if (existingReplicasCount > metaDesc.getRegionReplication()) { LOG.info( "Update replica count of {} from {}(in TableDescriptor)" + " to {}(existing ZNodes)", - getConnection().getMetaTableName(), metaDesc.getRegionReplication(), existingReplicasCount); + getConnection().getMetaTableName(), metaDesc.getRegionReplication(), + existingReplicasCount); metaDesc = TableDescriptorBuilder.newBuilder(metaDesc) .setRegionReplication(existingReplicasCount).build(); tableDescriptors.update(metaDesc); @@ -1677,8 +1678,8 @@ public TableStateManager getTableStateManager() { } /** - * Override base implementation to read from Master Local Region storage. - * This allows the master to return the cluster-specific meta table name. + * Override base implementation to read from Master Local Region storage. This allows the master + * to return the cluster-specific meta table name. */ @Override public TableName getMetaTableName() { @@ -1694,9 +1695,9 @@ public TableName getMetaTableName() { } /** - * Internal accessor for procedures to get the meta table name store. - * This is not exposed via MasterServices interface to avoid interface pollution. - * Package visibility is insufficient as procedures are in a sub-package. + * Internal accessor for procedures to get the meta table name store. This is not exposed via + * MasterServices interface to avoid interface pollution. Package visibility is insufficient as + * procedures are in a sub-package. * @return the meta table name store, or null if not yet initialized */ public MetaTableNameStore getMetaTableNameStoreInternal() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterAnnotationReadingPriorityFunction.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterAnnotationReadingPriorityFunction.java index f0d5ea07749f..b0bb167fc4fe 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterAnnotationReadingPriorityFunction.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterAnnotationReadingPriorityFunction.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.master; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ipc.AnnotationReadingPriorityFunction; import org.apache.yetus.audience.InterfaceAudience; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java index 524a64d79bb8..11001d9da4a5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java @@ -51,17 +51,15 @@ public MetaTableNameStore(MasterRegion masterRegion) { /** * Store the meta table name in the Master Local Region. *

- * This should be called once during cluster initialization (InitMetaProcedure). - * The stored value is cluster-specific and should not conflict with other clusters - * sharing the same HDFS. + * This should be called once during cluster initialization (InitMetaProcedure). The stored value + * is cluster-specific and should not conflict with other clusters sharing the same HDFS. * @param metaTableName the meta table name to store * @throws IOException if the storage operation fails */ public void store(TableName metaTableName) throws IOException { LOG.info("Storing meta table name in Master Local Region: {}", metaTableName); Put put = new Put(META_TABLE_NAME_ROW); - put.addColumn(INFO_FAMILY, NAME_QUALIFIER, - Bytes.toBytes(metaTableName.getNameAsString())); + put.addColumn(INFO_FAMILY, NAME_QUALIFIER, Bytes.toBytes(metaTableName.getNameAsString())); masterRegion.update(r -> r.put(put)); cachedMetaTableName = metaTableName; LOG.info("Successfully stored meta table name: {}", metaTableName); @@ -111,4 +109,3 @@ public boolean isStored() throws IOException { return !result.isEmpty(); } } - diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java index a9beb21364ec..8d5173d479bc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionPlacementMaintainer.java @@ -35,7 +35,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AsyncClusterConnection; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java index 52bb7d2d7667..3f04865e9bb1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; @@ -107,7 +106,9 @@ private void addToCache(Result result, byte[] family, byte[] qualifier) throws I } private void loadFromMeta() throws IOException { - try (Table table = masterServices.getConnection().getTable(masterServices.getConnection().getMetaTableName()); + try ( + Table table = + masterServices.getConnection().getTable(masterServices.getConnection().getMetaTableName()); ResultScanner scanner = table.getScanner(HConstants.NAMESPACE_FAMILY)) { for (Result result;;) { result = scanner.next(); @@ -213,7 +214,8 @@ public static void insertNamespaceToMeta(Connection conn, NamespaceDescriptor ns public void deleteNamespace(String namespaceName) throws IOException { checkMigrationDone(); Delete d = new Delete(Bytes.toBytes(namespaceName)); - try (Table table = masterServices.getConnection().getTable(masterServices.getConnection().getMetaTableName())) { + try (Table table = + masterServices.getConnection().getTable(masterServices.getConnection().getMetaTableName())) { table.delete(d); } cache.remove(namespaceName); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java index 243bedcce8e0..b71b03ccd6f9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.CatalogFamilyFormat; import org.apache.hadoop.hbase.ClientMetaTableAccessor; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.Result; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java index e604e97aaca5..d66c6459f411 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java @@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.PleaseHoldException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -1963,8 +1962,8 @@ private void checkMetaLoaded(RegionInfo hri, long procId) throws PleaseHoldExcep boolean meta = isMetaRegion(hri); boolean metaLoaded = isMetaLoaded(); if (!meta && !metaLoaded) { - throw new PleaseHoldException("Master not fully online; " + master.getConnection().getMetaTableName() + "=" - + meta + ", metaLoaded=" + metaLoaded); + throw new PleaseHoldException("Master not fully online; " + + master.getConnection().getMetaTableName() + "=" + meta + ", metaLoaded=" + metaLoaded); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java index fbf7c9af770b..e97f264de04c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java @@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.MetaMutationAnnotation; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownRegionException; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java index 50e9f914d7b5..50711115ae8f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -172,8 +171,7 @@ public static void visitMetaEntry(final RegionStateVisitor visitor, final Result LOG.debug( "Load hbase:meta entry region={}, regionState={}, lastHost={}, " + "regionLocation={}, openSeqNum={}", - regionInfo.getEncodedName(), state, lastHost, - regionLocation, openSeqNum); + regionInfo.getEncodedName(), state, lastHost, regionLocation, openSeqNum); visitor.visitRegionState(result, regionInfo, state, regionLocation, lastHost, openSeqNum); } } @@ -506,8 +504,7 @@ public void deleteMergeQualifiers(RegionInfo mergeRegion) throws IOException { + " in meta table, they are cleaned up already, Skip."); return; } - try (Table table = - master.getConnection().getTable(master.getConnection().getMetaTableName())) { + try (Table table = master.getConnection().getTable(master.getConnection().getMetaTableName())) { table.delete(delete); } LOG.info( diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java index d121bede0a57..c569c4a34301 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java @@ -38,7 +38,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.UnknownRegionException; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java index 0695bdd01483..46d8e8dc5a55 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/ReplicationBarrierCleaner.java @@ -23,7 +23,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; @@ -81,9 +80,8 @@ public synchronized void chore() { long deletedLastPushedSeqIds = 0; TableName tableName = null; List peerIds = null; - try (Table metaTable = conn.getTable(conn.getMetaTableName()); - ResultScanner scanner = metaTable.getScanner( - new Scan().addFamily(HConstants.REPLICATION_BARRIER_FAMILY).readAllVersions())) { + try (Table metaTable = conn.getTable(conn.getMetaTableName()); ResultScanner scanner = metaTable + .getScanner(new Scan().addFamily(HConstants.REPLICATION_BARRIER_FAMILY).readAllVersions())) { for (;;) { Result result = scanner.next(); if (result == null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/http/MetaBrowser.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/http/MetaBrowser.java index 231a2c565152..6c2216837a20 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/http/MetaBrowser.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/http/MetaBrowser.java @@ -33,7 +33,6 @@ import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer; import org.apache.hadoop.hbase.client.AsyncConnection; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java index 4a58e80b096a..b93eca943cb9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/CatalogJanitor.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -147,8 +146,8 @@ protected void chore() { + this.services.getServerManager().isClusterShutdown()); } } catch (IOException e) { - LOG.warn("Failed janitorial scan of {} table", - services.getConnection().getMetaTableName(), e); + LOG.warn("Failed janitorial scan of {} table", services.getConnection().getMetaTableName(), + e); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java index f14872401b94..d4ac3d8331cb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java @@ -32,7 +32,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -215,8 +214,8 @@ private static List createMetaEntries(final MasterServices masterSer "hbase:meta", createMetaEntriesFailures.size(), addMetaEntriesResults.size(), createMetaEntriesFailures.get(0).getMessage()); if (LOG.isDebugEnabled()) { - createMetaEntriesFailures.forEach(ioe -> LOG - .debug("Attempt to fix region hole in {} failed.", "hbase:meta", ioe)); + createMetaEntriesFailures + .forEach(ioe -> LOG.debug("Attempt to fix region hole in {} failed.", "hbase:meta", ioe)); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java index 965c9d728645..07dd3af70aea 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java @@ -22,7 +22,6 @@ import org.apache.hadoop.hbase.ClientMetaTableAccessor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.RegionInfo; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java index ed21e7f4b132..5b8f2571e337 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java @@ -24,7 +24,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotFoundException; @@ -395,8 +394,8 @@ private static void cleanRegionsInMeta(final MasterProcedureEnv env, final Table long now = EnvironmentEdgeManager.currentTime(); List deletes = new ArrayList<>(); try ( - Table metaTable = - env.getMasterServices().getConnection().getTable(env.getMasterServices().getConnection().getMetaTableName()); + Table metaTable = env.getMasterServices().getConnection() + .getTable(env.getMasterServices().getConnection().getMetaTableName()); ResultScanner scanner = metaTable.getScanner(tableScan)) { for (;;) { Result result = scanner.next(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java index cd8f2f9d1f45..6db264cca1c9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java @@ -20,7 +20,6 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java index 744d7ae7df11..271cb91910ad 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java @@ -25,10 +25,8 @@ import org.apache.hadoop.hbase.ClientMetaTableAccessor; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.Result; @@ -104,14 +102,16 @@ List getRegionsOnCrashedServer(MasterProcedureEnv env) { MetaTableAccessor.scanMetaForTableRegions(env.getMasterServices().getConnection(), visitor, null); } catch (IOException ioe) { - LOG.warn("Failed scan of {} for 'Unknown Servers'", env.getMasterServices().getConnection().getMetaTableName(), ioe); + LOG.warn("Failed scan of {} for 'Unknown Servers'", + env.getMasterServices().getConnection().getMetaTableName(), ioe); return ris; } // create the server state node too env.getAssignmentManager().getRegionStates().createServer(getServerName()); LOG.info("Found {} mentions of {} in {} of OPEN/OPENING Regions: {}", - visitor.getReassigns().size(), getServerName(), env.getMasterServices().getConnection().getMetaTableName(), visitor - .getReassigns().stream().map(RegionInfo::getEncodedName).collect(Collectors.joining(","))); + visitor.getReassigns().size(), getServerName(), + env.getMasterServices().getConnection().getMetaTableName(), visitor.getReassigns().stream() + .map(RegionInfo::getEncodedName).collect(Collectors.joining(","))); return visitor.getReassigns(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java index ad544eb20ab9..c83e97bf743d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java @@ -30,7 +30,6 @@ import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; @@ -113,7 +112,7 @@ protected Flow executeFromState(MasterProcedureEnv env, InitMetaState state) TableName metaTableName = TableName.valueOf("hbase", "meta"); HMaster master = (HMaster) env.getMasterServices(); MetaTableNameStore store = master.getMetaTableNameStoreInternal(); - + if (store != null && !store.isStored()) { LOG.info("Storing meta table name in Master Local Region: {}", metaTableName); store.store(metaTableName); @@ -122,7 +121,7 @@ protected Flow executeFromState(MasterProcedureEnv env, InitMetaState state) } else { LOG.info("Meta table name already stored, skipping"); } - + Configuration conf = env.getMasterConfiguration(); Path rootDir = CommonFSUtils.getRootDir(conf); TableDescriptor td = writeFsLayout(rootDir, env); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java index 06ad6cb1901a..418bda70b424 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java @@ -30,7 +30,6 @@ import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MetaQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MetaQueue.java index 81fe367f1d4a..8dd9dab23046 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MetaQueue.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MetaQueue.java @@ -19,7 +19,6 @@ import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.procedure2.LockStatus; import org.apache.hadoop.hbase.procedure2.Procedure; @@ -33,7 +32,7 @@ @InterfaceAudience.Private class MetaQueue extends Queue { - protected // Note: This is called during initialization, tableName is passed from parent + protected // Note: This is called during initialization, tableName is passed from parent MetaQueue(TableName metaTableName, LockStatus lockStatus) { super(metaTableName, 1, lockStatus); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java index 721907727820..89cbcb0e43ba 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java @@ -20,7 +20,6 @@ import java.io.IOException; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.BufferedMutator; import org.apache.hadoop.hbase.client.Connection; @@ -65,7 +64,8 @@ private void migrate(MasterProcedureEnv env) throws IOException { try (Table nsTable = conn.getTable(TableName.NAMESPACE_TABLE_NAME); ResultScanner scanner = nsTable.getScanner( new Scan().addFamily(TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES).readAllVersions()); - BufferedMutator mutator = conn.getBufferedMutator(env.getMasterServices().getConnection().getMetaTableName())) { + BufferedMutator mutator = + conn.getBufferedMutator(env.getMasterServices().getConnection().getMetaTableName())) { for (Result result;;) { result = scanner.next(); if (result == null) { @@ -88,8 +88,8 @@ protected Flow executeFromState(MasterProcedureEnv env, MigrateNamespaceTablePro try { switch (state) { case MIGRATE_NAMESPACE_TABLE_ADD_FAMILY: - TableDescriptor metaTableDesc = - env.getMasterServices().getTableDescriptors().get(env.getMasterServices().getConnection().getMetaTableName()); + TableDescriptor metaTableDesc = env.getMasterServices().getTableDescriptors() + .get(env.getMasterServices().getConnection().getMetaTableName()); if (!metaTableDesc.hasColumnFamily(HConstants.NAMESPACE_FAMILY)) { TableDescriptor newMetaTableDesc = TableDescriptorBuilder.newBuilder(metaTableDesc) .setColumnFamily( diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java index e54c515d988c..559b590baf46 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java @@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.CoprocessorDescriptor; @@ -110,8 +109,9 @@ protected void preflightChecks(MasterProcedureEnv env, Boolean enabled) throws H Set cfs = this.modifiedTableDescriptor.getColumnFamilyNames(); for (byte[] family : UNDELETABLE_META_COLUMNFAMILIES) { if (!cfs.contains(family)) { - throw new HBaseIOException("Delete of " + env.getMasterServices().getConnection().getMetaTableName() + " column family " - + Bytes.toString(family)); + throw new HBaseIOException( + "Delete of " + env.getMasterServices().getConnection().getMetaTableName() + + " column family " + Bytes.toString(family)); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java index 8c1c0a3832e2..9314e8375705 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java @@ -25,7 +25,6 @@ import java.util.stream.Collectors; import org.apache.commons.lang3.builder.ToStringBuilder; import org.apache.commons.lang3.builder.ToStringStyle; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.locking.LockProcedure; @@ -177,8 +176,9 @@ List getLocks() { addToLockedResources(lockedResources, peerLocks, Function.identity(), LockedResourceType.PEER); // TODO(Phase 6): Support replica-specific meta table names // TODO(HBASE-XXXXX - Phase 6): Get dynamic name from MasterServices - addToLockedResources(lockedResources, ImmutableMap.of(TableName.valueOf("hbase", "meta"), metaLock), - tn -> tn.getNameAsString(), LockedResourceType.META); + addToLockedResources(lockedResources, + ImmutableMap.of(TableName.valueOf("hbase", "meta"), metaLock), tn -> tn.getNameAsString(), + LockedResourceType.META); addToLockedResources(lockedResources, globalLocks, Function.identity(), LockedResourceType.GLOBAL); return lockedResources; @@ -241,7 +241,8 @@ public String toString() { .append("peerLocks", filterUnlocked(peerLocks)) // TODO(Phase 6): Support replica-specific meta table names // TODO(HBASE-XXXXX - Phase 6): Get dynamic name from MasterServices - .append("metaLocks", filterUnlocked(ImmutableMap.of(TableName.valueOf("hbase", "meta"), metaLock))) + .append("metaLocks", + filterUnlocked(ImmutableMap.of(TableName.valueOf("hbase", "meta"), metaLock))) .append("globalLocks", filterUnlocked(globalLocks)).build(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index 3ef578c4f63c..c5c3eea3e3da 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java @@ -56,7 +56,6 @@ import org.apache.hadoop.hbase.HBaseIOException; import org.apache.hadoop.hbase.HBaseRpcServicesBase; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.MultiActionResultTooLarge; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.PrivateCellUtil; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java index 5bb7e30e5e77..4caf746c6532 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Get; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java index 0f7c71354ff8..30f1d71975cb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseInterfaceAudience; - import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java index c6c2a88172a4..10b4a8a20941 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java @@ -65,7 +65,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.KeyValue; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.client.AsyncAdmin; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java index 1204aaa73d6c..de452b7ec0d7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java @@ -46,7 +46,6 @@ import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -153,8 +152,8 @@ public static TableDescriptor tryUpdateAndGetMetaTableDescriptor(Configuration c FileSystem fs, Path rootdir) throws IOException { // see if we already have meta descriptor on fs. Write one if not. TableName metaTableName = getMetaTableNameFromConf(conf); - Optional> opt = getTableDescriptorFromFs(fs, - CommonFSUtils.getTableDir(rootdir, metaTableName), false); + Optional> opt = + getTableDescriptorFromFs(fs, CommonFSUtils.getTableDir(rootdir, metaTableName), false); if (opt.isPresent()) { return opt.get().getSecond(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index d121da6e506d..db50478a188f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -81,7 +81,6 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MasterNotRunningException; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -2203,7 +2202,8 @@ else if (!inMeta && !inHdfs && !isDeployed) { } } } - LOG.info("Patching {} with .regioninfo: " + hbi.getHdfsHRI(), connection.getMetaTableName()); + LOG.info("Patching {} with .regioninfo: " + hbi.getHdfsHRI(), + connection.getMetaTableName()); int numReplicas = admin.getDescriptor(hbi.getTableName()).getRegionReplication(); HBaseFsckRepair.fixMetaHoleOnlineAndAddReplicas(getConf(), hbi.getHdfsHRI(), admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS)).getLiveServerMetrics().keySet(), diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java index a49b0a15c1a5..239a44c1b834 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java @@ -27,7 +27,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.ClusterMetrics.Option; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java index bb582ad4498b..9c1f9eb3d45a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionMover.java @@ -56,7 +56,6 @@ 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.UnknownRegionException; import org.apache.hadoop.hbase.client.Admin; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java index bbb7b8499cd2..dc1e6e140eec 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.ExtendedCell; import org.apache.hadoop.hbase.MetaCellComparator; - import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.io.hfile.CacheConfig; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java index a7df353f88bc..37fd9cbb6fe7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtil.java @@ -902,8 +902,7 @@ public SingleProcessHBaseCluster startMiniHBaseCluster(StartTestingClusterOption // Populate the master address configuration from mini cluster configuration. conf.set(HConstants.MASTER_ADDRS_KEY, MasterRegistry.getMasterAddr(c)); // Don't leave here till we've done a successful scan of the hbase:meta - try (Table t = getConnection().getMetaTable(); - ResultScanner s = t.getScanner(new Scan())) { + try (Table t = getConnection().getMetaTable(); ResultScanner s = t.getScanner(new Scan())) { for (;;) { if (s.next() == null) { break; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java index 51e9bd458b06..0c994349fe7a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java @@ -72,7 +72,8 @@ private TableDescriptor getMetaDescriptor() throws TableNotFoundException, IOExc TableDescriptor list = admin.listTableDescriptors(true).stream().filter(td -> td.isMetaTable()).findAny().get(); TableDescriptor listByName = - admin.listTableDescriptors(Collections.singletonList(UTIL.getConnection().getMetaTableName())).get(0); + admin.listTableDescriptors(Collections.singletonList(UTIL.getConnection().getMetaTableName())) + .get(0); TableDescriptor listByNs = admin.listTableDescriptorsByNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME).stream() .filter(td -> td.isMetaTable()).findAny().get(); @@ -144,7 +145,8 @@ public void testEditMeta() throws IOException { @Test public void testAlterMetaWithReadOnly() throws IOException { Admin admin = UTIL.getAdmin(); - TableDescriptor origMetaTableDesc = admin.getDescriptor(UTIL.getConnection().getMetaTableName()); + TableDescriptor origMetaTableDesc = + admin.getDescriptor(UTIL.getConnection().getMetaTableName()); assertFalse(origMetaTableDesc.isReadOnly()); TableDescriptor newTD = TableDescriptorBuilder.newBuilder(origMetaTableDesc).setReadOnly(true).build(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java index 5aa7cde8fb36..b5e52b7afaa4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java @@ -87,7 +87,8 @@ private void multiMutate(byte[] row, List mutations) throws IOExceptio } } MutateRowsRequest request = builder.build(); - AsyncTable table = UTIL.getAsyncConnection().getTable(UTIL.getConnection().getMetaTableName()); + AsyncTable table = + UTIL.getAsyncConnection().getTable(UTIL.getConnection().getMetaTableName()); CompletableFuture future = table. coprocessorService(MultiRowMutationService::newStub, (stub, controller, done) -> stub.mutateRows(controller, request, done), row); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java index e5bdb4ca0ca4..c2f5c90d2efa 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/AbstractTestRegionLocator.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.Region; @@ -50,7 +49,8 @@ public abstract class AbstractTestRegionLocator { protected static void startClusterAndCreateTable() throws Exception { UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(UTIL.getAdmin(), UTIL.getConnection().getMetaTableName(), REGION_REPLICATION); + HBaseTestingUtil.setReplicas(UTIL.getAdmin(), UTIL.getConnection().getMetaTableName(), + REGION_REPLICATION); TableDescriptor td = TableDescriptorBuilder.newBuilder(TABLE_NAME).setRegionReplication(REGION_REPLICATION) .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build(); @@ -173,8 +173,8 @@ private void assertMetaRegionLocation(HRegionLocation loc, int replicaId) { assertEquals(replicaId, region.getReplicaId()); ServerName expected; try { - expected = - findRegionLocation(UTIL.getConnection().getMetaTableName(), region.getStartKey(), replicaId); + expected = findRegionLocation(UTIL.getConnection().getMetaTableName(), region.getStartKey(), + replicaId); } catch (IOException e) { throw new RuntimeException(e); } @@ -192,13 +192,13 @@ private void assertMetaRegionLocations(List locs) { public void testMeta() throws IOException { assertMetaStartOrEndKeys(getStartKeys(UTIL.getConnection().getMetaTableName())); assertMetaStartOrEndKeys(getEndKeys(UTIL.getConnection().getMetaTableName())); - Pair startEndKeys = getStartEndKeys(UTIL.getConnection().getMetaTableName()); + Pair startEndKeys = + getStartEndKeys(UTIL.getConnection().getMetaTableName()); assertMetaStartOrEndKeys(startEndKeys.getFirst()); assertMetaStartOrEndKeys(startEndKeys.getSecond()); for (int replicaId = 0; replicaId < REGION_REPLICATION; replicaId++) { - assertMetaRegionLocation( - getRegionLocation(UTIL.getConnection().getMetaTableName(), HConstants.EMPTY_START_ROW, replicaId), - replicaId); + assertMetaRegionLocation(getRegionLocation(UTIL.getConnection().getMetaTableName(), + HConstants.EMPTY_START_ROW, replicaId), replicaId); } assertMetaRegionLocations( getRegionLocations(UTIL.getConnection().getMetaTableName(), HConstants.EMPTY_START_ROW)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java index 4d69ba1ff69d..42847d2c0fd6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java @@ -24,7 +24,6 @@ import java.util.HashSet; import java.util.Set; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableNameTestRule; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java index e8dd3b58e86f..8fbe6331cef7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java @@ -28,7 +28,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; @@ -47,8 +46,8 @@ private RegionReplicaTestHelper() { static void waitUntilAllMetaReplicasAreReady(HBaseTestingUtil util, ConnectionRegistry registry) throws IOException { Configuration conf = util.getConfiguration(); - int regionReplicaCount = - util.getAdmin().getDescriptor(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).getRegionReplication(); + int regionReplicaCount = util.getAdmin() + .getDescriptor(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).getRegionReplication(); Waiter.waitFor(conf, conf.getLong("hbase.client.sync.wait.timeout.msec", 60000), 200, true, new ExplainingPredicate() { @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java index 17912ec15354..f6902084bdca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index 30521b350800..2211048384a4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.ClusterMetrics.Option; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableExistsException; @@ -85,7 +84,8 @@ public class TestAdmin2 extends TestAdminBase { public void testCreateBadTables() throws IOException { String msg = null; try { - ADMIN.createTable(TableDescriptorBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).build()); + ADMIN.createTable(TableDescriptorBuilder + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).build()); } catch (TableExistsException e) { msg = e.toString(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java index bbf784144cdb..fca0fd1392d4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java @@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -55,7 +54,8 @@ public class TestAsyncAdminWithRegionReplicas extends TestAsyncAdminBase { @BeforeClass public static void setUpBeforeClass() throws Exception { TestAsyncAdminBase.setUpBeforeClass(); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), RegionInfoBuilder.FIRST_META_REGIONINFO.getTable(), 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), + RegionInfoBuilder.FIRST_META_REGIONINFO.getTable(), 3); try (ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent())) { RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, registry); @@ -139,7 +139,8 @@ public void testCloneTableSchema() throws IOException, InterruptedException, Exe @Test public void testGetTableRegions() throws InterruptedException, ExecutionException, IOException { - List metaRegions = admin.getRegions(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).get(); + List metaRegions = + admin.getRegions(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).get(); assertEquals(3, metaRegions.size()); for (int i = 0; i < 3; i++) { RegionInfo metaRegion = metaRegions.get(i); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java index c503e133fb6f..23c11a5c8931 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java @@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MatcherPredicate; - import org.apache.hadoop.hbase.MiniClusterRule; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.StartTestingClusterOption; @@ -164,8 +163,8 @@ public void test() throws Exception { TraceUtil.trace(() -> { try { - testLocator(miniClusterRule.getTestingUtility(), testUtil.getConnection().getMetaTableName(), - new Locator() { + testLocator(miniClusterRule.getTestingUtility(), + testUtil.getConnection().getMetaTableName(), new Locator() { @Override public void updateCachedLocationOnError(HRegionLocation loc, Throwable error) { locator.updateCachedLocationOnError(loc, error); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java index 33bf046ee00f..1879011bafd5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java @@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; @@ -107,10 +106,10 @@ public static void setUp() throws Exception { admin.balancerSwitch(false, true); // Enable hbase:meta replication. - HBaseTestingUtil.setReplicas(admin, TEST_UTIL.getConnection().getMetaTableName(), NUM_OF_META_REPLICA); - TEST_UTIL.waitFor(30000, - () -> TEST_UTIL.getMiniHBaseCluster().getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() - >= NUM_OF_META_REPLICA); + HBaseTestingUtil.setReplicas(admin, TEST_UTIL.getConnection().getMetaTableName(), + NUM_OF_META_REPLICA); + TEST_UTIL.waitFor(30000, () -> TEST_UTIL.getMiniHBaseCluster() + .getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() >= NUM_OF_META_REPLICA); SPLIT_KEYS = new byte[8][]; for (int i = 111; i < 999; i += 111) { @@ -130,8 +129,8 @@ public void setUpBeforeTest() throws InterruptedException, ExecutionException, I c.set(RegionLocator.LOCATOR_META_REPLICAS_MODE, metaReplicaMode.toString()); ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); - conn = - new AsyncConnectionImpl(c, registry, registry.getClusterId().get(), TableName.valueOf("hbase:meta"), null, User.getCurrent()); + conn = new AsyncConnectionImpl(c, registry, registry.getClusterId().get(), + TableName.valueOf("hbase:meta"), null, User.getCurrent()); locator = new AsyncNonMetaRegionLocator(conn, AsyncConnectionImpl.RETRY_TIMER); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java index e7e944fcd232..22ebf837ea30 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil; @@ -360,7 +359,8 @@ public void testTruncateReplicaRegionNotAllowed() throws Exception { @Test public void testTruncateRegionsMetaTableRegionsNotAllowed() throws Exception { - AsyncTableRegionLocator locator = ASYNC_CONN.getRegionLocator(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); + AsyncTableRegionLocator locator = + ASYNC_CONN.getRegionLocator(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); List regionLocations = locator.getAllRegionLocations().get(); HRegionLocation regionToBeTruncated = regionLocations.get(0); // 1 diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java index 8f82df838af7..547691c78c6b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java @@ -165,8 +165,8 @@ void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) { } }; try (AsyncConnectionImpl mockedConn = new AsyncConnectionImpl(CONN.getConfiguration(), - CONN.registry, CONN.registry.getClusterId().get(), - TableName.valueOf("hbase:meta"), null, User.getCurrent()) { + CONN.registry, CONN.registry.getClusterId().get(), TableName.valueOf("hbase:meta"), null, + User.getCurrent()) { @Override AsyncRegionLocator getLocator() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java index e044afdab239..a742b03e95dc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java @@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; @@ -73,7 +72,8 @@ public void testCreateTable() throws Exception { static TableState.State getStateFromMeta(TableName table) throws Exception { Optional state = ClientMetaTableAccessor - .getTableState(ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()), table).get(); + .getTableState(ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()), table) + .get(); assertTrue(state.isPresent()); return state.get().getState(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java index 8c3628346987..1c1c54f59df0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java @@ -26,7 +26,6 @@ import java.util.Set; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.testclassification.ClientTests; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java index 4b6d9e43056e..e7f64055c29a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.testclassification.ClientTests; @@ -126,7 +125,8 @@ public void testListTables() throws Exception { assertTrue("tableName should be equal in order", tableDescs.get(j).getTableName().equals(tables[i])); } - assertTrue(tableDescs.get(size - 1).getTableName().equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable())); + assertTrue(tableDescs.get(size - 1).getTableName() + .equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable())); for (int i = 0; i < tables.length; i++) { admin.disableTable(tables[i]).join(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java index 65a0dc22f69d..ea8200995d8b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableUseMetaReplicas.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.ObserverContext; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java index 41035c757388..2416cd0b0e2b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java @@ -27,7 +27,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.security.User; @@ -72,10 +71,10 @@ public static void setUp() throws Exception { admin.balancerSwitch(false, true); // Enable hbase:meta replication. - HBaseTestingUtil.setReplicas(admin, TEST_UTIL.getConnection().getMetaTableName(), numOfMetaReplica); - TEST_UTIL.waitFor(30000, - () -> TEST_UTIL.getMiniHBaseCluster().getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() - >= numOfMetaReplica); + HBaseTestingUtil.setReplicas(admin, TEST_UTIL.getConnection().getMetaTableName(), + numOfMetaReplica); + TEST_UTIL.waitFor(30000, () -> TEST_UTIL.getMiniHBaseCluster() + .getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() >= numOfMetaReplica); registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); CONN = new AsyncConnectionImpl(conf, registry, registry.getClusterId().get(), @@ -94,19 +93,19 @@ public void testMetaChangeFromReplicaNoReplica() throws IOException, Interrupted CONN.getConfiguration().get(RegionLocator.LOCATOR_META_REPLICAS_MODE_LOADBALANCE_SELECTOR, CatalogReplicaLoadBalanceSimpleSelector.class.getName()); - CatalogReplicaLoadBalanceSelector metaSelector = CatalogReplicaLoadBalanceSelectorFactory - .createSelector(replicaSelectorClass, TEST_UTIL.getConnection().getMetaTableName(), CONN, () -> { - int numOfReplicas = CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS; - try { - RegionLocations metaLocations = CONN.registry.getMetaRegionLocations() - .get(CONN.connConf.getMetaReadRpcTimeoutNs(), TimeUnit.NANOSECONDS); - numOfReplicas = metaLocations.size(); - } catch (Exception e) { - LOG.error("Failed to get table meta table's region replication, ", - e); - } - return numOfReplicas; - }); + CatalogReplicaLoadBalanceSelector metaSelector = + CatalogReplicaLoadBalanceSelectorFactory.createSelector(replicaSelectorClass, + TEST_UTIL.getConnection().getMetaTableName(), CONN, () -> { + int numOfReplicas = CatalogReplicaLoadBalanceSelector.UNINITIALIZED_NUM_OF_REPLICAS; + try { + RegionLocations metaLocations = CONN.registry.getMetaRegionLocations() + .get(CONN.connConf.getMetaReadRpcTimeoutNs(), TimeUnit.NANOSECONDS); + numOfReplicas = metaLocations.size(); + } catch (Exception e) { + LOG.error("Failed to get table meta table's region replication, ", e); + } + return numOfReplicas; + }); // Loop for 100 times, it should cover all replica ids. int[] replicaIdCount = new int[numOfMetaReplica]; @@ -118,8 +117,8 @@ public void testMetaChangeFromReplicaNoReplica() throws IOException, Interrupted // Change to No meta replica HBaseTestingUtil.setReplicas(admin, TEST_UTIL.getConnection().getMetaTableName(), 1); - TEST_UTIL.waitFor(30000, - () -> TEST_UTIL.getMiniHBaseCluster().getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() == 1); + TEST_UTIL.waitFor(30000, () -> TEST_UTIL.getMiniHBaseCluster() + .getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() == 1); CatalogReplicaLoadBalanceSelector metaSelectorWithNoReplica = CatalogReplicaLoadBalanceSelectorFactory.createSelector(replicaSelectorClass, @@ -130,8 +129,7 @@ public void testMetaChangeFromReplicaNoReplica() throws IOException, Interrupted .get(CONN.connConf.getMetaReadRpcTimeoutNs(), TimeUnit.NANOSECONDS); numOfReplicas = metaLocations.size(); } catch (Exception e) { - LOG.error("Failed to get table meta table's region replication, ", - e); + LOG.error("Failed to get table meta table's region replication, ", e); } return numOfReplicas; }); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java index f118e2ce746f..c2431862473a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java @@ -22,7 +22,6 @@ import java.util.List; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; @@ -48,7 +47,8 @@ public void testReplicaCleanup() throws Exception { ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher(); List metaReplicaZnodes = zkw.getMetaReplicaNodes(); assertEquals(3, metaReplicaZnodes.size()); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), RegionInfoBuilder.FIRST_META_REGIONINFO.getTable(), 1); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), + RegionInfoBuilder.FIRST_META_REGIONINFO.getTable(), 1); metaReplicaZnodes = zkw.getMetaReplicaNodes(); assertEquals(1, metaReplicaZnodes.size()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java index b6f386cd93b2..e46ae64ad24d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientScannerTimeouts.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.SingleProcessHBaseCluster.MiniHBaseClusterRegionServer; import org.apache.hadoop.hbase.TableName; @@ -455,7 +454,9 @@ public ScanResponse scan(final RpcController controller, final ScanRequest reque ScanResponse scanRes = super.scan(controller, request); String regionName = Bytes.toString(request.getRegion().getValue().toByteArray()); try { - if (!regionName.contains(TEST_UTIL.getConnection().getMetaTableName().getNameAsString())) { + if ( + !regionName.contains(TEST_UTIL.getConnection().getMetaTableName().getNameAsString()) + ) { tableScannerId = scanRes.getScannerId(); if (sleepOnOpen) { try { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java index 7845a9cf89ff..8f6a41e3a75d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientSideRegionScanner.java @@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.client.metrics.ScanMetricsRegionInfo; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java index 05ed229eeccb..7f23390390f5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestEnableTable.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java index cc35cd52dba6..65410ed51ff9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide5.java @@ -48,7 +48,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.KeyValue; - import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java index e73baee5dfc2..0f5c0d30907c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java @@ -21,7 +21,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.testclassification.MediumTests; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java index e0a681182c94..398c4338140c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMasterRegistry.java @@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.Waiter; @@ -61,7 +60,8 @@ public static void setUp() throws Exception { StartTestingClusterOption.Builder builder = StartTestingClusterOption.builder(); builder.numMasters(3).numRegionServers(3); TEST_UTIL.startMiniCluster(builder.build()); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TEST_UTIL.getConnection().getMetaTableName(), 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TEST_UTIL.getConnection().getMetaTableName(), + 3); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java index 3b39bee7ec85..f8a97a8bef16 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaCache.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.MultiActionResultTooLarge; import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.RegionTooBusyException; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java index 588324487acb..4df81f3995c6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaRegionLocationCache; - import org.apache.hadoop.hbase.MultithreadedTestUtil; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.master.HMaster; @@ -63,7 +62,8 @@ public class TestMetaRegionLocationCache { @BeforeClass public static void setUp() throws Exception { TEST_UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TEST_UTIL.getConnection().getMetaTableName(), 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TEST_UTIL.getConnection().getMetaTableName(), + 3); REGISTRY = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, REGISTRY); TEST_UTIL.getAdmin().balancerSwitch(false, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java index 0cf228c6065e..758a5e915184 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java @@ -24,7 +24,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -53,8 +52,8 @@ public static void setUp() throws Exception { @Test public void testMetaHTDReplicaCount() throws Exception { - assertEquals(3, - TEST_UTIL.getAdmin().getDescriptor(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).getRegionReplication()); + assertEquals(3, TEST_UTIL.getAdmin() + .getDescriptor(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).getRegionReplication()); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java index 3fc3fe62698e..7e9e89798ee6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasShutdownHandling.java @@ -28,7 +28,6 @@ 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.regionserver.StorefileRefresherChore; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java index c631fa11c65e..387152ebe302 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiActionMetricsFromClient.java @@ -22,7 +22,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java index 6af93a59ec39..e6dfc359b2c1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java @@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java index eaeabeeeac46..f9eec4c4f1f6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestRpcConnectionRegistry.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.regionserver.BootstrapNodeManager; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java index 5e94df783d54..db1bf6511d58 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.StartTestingClusterOption; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java index b0cc3dc20cf3..005786bf603e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java @@ -20,7 +20,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; @@ -50,7 +49,8 @@ public void testShutdownOfReplicaHolder() throws Exception { // checks that the when the server holding meta replica is shut down, the meta replica // can be recovered try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration()); - RegionLocator locator = conn.getRegionLocator(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable())) { + RegionLocator locator = + conn.getRegionLocator(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable())) { HRegionLocation hrl = locator.getRegionLocations(HConstants.EMPTY_START_ROW, true).get(1); ServerName oldServer = hrl.getServerName(); TEST_UTIL.getHBaseClusterInterface().killRegionServer(oldServer); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java index da685d24a14e..deca00e04086 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromClient.java @@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.TableNotFoundException; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java index 354811d41d3a..2ca6ad47a875 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -64,7 +63,8 @@ public class TestZKConnectionRegistry { @BeforeClass public static void setUp() throws Exception { TEST_UTIL.startMiniCluster(3); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TEST_UTIL.getConnection().getMetaTableName(), 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TEST_UTIL.getConnection().getMetaTableName(), + 3); REGISTRY = new ZKConnectionRegistry(TEST_UTIL.getConfiguration(), null); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java index ce86a3e31e59..e4b815cf8364 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.LocalHBaseCluster; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java index 6e6e5f81cb60..ee52ebe04f71 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.PleaseHoldException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java index 8cfceb28919e..69561c105db4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFileSystemWithStoreFileTracking.java @@ -23,7 +23,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MediumTests; @@ -64,7 +63,8 @@ public static void teardownTest() throws Exception { @Test public void tesMetaDescriptorHasSFTConfig() throws Exception { - TableDescriptor descriptor = UTIL.getAdmin().getDescriptor(UTIL.getConnection().getMetaTableName()); + TableDescriptor descriptor = + UTIL.getAdmin().getDescriptor(UTIL.getConnection().getMetaTableName()); assertEquals(FILE.name(), descriptor.getValue(TRACKER_IMPL)); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java index 7957502a96d3..ac07aeb7d535 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterOperationsForRegionReplicas.java @@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java index 91398b310425..560580e09668 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterRepairMode.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AsyncTable; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java index 0fe4b499ad4c..d76aa14693bb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterTransitions.java @@ -22,7 +22,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java index 13821417282c..d0a95e611def 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaAssignmentWithStopMaster.java @@ -22,7 +22,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.client.Connection; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java index 7f42783f3d28..161952a66813 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateAndMirrorMetaLocations.java @@ -142,7 +142,8 @@ public void test() throws Exception { } // wait until all meta regions have been assigned UTIL.waitFor(30000, - () -> UTIL.getMiniHBaseCluster().getRegions(UTIL.getConnection().getMetaTableName()).size() == 2); + () -> UTIL.getMiniHBaseCluster().getRegions(UTIL.getConnection().getMetaTableName()).size() + == 2); // make sure all the SCPs are finished waitUntilNoSCP(); checkMirrorLocation(2); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java index 268791a1a294..2d575d6043a1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMigrateNamespaceTable.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java index 6e2c41648e91..e50ca9853009 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRecreateCluster.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.StartTestingClusterOption; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java index 8f1df662f7b9..8dad0a84b4e9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRestartWithEmptyWALDirectory.java @@ -24,7 +24,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java index a3f0f3940cdc..45cfa62d5c15 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestServerCrashProcedureCarryingMetaStuck.java @@ -21,7 +21,6 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.client.AsyncAdmin; import org.apache.hadoop.hbase.client.AsyncConnection; import org.apache.hadoop.hbase.client.ConnectionFactory; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java index 578767d86348..9bda3d92266a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java @@ -21,7 +21,6 @@ import java.util.concurrent.CountDownLatch; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.ProcedureTestUtil; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -152,10 +151,12 @@ public static void setUp() throws Exception { UTIL.getAdmin().balancerSwitch(false, true); HRegionServer srcRs = UTIL.getRSForFirstRegionInTable(TABLE_NAME); if (!srcRs.getRegions(UTIL.getConnection().getMetaTableName()).isEmpty()) { - RegionInfo metaRegion = srcRs.getRegions(UTIL.getConnection().getMetaTableName()).get(0).getRegionInfo(); + RegionInfo metaRegion = + srcRs.getRegions(UTIL.getConnection().getMetaTableName()).get(0).getRegionInfo(); HRegionServer dstRs = UTIL.getOtherRegionServer(srcRs); UTIL.getAdmin().move(metaRegion.getEncodedNameAsBytes(), dstRs.getServerName()); - UTIL.waitFor(30000, () -> !dstRs.getRegions(UTIL.getConnection().getMetaTableName()).isEmpty()); + UTIL.waitFor(30000, + () -> !dstRs.getRegions(UTIL.getConnection().getMetaTableName()).isEmpty()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java index e5d2f55af6d4..db6040926fc8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestOpenRegionProcedureBackoff.java @@ -25,7 +25,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.ProcedureTestUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.AsyncAdmin; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java index b78a32ddc951..469def163922 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java @@ -37,7 +37,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java index b2e21486a388..d58e96d4bfe8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRollbackSCP.java @@ -29,7 +29,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; @@ -168,7 +167,8 @@ public void describeTo(Description description) { @Test public void testFailAndRollback() throws Exception { - HRegionServer rsWithMeta = UTIL.getRSForFirstRegionInTable(UTIL.getConnection().getMetaTableName()); + HRegionServer rsWithMeta = + UTIL.getRSForFirstRegionInTable(UTIL.getConnection().getMetaTableName()); UTIL.getMiniHBaseCluster().killRegionServer(rsWithMeta.getServerName()); UTIL.waitFor(15000, () -> getSCPForServer(rsWithMeta.getServerName()) != null); ServerCrashProcedure scp = getSCPForServer(rsWithMeta.getServerName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java index 088567e24f0a..4ea9450cdea1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java @@ -29,7 +29,6 @@ import java.util.stream.Collectors; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java index 9ed292445c1a..92dda7532131 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestFavoredStochasticLoadBalancer.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java index e85165bca4ac..4ae6615678a7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestMetaTableIsolationBalancerConditional.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -150,8 +149,8 @@ private static void validateRegionLocations(Map> tabl // Validate that the region assignments ServerName metaServer; try { - metaServer = - tableToServers.get(TEST_UTIL.getConnection().getMetaTableName()).stream().findFirst().orElseThrow(); + metaServer = tableToServers.get(TEST_UTIL.getConnection().getMetaTableName()).stream() + .findFirst().orElseThrow(); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java index d6e4b962f3bf..0e717d6e3b74 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestReplicationBarrierCleaner.java @@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Delete; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java index 406ac78e2c03..8f8d872c7164 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestCatalogJanitorInMemoryStates.java @@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaMockingUtil; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNameTestRule; import org.apache.hadoop.hbase.Waiter; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java index fab54d8d0d91..37d4fbd1aeee 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/janitor/TestMetaFixerNoCluster.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.SortedSet; import org.apache.hadoop.hbase.HBaseClassTestRule; - import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.testclassification.MasterTests; @@ -49,18 +48,22 @@ public class TestMetaFixerNoCluster { private static byte[] C = Bytes.toBytes("c"); private static byte[] D = Bytes.toBytes("d"); private static RegionInfo ALL = RegionInfoBuilder.FIRST_META_REGIONINFO; - private static RegionInfo _ARI = - RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setEndKey(A).build(); - private static RegionInfo _BRI = - RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setEndKey(B).build(); + private static RegionInfo _ARI = RegionInfoBuilder + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setEndKey(A).build(); + private static RegionInfo _BRI = RegionInfoBuilder + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setEndKey(B).build(); private static RegionInfo ABRI = - RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(A).setEndKey(B).build(); + RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(A) + .setEndKey(B).build(); private static RegionInfo ACRI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(A).setEndKey(C).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(A).setEndKey(C) + .build(); private static RegionInfo CDRI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(C).setEndKey(D).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(C).setEndKey(D) + .build(); private static RegionInfo ADRI = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(A).setEndKey(D).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(A).setEndKey(D) + .build(); private static RegionInfo D_RI = org.apache.hadoop.hbase.client.RegionInfoBuilder .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(D).build(); private static RegionInfo C_RI = org.apache.hadoop.hbase.client.RegionInfoBuilder diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java index c650dd6511f6..72e7fb89c0f9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedurePriority.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.client.AsyncAdmin; @@ -147,15 +146,14 @@ public static void tearDown() throws Exception { @Test public void test() throws Exception { - RegionServerThread rsWithMetaThread = UTIL.getMiniHBaseCluster().getRegionServerThreads() - .stream().filter(t -> { + RegionServerThread rsWithMetaThread = + UTIL.getMiniHBaseCluster().getRegionServerThreads().stream().filter(t -> { try { return !t.getRegionServer().getRegions(UTIL.getConnection().getMetaTableName()).isEmpty(); } catch (IOException e) { throw new RuntimeException(e); } - }) - .findAny().get(); + }).findAny().get(); HRegionServer rsNoMeta = UTIL.getOtherRegionServer(rsWithMetaThread.getRegionServer()); FAIL = true; UTIL.getMiniHBaseCluster().killRegionServer(rsNoMeta.getServerName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java index 14f3b0cd5152..09e5f762ee94 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestTableProcedureWaitingQueueCleanup.java @@ -22,7 +22,6 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java index 7f76dda62f0d..693e1981ba34 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.YouAreDeadException; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java index e979fe1201fa..71dd054747cc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java @@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueTestUtil; import org.apache.hadoop.hbase.KeyValueUtil; - import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java index 7eae181ea0f7..22cf84e9100c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java @@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.Stoppable; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java index a5b7a583c793..5ff46259d505 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java @@ -32,7 +32,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Delete; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java index 1f3232dccffb..eede4752a789 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtil; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -78,8 +77,7 @@ public void testReadAndWriteRegionInfoFile() throws IOException, InterruptedExce long modtime = getModTime(r); HBaseTestingUtil.closeRegionAndWAL(r); Thread.sleep(1001); - r = HRegion.openHRegion(ROOT_DIR, ri, fsTableDescriptors.get(ri.getTable()), null, - CONF); + r = HRegion.openHRegion(ROOT_DIR, ri, fsTableDescriptors.get(ri.getTable()), null, CONF); // Ensure the file is not written for a second time. long modtime2 = getModTime(r); assertEquals(modtime, modtime2); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java index 3f8c2f13886f..9c83ec9e1d4b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java @@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -72,7 +71,8 @@ public class TestRegionInfo { public void testIsStart() { assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isFirst()); org.apache.hadoop.hbase.client.RegionInfo ri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(Bytes.toBytes("not_start")).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()) + .setStartKey(Bytes.toBytes("not_start")).build(); assertFalse(ri.isFirst()); } @@ -80,7 +80,8 @@ public void testIsStart() { public void testIsEnd() { assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isFirst()); org.apache.hadoop.hbase.client.RegionInfo ri = org.apache.hadoop.hbase.client.RegionInfoBuilder - .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setEndKey(Bytes.toBytes("not_end")).build(); + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()) + .setEndKey(Bytes.toBytes("not_end")).build(); assertFalse(ri.isLast()); } @@ -105,14 +106,17 @@ public void testIsOverlap() { org.apache.hadoop.hbase.client.RegionInfo ari = org.apache.hadoop.hbase.client.RegionInfoBuilder .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setEndKey(a).build(); org.apache.hadoop.hbase.client.RegionInfo abri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()) - .setStartKey(a).setEndKey(b).build(); + org.apache.hadoop.hbase.client.RegionInfoBuilder + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(a).setEndKey(b) + .build(); org.apache.hadoop.hbase.client.RegionInfo adri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()) - .setStartKey(a).setEndKey(d).build(); + org.apache.hadoop.hbase.client.RegionInfoBuilder + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(a).setEndKey(d) + .build(); org.apache.hadoop.hbase.client.RegionInfo cdri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()) - .setStartKey(c).setEndKey(d).build(); + org.apache.hadoop.hbase.client.RegionInfoBuilder + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(c).setEndKey(d) + .build(); org.apache.hadoop.hbase.client.RegionInfo dri = org.apache.hadoop.hbase.client.RegionInfoBuilder .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(d).build(); assertTrue(all.isOverlap(all)); @@ -143,16 +147,19 @@ public void testIsOverlaps() { org.apache.hadoop.hbase.client.RegionInfo ari = org.apache.hadoop.hbase.client.RegionInfoBuilder .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setEndKey(a).build(); org.apache.hadoop.hbase.client.RegionInfo abri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()) - .setStartKey(a).setEndKey(b).build(); + org.apache.hadoop.hbase.client.RegionInfoBuilder + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(a).setEndKey(b) + .build(); org.apache.hadoop.hbase.client.RegionInfo eri = org.apache.hadoop.hbase.client.RegionInfoBuilder .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setEndKey(e).build(); org.apache.hadoop.hbase.client.RegionInfo cdri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()) - .setStartKey(c).setEndKey(d).build(); + org.apache.hadoop.hbase.client.RegionInfoBuilder + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(c).setEndKey(d) + .build(); org.apache.hadoop.hbase.client.RegionInfo efri = - org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()) - .setStartKey(e).setEndKey(f).build(); + org.apache.hadoop.hbase.client.RegionInfoBuilder + .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).setStartKey(e).setEndKey(f) + .build(); assertFalse(ari.isOverlap(abri)); assertTrue(abri.isOverlap(eri)); assertFalse(cdri.isOverlap(efri)); @@ -181,7 +188,8 @@ public void testReadAndWriteHRegionInfoFile() throws IOException, InterruptedExc long modtime = getModTime(r); HBaseTestingUtil.closeRegionAndWAL(r); Thread.sleep(1001); - r = HRegion.openHRegion(basedir, hri, fsTableDescriptors.get(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()), null, + r = HRegion.openHRegion(basedir, hri, + fsTableDescriptors.get(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()), null, htu.getConfiguration()); // Ensure the file is not written for a second time. long modtime2 = getModTime(r); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java index 77a2bb71e988..ffc0d5cd157a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TestMetaTableAccessor; import org.apache.hadoop.hbase.client.Consistency; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java index 18e8cb69d66d..e7f16d7dc666 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerCrashDisableWAL.java @@ -22,7 +22,6 @@ import java.io.IOException; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java index 22b4d025ce38..1122ad8e8096 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java @@ -23,7 +23,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.NotServingRegionException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -97,13 +96,13 @@ public static void stopMasterAndCacheMetaLocation(HBaseTestingUtil HTU) throws IOException, InterruptedException { // cache meta location, so we will not go to master to lookup meta region location for (JVMClusterUtil.RegionServerThread t : HTU.getMiniHBaseCluster().getRegionServerThreads()) { - try (RegionLocator locator = - t.getRegionServer().getConnection().getRegionLocator(HTU.getConnection().getMetaTableName())) { + try (RegionLocator locator = t.getRegionServer().getConnection() + .getRegionLocator(HTU.getConnection().getMetaTableName())) { locator.getAllRegionLocations(); } } - try ( - RegionLocator locator = HTU.getConnection().getRegionLocator(HTU.getConnection().getMetaTableName())) { + try (RegionLocator locator = + HTU.getConnection().getRegionLocator(HTU.getConnection().getMetaTableName())) { locator.getAllRegionLocations(); } // Stop master diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java index 97532aadae54..b1c3fb02691d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerRejectDuringAbort.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.StartTestingClusterOption; import org.apache.hadoop.hbase.TableName; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java index 924b0bc98c85..6e938755c5da 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestShutdownWhileWALBroken.java @@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.ExplainingPredicate; import org.apache.hadoop.hbase.YouAreDeadException; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java index c3750ba66e39..2f41ec9a51cf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestLogRolling.java @@ -31,7 +31,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.StartTestingClusterOption; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java index 1d1ed79fbdf6..cf4d61cf3a3a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java index 48764dc2213c..66ce18e0b847 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java @@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java index 91c65907bfa9..4d7548da277a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; - import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; @@ -95,7 +94,8 @@ public void testContendedLogRolling() throws Exception { CommonFSUtils.setRootDir(conf, dir); FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(TEST_UTIL.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration()); - TableDescriptor metaTableDescriptor = fsTableDescriptors.get(TEST_UTIL.getConnection().getMetaTableName()); + TableDescriptor metaTableDescriptor = + fsTableDescriptors.get(TEST_UTIL.getConnection().getMetaTableName()); conf.set(FSHLogProvider.WRITER_IMPL, HighLatencySyncWriter.class.getName()); final WALFactory wals = new WALFactory(conf, TestLogRollingNoCluster.class.getName()); final WAL wal = wals.getWAL(null); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java index 79e40861f7ed..78a299c1e35e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.security.access.PermissionStorage; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java index a50241c93e25..2579d8d9c1ad 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestMetaRegionReplicaReplication.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; @@ -99,7 +98,8 @@ public void before() throws Exception { // conf.setInt(HConstants.META_REPLICAS_NUM, numOfMetaReplica); HTU.startMiniCluster(NB_SERVERS); // Enable hbase:meta replication. - HBaseTestingUtil.setReplicas(HTU.getAdmin(), HTU.getConnection().getMetaTableName(), numOfMetaReplica); + HBaseTestingUtil.setReplicas(HTU.getAdmin(), HTU.getConnection().getMetaTableName(), + numOfMetaReplica); HTU.waitFor(30000, () -> HTU.getMiniHBaseCluster().getRegions(HTU.getConnection().getMetaTableName()).size() @@ -130,7 +130,8 @@ public void testHBaseMetaReplicates() throws Exception { getMetaCells(table.getName())); // Try delete. HTU.deleteTableIfAny(table.getName()); - verifyDeletedReplication(HTU.getConnection().getMetaTableName(), numOfMetaReplica, table.getName()); + verifyDeletedReplication(HTU.getConnection().getMetaTableName(), numOfMetaReplica, + table.getName()); } } @@ -417,7 +418,8 @@ private void getMetaReplicaReadRequests(final Region[] metaRegions, final long[] @Test public void testHBaseMetaReplicaGets() throws Exception { TableName tn = TableName.valueOf(this.name.getMethodName()); - final Region[] metaRegions = getAllRegions(HTU.getConnection().getMetaTableName(), numOfMetaReplica); + final Region[] metaRegions = + getAllRegions(HTU.getConnection().getMetaTableName(), numOfMetaReplica); long[] readReqsForMetaReplicas = new long[numOfMetaReplica]; long[] readReqsForMetaReplicasAfterGet = new long[numOfMetaReplica]; long[] readReqsForMetaReplicasAfterGetAllLocations = new long[numOfMetaReplica]; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java index 1da006fc0055..b4274ed0520d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java @@ -45,7 +45,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; - import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; @@ -196,8 +195,8 @@ public void testWALEntryFilter() throws IOException { new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, TableName.valueOf("test"), -1, -1, uuid), we); assertTrue(wef.filter(e) == e); // Test system WAL edit. - e = new WAL.Entry( - new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, TEST_UTIL.getConnection().getMetaTableName(), -1, -1, uuid), we); + e = new WAL.Entry(new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, + TEST_UTIL.getConnection().getMetaTableName(), -1, -1, uuid), we); assertNull(wef.filter(e)); } finally { rs.terminate("Done"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java index 31e433aac380..4fa03854d61e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Connection; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java index 398fd317f415..ccf1729ba24a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsKillRS.java @@ -29,7 +29,6 @@ import java.util.Set; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java index 69d2c63985c7..414d84527485 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestRpcAccessChecks.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -237,8 +236,8 @@ public void testRunCleanerChore() throws Exception { public void testExecProcedure() throws Exception { verifyAdminCheckForAction((admin) -> { // Using existing table instead of creating a new one. - admin.execProcedure("flush-table-proc", TEST_UTIL.getConnection().getMetaTableName().getNameAsString(), - new HashMap<>()); + admin.execProcedure("flush-table-proc", + TEST_UTIL.getConnection().getMetaTableName().getNameAsString(), new HashMap<>()); }); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java index 0dc9c76914f6..d4cff355a050 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/token/TestGenerateDelegationToken.java @@ -29,7 +29,6 @@ import java.util.Collection; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.client.AsyncConnection; import org.apache.hadoop.hbase.client.AsyncTable; import org.apache.hadoop.hbase.client.Connection; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java index c77e6afad56f..9f262a73fc46 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/snapshot/TestRegionSnapshotTask.java @@ -33,7 +33,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptor; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java index 3e3784835f81..2c360f74db76 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java @@ -41,7 +41,6 @@ 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; @@ -199,8 +198,8 @@ protected void deleteRegion(Configuration conf, final TableDescriptor htd, byte[ } if (metaRow) { - try ( - Table meta = connection.getTable(TEST_UTIL.getConnection().getMetaTableName(), tableExecutorService)) { + try (Table meta = connection.getTable(TEST_UTIL.getConnection().getMetaTableName(), + tableExecutorService)) { Delete delete = new Delete(deleteRow); meta.delete(delete); } @@ -512,8 +511,9 @@ public boolean tableHasErrors(HbckTableInfo table) { protected void deleteMetaRegion(Configuration conf, boolean unassign, boolean hdfs, boolean regionInfoOnly) throws IOException, InterruptedException { - HRegionLocation metaLocation = connection.getRegionLocator(TEST_UTIL.getConnection().getMetaTableName()) - .getRegionLocation(HConstants.EMPTY_START_ROW); + HRegionLocation metaLocation = + connection.getRegionLocator(TEST_UTIL.getConnection().getMetaTableName()) + .getRegionLocation(HConstants.EMPTY_START_ROW); ServerName hsa = metaLocation.getServerName(); RegionInfo hri = metaLocation.getRegion(); if (unassign) { @@ -527,8 +527,9 @@ protected void deleteMetaRegion(Configuration conf, boolean unassign, boolean hd LOG.info("deleting hdfs .regioninfo data: " + hri.toString() + hsa.toString()); Path rootDir = CommonFSUtils.getRootDir(conf); FileSystem fs = rootDir.getFileSystem(conf); - Path p = new Path(rootDir + "/" + TEST_UTIL.getConnection().getMetaTableName().getNameAsString(), - hri.getEncodedName()); + Path p = + new Path(rootDir + "/" + TEST_UTIL.getConnection().getMetaTableName().getNameAsString(), + hri.getEncodedName()); Path hriPath = new Path(p, HRegionFileSystem.REGION_INFO_FILE); fs.delete(hriPath, true); } @@ -537,8 +538,9 @@ protected void deleteMetaRegion(Configuration conf, boolean unassign, boolean hd LOG.info("deleting hdfs data: " + hri.toString() + hsa.toString()); Path rootDir = CommonFSUtils.getRootDir(conf); FileSystem fs = rootDir.getFileSystem(conf); - Path p = new Path(rootDir + "/" + TEST_UTIL.getConnection().getMetaTableName().getNameAsString(), - hri.getEncodedName()); + Path p = + new Path(rootDir + "/" + TEST_UTIL.getConnection().getMetaTableName().getNameAsString(), + hri.getEncodedName()); HBaseFsck.debugLsr(conf, p); boolean success = fs.delete(p, true); LOG.info("Deleted " + p + " sucessfully? " + success); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java index a158ebdfc4e4..f4c698c2e392 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java @@ -36,7 +36,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseCommonTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java index ad43a13eef49..63a49f69ab8f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckCleanReplicationBarriers.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.MetaTableAccessor; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java index d550dafaac7e..68337ddbe35b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMover1.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.SingleProcessHBaseCluster; import org.apache.hadoop.hbase.TableName; @@ -287,8 +286,7 @@ public void testRegionServerPort() throws Exception { @Test public void testLoadMetaRegion() throws Exception { HRegionServer rsWithMeta = TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads().stream() - .map(t -> t.getRegionServer()) - .filter(rs -> { + .map(t -> t.getRegionServer()).filter(rs -> { try { return rs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() > 0; } catch (IOException e) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java index 71c5c2de8ce4..cff8b0336787 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestRegionMoverWithRSGroupEnable.java @@ -27,7 +27,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -87,15 +86,13 @@ public void setUp() throws Exception { Collection allServers = admin.getRegionServers(); // Remove rs contains hbase:meta, otherwise test looks unstable and buggy in test env. ServerName rsContainMeta = TEST_UTIL.getMiniHBaseCluster().getRegionServerThreads().stream() - .map(t -> t.getRegionServer()) - .filter(rs -> { + .map(t -> t.getRegionServer()).filter(rs -> { try { return rs.getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() > 0; } catch (IOException e) { throw new RuntimeException(e); } - }).findFirst().get() - .getServerName(); + }).findFirst().get().getServerName(); LOG.info("{} contains hbase:meta", rsContainMeta); List modifiable = new ArrayList<>(allServers); modifiable.remove(rsContainMeta); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java index 5d7c06e80e8d..c887fa4f4cb1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java @@ -59,7 +59,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; @@ -470,8 +469,8 @@ public void testOldRecoveredEditsFileSidelined() throws IOException { private Path createRecoveredEditsPathForRegion() throws IOException { byte[] encoded = RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(); - Path p = WALSplitUtil.getRegionSplitEditsPath(TEST_UTIL.getConnection().getMetaTableName(), encoded, 1, - FILENAME_BEING_SPLIT, TMPDIRNAME, conf, ""); + Path p = WALSplitUtil.getRegionSplitEditsPath(TEST_UTIL.getConnection().getMetaTableName(), + encoded, 1, FILENAME_BEING_SPLIT, TMPDIRNAME, conf, ""); return p; } diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java index cd5466d18754..b8b2e17b280d 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java @@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.KeyValue; - import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableNotFoundException; @@ -1076,11 +1075,11 @@ public TRegionInfo getRegionInfo(ByteBuffer searchRow) throws IOError { Result startRowResult = // TODO(HBASE-XXXXX - Phase 6): Get dynamic name from connection // For now, hardcode default. Future: use getConnection(user).getMetaTableName().getName() - getReverseScanResult(TableName.valueOf("hbase", "meta").getName(), row, HConstants.CATALOG_FAMILY); + getReverseScanResult(TableName.valueOf("hbase", "meta").getName(), row, + HConstants.CATALOG_FAMILY); if (startRowResult == null) { - throw new IOException("Cannot find row in hbase:meta, row=" - + Bytes.toStringBinary(row)); + throw new IOException("Cannot find row in hbase:meta, row=" + Bytes.toStringBinary(row)); } // find region start and end keys diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java index f44a3de32c71..fc810778f875 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java @@ -19,7 +19,6 @@ import com.google.errorprone.annotations.RestrictedApi; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -183,7 +182,8 @@ public static void setMetaLocation(ZKWatcher zookeeper, ServerName serverName, i ZKUtil.setData(zookeeper, zookeeper.getZNodePaths().getZNodeForReplica(replicaId), data); } catch (KeeperException.NoNodeException nne) { if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) { - LOG.debug("{} region location doesn't exist, create it", TableName.valueOf("hbase", "meta")); + LOG.debug("{} region location doesn't exist, create it", + TableName.valueOf("hbase", "meta")); } else { LOG.debug("{} region location doesn't exist for replicaId={}, create it", TableName.valueOf("hbase", "meta"), replicaId); @@ -238,8 +238,8 @@ public static void deleteMetaLocation(ZKWatcher zookeeper, int replicaId) throws if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) { LOG.info("Deleting {} region location in ZooKeeper", TableName.valueOf("hbase", "meta")); } else { - LOG.info("Deleting {} for {} region location in ZooKeeper", TableName.valueOf("hbase", "meta"), - replicaId); + LOG.info("Deleting {} for {} region location in ZooKeeper", + TableName.valueOf("hbase", "meta"), replicaId); } try { // Just delete the node. Don't need any watches. diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java index 64023d0efe86..296c961cc952 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java @@ -32,7 +32,6 @@ import java.util.LinkedList; import java.util.List; import org.apache.hadoop.hbase.HConstants; - import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.yetus.audience.InterfaceAudience; From 2f7c86e19cc4730827e830f5f0758e77de63fe16 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Mon, 26 Jan 2026 16:24:36 -0500 Subject: [PATCH 14/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: Format code part 4 --- .../apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java | 4 ++++ .../org/apache/hadoop/hbase/mapreduce/TestImportExport.java | 2 +- .../hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java | 4 ++++ .../hadoop/hbase/mapreduce/TestTableInputFormatBase.java | 4 ++++ .../src/main/java/org/apache/hadoop/hbase/HBaseCluster.java | 4 +++- 5 files changed, 16 insertions(+), 2 deletions(-) diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java index 37096e408a74..0c1da1064f1c 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java @@ -1205,6 +1205,10 @@ public Configuration getConfiguration() { return delegate.getConfiguration(); } + @Override public TableName getMetaTableName() { + return delegate.getMetaTableName(); + } + @Override public Table getTable(TableName tableName) throws IOException { return delegate.getTable(tableName); diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java index 1cc83ac4ec26..072baaed7344 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java @@ -255,7 +255,7 @@ public void testSimpleCase() throws Throwable { */ @Test public void testMetaExport() throws Throwable { - String[] args = new String[] { connection.getMetaTableName().getNameAsString(), FQ_OUTPUT_DIR, + String[] args = new String[] { UTIL.getConnection().getMetaTableName().getNameAsString(), FQ_OUTPUT_DIR, "1", "0", "0" }; assertTrue(runExport(args)); } diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java index 7c136fa2a19f..d60395f4eeef 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java @@ -145,6 +145,10 @@ public Configuration getConfiguration() { return this.configuration; } + @Override public TableName getMetaTableName() { + return null; + } + @Override public BufferedMutator getBufferedMutator(TableName tableName) throws IOException { return null; diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java index 7b2170d19520..defd7afde4bd 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java @@ -231,6 +231,10 @@ public Configuration getConfiguration() { throw new UnsupportedOperationException(); } + @Override public TableName getMetaTableName() { + throw new UnsupportedOperationException(); + } + @Override public Table getTable(TableName tableName) throws IOException { Table table = mock(Table.class); diff --git a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCluster.java b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCluster.java index eb40f4eee439..71bf017e7f48 100644 --- a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCluster.java +++ b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCluster.java @@ -22,6 +22,8 @@ import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.master.MetaTableNameStore; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Threads; import org.apache.yetus.audience.InterfaceAudience; @@ -343,7 +345,7 @@ public boolean restoreClusterMetrics(ClusterMetrics desiredStatus) throws IOExce * Get the ServerName of region server serving the first hbase:meta region */ public ServerName getServerHoldingMeta() throws IOException { - return getServerHoldingRegion(MetaTableName.getInstance(), + return getServerHoldingRegion(TableName.valueOf("hbase:meta"), RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()); } From 7fd07594d671cd880b1be797bd4ecf172f318456 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Tue, 27 Jan 2026 11:24:27 -0500 Subject: [PATCH 15/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: resolve conflicts --- .../org/apache/hadoop/hbase/rest/TestStatusResource.java | 7 ++----- .../java/org/apache/hadoop/hbase/zookeeper/ZKDump.java | 2 +- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java index ce3ec9e6ca26..e026d5931721 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java @@ -55,16 +55,13 @@ public class TestStatusResource { private static final Logger LOG = LoggerFactory.getLogger(TestStatusResource.class); - private static final byte[] META_REGION_NAME = - Bytes.toBytes(connection.getMetaTableName() + ",,1"); - private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); private static final HBaseRESTTestingUtility REST_TEST_UTIL = new HBaseRESTTestingUtility(); private static Client client; private static JAXBContext context; private static Configuration conf; - private static void validate(StorageClusterStatusModel model) { + private static void validate(StorageClusterStatusModel model) throws IOException { assertNotNull(model); assertTrue(model.getRegions() + ">= 1", model.getRegions() >= 1); assertTrue(model.getRequests() >= 0); @@ -78,7 +75,7 @@ private static void validate(StorageClusterStatusModel model) { assertTrue(node.getStartCode() > 0L); assertTrue(node.getRequests() >= 0); for (StorageClusterStatusModel.Node.Region region : node.getRegions()) { - if (Bytes.equals(region.getName(), META_REGION_NAME)) { + if (Bytes.equals(region.getName(), Bytes.toBytes(TEST_UTIL.getConnection().getMetaTableName() + ",,1"))) { foundMeta = true; } } diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java index 296c961cc952..91ebbe983dbd 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java @@ -75,7 +75,7 @@ public static String dump(final ZKWatcher zkWatcher) { sb.append("\n ").append(child); } } - sb.append("\nRegion server holding ").append(TableName.valueOf("hbase", "meta")).append(":"); + sb.append("\nRegion server holding hbase:meta:"); sb.append("\n ").append(MetaTableLocator.getMetaRegionLocation(zkWatcher)); int numMetaReplicas = zkWatcher.getMetaReplicaNodes().size(); for (int i = 1; i < numMetaReplicas; i++) { From d14945b7b2bfd6a9ed89dc5ee6f0ecf28cf6c69c Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Tue, 27 Jan 2026 13:05:14 -0500 Subject: [PATCH 16/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: spotless --- .../mapreduce/TestHFileOutputFormat2.java | 3 ++- .../hbase/mapreduce/TestImportExport.java | 4 ++-- .../TestMultiTableInputFormatBase.java | 3 ++- .../mapreduce/TestTableInputFormatBase.java | 3 ++- .../hadoop/hbase/rest/TestStatusResource.java | 5 ++++- .../hbase/master/TableNamespaceManager.java | 9 +++----- .../procedure/HBCKServerCrashProcedure.java | 4 ++-- .../procedure/MasterProcedureScheduler.java | 4 +--- .../hbase/master/procedure/MetaQueue.java | 3 +-- .../hbase/master/procedure/SchemaLocking.java | 6 ++---- .../hadoop/hbase/util/FSTableDescriptors.java | 2 +- .../client/DummyAsyncClusterConnection.java | 2 +- .../hbase/client/RegionReplicaTestHelper.java | 4 ++-- .../apache/hadoop/hbase/client/TestAdmin.java | 2 +- .../hadoop/hbase/client/TestAdmin2.java | 8 +++---- .../TestAsyncAdminWithRegionReplicas.java | 10 ++++----- .../client/TestAsyncNonMetaRegionLocator.java | 2 +- ...ncNonMetaRegionLocatorConcurrenyLimit.java | 5 +++-- .../client/TestAsyncRegionAdminApi2.java | 21 +++++++------------ .../hbase/client/TestAsyncRegionLocator.java | 5 +++-- ...stAsyncSingleRequestRpcRetryingCaller.java | 9 ++++---- .../hbase/client/TestAsyncTableAdminApi.java | 6 +++--- .../hbase/client/TestAsyncTableAdminApi2.java | 2 +- .../hbase/client/TestAsyncTableAdminApi3.java | 20 +++++++++--------- .../hbase/client/TestCleanupMetaReplica.java | 4 ++-- .../TestIncreaseMetaReplicaThroughConfig.java | 2 +- .../client/TestMetaWithReplicasBasic.java | 4 ++-- hbase-shell/src/main/ruby/hbase/table.rb | 2 +- .../org/apache/hadoop/hbase/HBaseCluster.java | 2 -- .../thrift/ThriftHBaseServiceHandler.java | 8 +++---- .../hbase/zookeeper/MetaTableLocator.java | 20 +++++++----------- .../apache/hadoop/hbase/zookeeper/ZKDump.java | 1 - 32 files changed, 85 insertions(+), 100 deletions(-) diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java index 0c1da1064f1c..4fc5e3389902 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java @@ -1205,7 +1205,8 @@ public Configuration getConfiguration() { return delegate.getConfiguration(); } - @Override public TableName getMetaTableName() { + @Override + public TableName getMetaTableName() { return delegate.getMetaTableName(); } diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java index 072baaed7344..823c33833bea 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestImportExport.java @@ -255,8 +255,8 @@ public void testSimpleCase() throws Throwable { */ @Test public void testMetaExport() throws Throwable { - String[] args = new String[] { UTIL.getConnection().getMetaTableName().getNameAsString(), FQ_OUTPUT_DIR, - "1", "0", "0" }; + String[] args = new String[] { UTIL.getConnection().getMetaTableName().getNameAsString(), + FQ_OUTPUT_DIR, "1", "0", "0" }; assertTrue(runExport(args)); } diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java index d60395f4eeef..f10437d6c651 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java @@ -145,7 +145,8 @@ public Configuration getConfiguration() { return this.configuration; } - @Override public TableName getMetaTableName() { + @Override + public TableName getMetaTableName() { return null; } diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java index defd7afde4bd..08aa72536cc8 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestTableInputFormatBase.java @@ -231,7 +231,8 @@ public Configuration getConfiguration() { throw new UnsupportedOperationException(); } - @Override public TableName getMetaTableName() { + @Override + public TableName getMetaTableName() { throw new UnsupportedOperationException(); } diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java index e026d5931721..f6fa703e1183 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestStatusResource.java @@ -75,7 +75,10 @@ private static void validate(StorageClusterStatusModel model) throws IOException assertTrue(node.getStartCode() > 0L); assertTrue(node.getRequests() >= 0); for (StorageClusterStatusModel.Node.Region region : node.getRegions()) { - if (Bytes.equals(region.getName(), Bytes.toBytes(TEST_UTIL.getConnection().getMetaTableName() + ",,1"))) { + if ( + Bytes.equals(region.getName(), + Bytes.toBytes(TEST_UTIL.getConnection().getMetaTableName() + ",,1")) + ) { foundMeta = true; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java index 3f04865e9bb1..8b527e6a9001 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableNamespaceManager.java @@ -106,9 +106,7 @@ private void addToCache(Result result, byte[] family, byte[] qualifier) throws I } private void loadFromMeta() throws IOException { - try ( - Table table = - masterServices.getConnection().getTable(masterServices.getConnection().getMetaTableName()); + try (Table table = masterServices.getConnection().getMetaTable(); ResultScanner scanner = table.getScanner(HConstants.NAMESPACE_FAMILY)) { for (Result result;;) { result = scanner.next(); @@ -206,7 +204,7 @@ public static void insertNamespaceToMeta(Connection conn, NamespaceDescriptor ns Put put = new Put(row, true).addColumn(HConstants.NAMESPACE_FAMILY, HConstants.NAMESPACE_COL_DESC_QUALIFIER, ProtobufUtil.toProtoNamespaceDescriptor(ns).toByteArray()); - try (Table table = conn.getTable(conn.getMetaTableName())) { + try (Table table = conn.getMetaTable()) { table.put(put); } } @@ -214,8 +212,7 @@ public static void insertNamespaceToMeta(Connection conn, NamespaceDescriptor ns public void deleteNamespace(String namespaceName) throws IOException { checkMigrationDone(); Delete d = new Delete(Bytes.toBytes(namespaceName)); - try (Table table = - masterServices.getConnection().getTable(masterServices.getConnection().getMetaTableName())) { + try (Table table = masterServices.getConnection().getMetaTable()) { table.delete(d); } cache.remove(namespaceName); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java index 271cb91910ad..3f2ddca33c96 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java @@ -152,8 +152,8 @@ public boolean visit(Result result) throws IOException { RegionState rs = new RegionState(hrl.getRegion(), state, hrl.getServerName()); if (rs.isClosing()) { // Move region to CLOSED in hbase:meta. - LOG.info("Moving {} from CLOSING to CLOSED in {}", - hrl.getRegion().getRegionNameAsString(), "hbase:meta"); + LOG.info("Moving {} from CLOSING to CLOSED in hbase:meta", + hrl.getRegion().getRegionNameAsString()); try { MetaTableAccessor.updateRegionState(this.connection, hrl.getRegion(), RegionState.State.CLOSED); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java index 418bda70b424..7d8782eb8268 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java @@ -561,7 +561,6 @@ private static String getPeerId(Procedure proc) { // Meta Queue Lookup Helpers // ============================================================================ private MetaQueue getMetaQueue() { - // TODO(HBASE-XXXXX - Phase 6): Make meta table name dynamic from MasterServices // For now, hardcode default. Future: pass metaTableName via constructor from Master TableName metaTableName = TableName.valueOf("hbase", "meta"); MetaQueue node = AvlTree.get(metaMap, metaTableName, META_QUEUE_KEY_COMPARATOR); @@ -1082,8 +1081,7 @@ public boolean waitMetaExclusiveLock(Procedure procedure) { return false; } waitProcedure(lock, procedure); - // TODO(Phase 6): Support replica-specific meta table names - // TODO(HBASE-XXXXX - Phase 6): Get dynamic name from MasterServices + // TODO: Get dynamic name from MasterServices logLockedResource(LockedResourceType.META, "hbase:meta"); return true; } finally { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MetaQueue.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MetaQueue.java index 8dd9dab23046..901541d7302c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MetaQueue.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MetaQueue.java @@ -32,8 +32,7 @@ @InterfaceAudience.Private class MetaQueue extends Queue { - protected // Note: This is called during initialization, tableName is passed from parent - MetaQueue(TableName metaTableName, LockStatus lockStatus) { + protected MetaQueue(TableName metaTableName, LockStatus lockStatus) { super(metaTableName, 1, lockStatus); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java index 9314e8375705..0ba91224030f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java @@ -174,8 +174,7 @@ List getLocks() { addToLockedResources(lockedResources, regionLocks, Function.identity(), LockedResourceType.REGION); addToLockedResources(lockedResources, peerLocks, Function.identity(), LockedResourceType.PEER); - // TODO(Phase 6): Support replica-specific meta table names - // TODO(HBASE-XXXXX - Phase 6): Get dynamic name from MasterServices + // TODO: Get dynamic name from MasterServices addToLockedResources(lockedResources, ImmutableMap.of(TableName.valueOf("hbase", "meta"), metaLock), tn -> tn.getNameAsString(), LockedResourceType.META); @@ -239,8 +238,7 @@ public String toString() { .append("tableLocks", filterUnlocked(tableLocks)) .append("regionLocks", filterUnlocked(regionLocks)) .append("peerLocks", filterUnlocked(peerLocks)) - // TODO(Phase 6): Support replica-specific meta table names - // TODO(HBASE-XXXXX - Phase 6): Get dynamic name from MasterServices + // TODO: Get dynamic name from MasterServices .append("metaLocks", filterUnlocked(ImmutableMap.of(TableName.valueOf("hbase", "meta"), metaLock))) .append("globalLocks", filterUnlocked(globalLocks)).build(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java index de452b7ec0d7..852145633365 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java @@ -144,7 +144,7 @@ public static void tryUpdateMetaTableDescriptor(Configuration conf) throws IOExc } private static TableName getMetaTableNameFromConf(Configuration conf) { - // TODO(Phase 6): Support replica-specific meta table names from configuration + // TODO: Support replica-specific meta table names from masterRegion return TableName.valueOf("hbase", "meta"); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncClusterConnection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncClusterConnection.java index db3ab3496f97..0a2dafa61090 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncClusterConnection.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/DummyAsyncClusterConnection.java @@ -46,7 +46,7 @@ public Configuration getConfiguration() { @Override public TableName getMetaTableName() { - return TableName.META_TABLE_NAME; + return null; } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java index 8fbe6331cef7..1c91c4468fec 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java @@ -46,8 +46,8 @@ private RegionReplicaTestHelper() { static void waitUntilAllMetaReplicasAreReady(HBaseTestingUtil util, ConnectionRegistry registry) throws IOException { Configuration conf = util.getConfiguration(); - int regionReplicaCount = util.getAdmin() - .getDescriptor(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).getRegionReplication(); + int regionReplicaCount = + util.getAdmin().getDescriptor(util.getConnection().getMetaTableName()).getRegionReplication(); Waiter.waitFor(conf, conf.getLong("hbase.client.sync.wait.timeout.msec", 60000), 200, true, new ExplainingPredicate() { @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java index f6902084bdca..41d7902d91e1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java @@ -64,7 +64,7 @@ public class TestAdmin extends TestAdminBase { @Test public void testListTableDescriptors() throws IOException { TableDescriptor metaTableDescriptor = - TEST_UTIL.getAdmin().getDescriptor(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); + TEST_UTIL.getAdmin().getDescriptor(TEST_UTIL.getConnection().getMetaTableName()); List tableDescriptors = TEST_UTIL.getAdmin().listTableDescriptors(true); assertTrue(tableDescriptors.contains(metaTableDescriptor)); tableDescriptors = TEST_UTIL.getAdmin().listTableDescriptors(false); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java index 2211048384a4..631221f1e2e2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java @@ -84,14 +84,14 @@ public class TestAdmin2 extends TestAdminBase { public void testCreateBadTables() throws IOException { String msg = null; try { - ADMIN.createTable(TableDescriptorBuilder - .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).build()); + ADMIN.createTable( + TableDescriptorBuilder.newBuilder(TEST_UTIL.getConnection().getMetaTableName()).build()); } catch (TableExistsException e) { msg = e.toString(); } assertTrue("Unexcepted exception message " + msg, msg != null && msg.startsWith(TableExistsException.class.getName()) - && msg.contains(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable().getNameAsString())); + && msg.contains(TEST_UTIL.getConnection().getMetaTableName().getNameAsString())); // Now try and do concurrent creation with a bunch of threads. TableDescriptor tableDescriptor = @@ -487,7 +487,7 @@ private HRegionServer startAndWriteData(TableName tableName, byte[] value) @Test public void testDisableCatalogTable() throws Exception { try { - ADMIN.disableTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); + ADMIN.disableTable(TEST_UTIL.getConnection().getMetaTableName()); fail("Expected to throw ConstraintException"); } catch (ConstraintException e) { } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java index fca0fd1392d4..fb6ce2030f91 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java @@ -54,8 +54,8 @@ public class TestAsyncAdminWithRegionReplicas extends TestAsyncAdminBase { @BeforeClass public static void setUpBeforeClass() throws Exception { TestAsyncAdminBase.setUpBeforeClass(); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), - RegionInfoBuilder.FIRST_META_REGIONINFO.getTable(), 3); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TEST_UTIL.getConnection().getMetaTableName(), + 3); try (ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent())) { RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, registry); @@ -81,7 +81,7 @@ public void testMoveNonDefaultReplica() throws InterruptedException, ExecutionException, IOException { createTableWithDefaultConf(tableName, 3); testMoveNonDefaultReplica(tableName); - testMoveNonDefaultReplica(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); + testMoveNonDefaultReplica(TEST_UTIL.getConnection().getMetaTableName()); } @Test @@ -140,11 +140,11 @@ public void testCloneTableSchema() throws IOException, InterruptedException, Exe @Test public void testGetTableRegions() throws InterruptedException, ExecutionException, IOException { List metaRegions = - admin.getRegions(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).get(); + admin.getRegions(TEST_UTIL.getConnection().getMetaTableName()).get(); assertEquals(3, metaRegions.size()); for (int i = 0; i < 3; i++) { RegionInfo metaRegion = metaRegions.get(i); - assertEquals(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable(), metaRegion.getTable()); + assertEquals(TEST_UTIL.getConnection().getMetaTableName(), metaRegion.getTable()); assertEquals(i, metaRegion.getReplicaId()); } createTableWithDefaultConf(tableName, 3); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java index 1879011bafd5..a3b60c54fd39 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java @@ -130,7 +130,7 @@ public void setUpBeforeTest() throws InterruptedException, ExecutionException, I ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); conn = new AsyncConnectionImpl(c, registry, registry.getClusterId().get(), - TableName.valueOf("hbase:meta"), null, User.getCurrent()); + TEST_UTIL.getConnection().getMetaTableName(), null, User.getCurrent()); locator = new AsyncNonMetaRegionLocator(conn, AsyncConnectionImpl.RETRY_TIMER); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java index 5a94c3feaa91..9b319b17a944 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java @@ -126,8 +126,9 @@ public static void setUp() throws Exception { TEST_UTIL.getAdmin().balancerSwitch(false, true); ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); - CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, - registry.getClusterId().get(), TableName.valueOf("hbase:meta"), null, User.getCurrent()); + CONN = + new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, registry.getClusterId().get(), + TEST_UTIL.getConnection().getMetaTableName(), null, User.getCurrent()); LOCATOR = new AsyncNonMetaRegionLocator(CONN, AsyncConnectionImpl.RETRY_TIMER); SPLIT_KEYS = IntStream.range(1, 256).mapToObj(i -> Bytes.toBytes(String.format("%02x", i))) .toArray(byte[][]::new); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java index 22ebf837ea30..d7074928585f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi2.java @@ -85,8 +85,7 @@ public void testSplitSwitch() throws Exception { final int rows = 10000; TestAsyncRegionAdminApi.loadData(tableName, families, rows); - AsyncTable metaTable = - ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); + AsyncTable metaTable = ASYNC_CONN.getMetaTable(); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); int originalCount = regionLocations.size(); @@ -117,8 +116,7 @@ public void testMergeSwitch() throws Exception { byte[][] families = { FAMILY }; TestAsyncRegionAdminApi.loadData(tableName, families, 1000); - AsyncTable metaTable = - ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); + AsyncTable metaTable = ASYNC_CONN.getMetaTable(); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); int originalCount = regionLocations.size(); @@ -163,8 +161,7 @@ public void testMergeRegions() throws Exception { byte[][] splitRows = new byte[][] { Bytes.toBytes("3"), Bytes.toBytes("6") }; createTableWithDefaultConf(tableName, splitRows); - AsyncTable metaTable = - ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); + AsyncTable metaTable = ASYNC_CONN.getMetaTable(); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); RegionInfo regionA; @@ -244,8 +241,7 @@ private void splitTest(TableName tableName, int rowCount, boolean isSplitRegion, // create table createTableWithDefaultConf(tableName); - AsyncTable metaTable = - ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); + AsyncTable metaTable = ASYNC_CONN.getMetaTable(); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); assertEquals(1, regionLocations.size()); @@ -302,8 +298,7 @@ public void testTruncateRegion() throws Exception { final byte[][] bFamilies = new byte[][] { Bytes.toBytes(family1), Bytes.toBytes(family2) }; createTableWithDefaultConf(tableName, splitKeys, bFamilies); - AsyncTable metaTable = - ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); + AsyncTable metaTable = ASYNC_CONN.getMetaTable(); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); RegionInfo regionToBeTruncated = regionLocations.get(0).getRegion(); @@ -337,8 +332,7 @@ public void testTruncateReplicaRegionNotAllowed() throws Exception { final byte[][] bFamilies = new byte[][] { Bytes.toBytes(family1), Bytes.toBytes(family2) }; createTableWithDefaultConf(tableName, 2, splitKeys, bFamilies); - AsyncTable metaTable = - ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); + AsyncTable metaTable = ASYNC_CONN.getMetaTable(); List regionLocations = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); RegionInfo primaryRegion = regionLocations.get(0).getRegion(); @@ -359,8 +353,7 @@ public void testTruncateReplicaRegionNotAllowed() throws Exception { @Test public void testTruncateRegionsMetaTableRegionsNotAllowed() throws Exception { - AsyncTableRegionLocator locator = - ASYNC_CONN.getRegionLocator(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); + AsyncTableRegionLocator locator = ASYNC_CONN.getRegionLocator(ASYNC_CONN.getMetaTableName()); List regionLocations = locator.getAllRegionLocations().get(); HRegionLocation regionToBeTruncated = regionLocations.get(0); // 1 diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java index 83c826fcfbf9..6d872e5fa8c6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java @@ -101,8 +101,9 @@ public static void setUp() throws Exception { TEST_UTIL.waitTableAvailable(TABLE_NAME); ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); - CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, - registry.getClusterId().get(), TableName.valueOf("hbase:meta"), null, User.getCurrent()); + CONN = + new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, registry.getClusterId().get(), + TEST_UTIL.getConnection().getMetaTableName(), null, User.getCurrent()); LOCATOR = CONN.getLocator(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java index 547691c78c6b..3bed588e523f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncSingleRequestRpcRetryingCaller.java @@ -74,8 +74,9 @@ public static void setUpBeforeClass() throws Exception { TEST_UTIL.waitTableAvailable(TABLE_NAME); ConnectionRegistry registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); - CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, - registry.getClusterId().get(), TableName.valueOf("hbase:meta"), null, User.getCurrent()); + CONN = + new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, registry.getClusterId().get(), + TEST_UTIL.getConnection().getMetaTableName(), null, User.getCurrent()); } @AfterClass @@ -165,8 +166,8 @@ void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) { } }; try (AsyncConnectionImpl mockedConn = new AsyncConnectionImpl(CONN.getConfiguration(), - CONN.registry, CONN.registry.getClusterId().get(), TableName.valueOf("hbase:meta"), null, - User.getCurrent()) { + CONN.registry, CONN.registry.getClusterId().get(), + TEST_UTIL.getConnection().getMetaTableName(), null, User.getCurrent()) { @Override AsyncRegionLocator getLocator() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java index a742b03e95dc..fde2c951529a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi.java @@ -72,7 +72,7 @@ public void testCreateTable() throws Exception { static TableState.State getStateFromMeta(TableName table) throws Exception { Optional state = ClientMetaTableAccessor - .getTableState(ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()), table) + .getTableState(ASYNC_CONN.getTable(TEST_UTIL.getConnection().getMetaTableName()), table) .get(); assertTrue(state.isPresent()); return state.get().getState(); @@ -81,7 +81,7 @@ static TableState.State getStateFromMeta(TableName table) throws Exception { @Test public void testCreateTableNumberOfRegions() throws Exception { AsyncTable metaTable = - ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); + ASYNC_CONN.getTable(TEST_UTIL.getConnection().getMetaTableName()); createTableWithDefaultConf(tableName); List regionLocations = @@ -130,7 +130,7 @@ public void testCreateTableWithRegions() throws Exception { assertTrue("Table should be created with splitKyes + 1 rows in META", tableAvailable); AsyncTable metaTable = - ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); + ASYNC_CONN.getTable(TEST_UTIL.getConnection().getMetaTableName()); List regions = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); Iterator hris = regions.iterator(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java index 1c1c54f59df0..c6eefefe1392 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi2.java @@ -57,7 +57,7 @@ public class TestAsyncTableAdminApi2 extends TestAsyncAdminBase { @Test public void testDisableCatalogTable() throws Exception { try { - this.admin.disableTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).join(); + this.admin.disableTable(TEST_UTIL.getConnection().getMetaTableName()).join(); fail("Expected to throw ConstraintException"); } catch (Exception e) { } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java index e7f64055c29a..9248549d8599 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableAdminApi3.java @@ -65,10 +65,10 @@ public void testTableExist() throws Exception { TEST_UTIL.createTable(tableName, FAMILY); exist = admin.tableExists(tableName).get(); assertTrue(exist); - exist = admin.tableExists(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).get(); + exist = admin.tableExists(TEST_UTIL.getConnection().getMetaTableName()).get(); assertTrue(exist); // meta table already exists - exist = admin.tableExists(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).get(); + exist = admin.tableExists(TEST_UTIL.getConnection().getMetaTableName()).get(); assertTrue(exist); } @@ -117,7 +117,7 @@ public void testListTables() throws Exception { assertEquals(0, size); Collections.addAll(tableNames, tables); - tableNames.add(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); + tableNames.add(TEST_UTIL.getConnection().getMetaTableName()); tableDescs = admin.listTableDescriptors(tableNames).get(); size = tableDescs.size(); assertEquals(tables.length + 1, size); @@ -125,8 +125,8 @@ public void testListTables() throws Exception { assertTrue("tableName should be equal in order", tableDescs.get(j).getTableName().equals(tables[i])); } - assertTrue(tableDescs.get(size - 1).getTableName() - .equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable())); + assertTrue( + tableDescs.get(size - 1).getTableName().equals(TEST_UTIL.getConnection().getMetaTableName())); for (int i = 0; i < tables.length; i++) { admin.disableTable(tables[i]).join(); @@ -205,7 +205,7 @@ public void testDisableAndEnableTable() throws Exception { assertTrue(ok); // meta table can not be disabled. try { - admin.disableTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).get(); + admin.disableTable(TEST_UTIL.getConnection().getMetaTableName()).get(); fail("meta table can not be disabled"); } catch (ExecutionException e) { Throwable cause = e.getCause(); @@ -286,7 +286,7 @@ public void testEnableTableRetainAssignment() throws Exception { createTableWithDefaultConf(tableName, splitKeys); AsyncTable metaTable = - ASYNC_CONN.getTable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); + ASYNC_CONN.getTable(TEST_UTIL.getConnection().getMetaTableName()); List regions = ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName).get(); assertEquals( @@ -315,8 +315,8 @@ public void testIsTableEnabledAndDisabled() throws Exception { assertTrue(admin.isTableDisabled(tableName).get()); // meta table is always enabled - assertTrue(admin.isTableEnabled(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).get()); - assertFalse(admin.isTableDisabled(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).get()); + assertTrue(admin.isTableEnabled(TEST_UTIL.getConnection().getMetaTableName()).get()); + assertFalse(admin.isTableDisabled(TEST_UTIL.getConnection().getMetaTableName()).get()); } @Test @@ -324,6 +324,6 @@ public void testIsTableAvailable() throws Exception { createTableWithDefaultConf(tableName); TEST_UTIL.waitTableAvailable(tableName); assertTrue(admin.isTableAvailable(tableName).get()); - assertTrue(admin.isTableAvailable(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).get()); + assertTrue(admin.isTableAvailable(TEST_UTIL.getConnection().getMetaTableName()).get()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java index c2431862473a..813bc5d79267 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCleanupMetaReplica.java @@ -47,8 +47,8 @@ public void testReplicaCleanup() throws Exception { ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher(); List metaReplicaZnodes = zkw.getMetaReplicaNodes(); assertEquals(3, metaReplicaZnodes.size()); - HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), - RegionInfoBuilder.FIRST_META_REGIONINFO.getTable(), 1); + HBaseTestingUtil.setReplicas(TEST_UTIL.getAdmin(), TEST_UTIL.getConnection().getMetaTableName(), + 1); metaReplicaZnodes = zkw.getMetaReplicaNodes(); assertEquals(1, metaReplicaZnodes.size()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java index 0f5c0d30907c..e1dfe3f0113c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestIncreaseMetaReplicaThroughConfig.java @@ -49,7 +49,7 @@ public static void setUp() throws Exception { public void testUpgradeAndIncreaseReplicaCount() throws Exception { HMaster oldMaster = TEST_UTIL.getMiniHBaseCluster().getMaster(); TableDescriptors oldTds = oldMaster.getTableDescriptors(); - TableDescriptor oldMetaTd = oldTds.get(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()); + TableDescriptor oldMetaTd = oldTds.get(TEST_UTIL.getConnection().getMetaTableName()); assertEquals(3, oldMetaTd.getRegionReplication()); // force update the replica count to 1 and then kill the master, to simulate that hen upgrading, // we have no region replication in meta table descriptor but we actually have meta region diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java index 758a5e915184..41ca9747b5d2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java @@ -52,8 +52,8 @@ public static void setUp() throws Exception { @Test public void testMetaHTDReplicaCount() throws Exception { - assertEquals(3, TEST_UTIL.getAdmin() - .getDescriptor(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable()).getRegionReplication()); + assertEquals(3, TEST_UTIL.getAdmin().getDescriptor(TEST_UTIL.getConnection().getMetaTableName()) + .getRegionReplication()); } @Test diff --git a/hbase-shell/src/main/ruby/hbase/table.rb b/hbase-shell/src/main/ruby/hbase/table.rb index ff852b357ffc..458b254166e4 100644 --- a/hbase-shell/src/main/ruby/hbase/table.rb +++ b/hbase-shell/src/main/ruby/hbase/table.rb @@ -748,7 +748,7 @@ def get_all_columns # Checks if current table is one of the 'meta' tables def is_meta_table? - org.apache.hadoop.hbase.MetaTableName.getInstance.equals(@table.getName) + org.apache.hadoop.hbase.TableName.is_meta_table.equals(@table.getName) end # Given a column specification in the format FAMILY[:QUALIFIER[:CONVERTER]] diff --git a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCluster.java b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCluster.java index 71bf017e7f48..dbcd3972afeb 100644 --- a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCluster.java +++ b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCluster.java @@ -22,8 +22,6 @@ import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.client.RegionInfoBuilder; -import org.apache.hadoop.hbase.master.HMaster; -import org.apache.hadoop.hbase.master.MetaTableNameStore; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Threads; import org.apache.yetus.audience.InterfaceAudience; diff --git a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java index b8b2e17b280d..7ecfe714d134 100644 --- a/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java +++ b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java @@ -1072,11 +1072,9 @@ private void closeTable(Table table) throws IOError { public TRegionInfo getRegionInfo(ByteBuffer searchRow) throws IOError { try { byte[] row = getBytes(searchRow); - Result startRowResult = - // TODO(HBASE-XXXXX - Phase 6): Get dynamic name from connection - // For now, hardcode default. Future: use getConnection(user).getMetaTableName().getName() - getReverseScanResult(TableName.valueOf("hbase", "meta").getName(), row, - HConstants.CATALOG_FAMILY); + Result startRowResult = getReverseScanResult( + connectionCache.getAdmin().getConnection().getMetaTableName().getName(), row, + HConstants.CATALOG_FAMILY); if (startRowResult == null) { throw new IOException("Cannot find row in hbase:meta, row=" + Bytes.toStringBinary(row)); diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java index fc810778f875..95fb3a0cce73 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java @@ -21,7 +21,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException; import org.apache.hadoop.hbase.ServerName; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.master.RegionState; @@ -166,12 +165,11 @@ public static void setMetaLocation(ZKWatcher zookeeper, ServerName serverName, public static void setMetaLocation(ZKWatcher zookeeper, ServerName serverName, int replicaId, RegionState.State state) throws KeeperException { if (serverName == null) { - LOG.warn("Tried to set null ServerName in {}; skipping -- ServerName required", - TableName.valueOf("hbase", "meta")); + LOG.warn("Tried to set null ServerName in hbase:meta; skipping -- ServerName required"); return; } - LOG.info("Setting {} replicaId={} location in ZooKeeper as {}, state={}", - TableName.valueOf("hbase", "meta"), replicaId, serverName, state); + LOG.info("Setting hbase:meta replicaId={} location in ZooKeeper as {}, state={}", replicaId, + serverName, state); // Make the MetaRegionServer pb and then get its bytes and save this as // the znode content. MetaRegionServer pbrsr = @@ -182,11 +180,10 @@ public static void setMetaLocation(ZKWatcher zookeeper, ServerName serverName, i ZKUtil.setData(zookeeper, zookeeper.getZNodePaths().getZNodeForReplica(replicaId), data); } catch (KeeperException.NoNodeException nne) { if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) { - LOG.debug("{} region location doesn't exist, create it", - TableName.valueOf("hbase", "meta")); + LOG.debug("hbase:meta region location doesn't exist, create it"); } else { - LOG.debug("{} region location doesn't exist for replicaId={}, create it", - TableName.valueOf("hbase", "meta"), replicaId); + LOG.debug("hbase:meta region location doesn't exist for replicaId={}, create it", + replicaId); } ZKUtil.createAndWatch(zookeeper, zookeeper.getZNodePaths().getZNodeForReplica(replicaId), data); @@ -236,10 +233,9 @@ public static void deleteMetaLocation(ZKWatcher zookeeper) throws KeeperExceptio public static void deleteMetaLocation(ZKWatcher zookeeper, int replicaId) throws KeeperException { if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) { - LOG.info("Deleting {} region location in ZooKeeper", TableName.valueOf("hbase", "meta")); + LOG.info("Deleting hbase:meta region location in ZooKeeper"); } else { - LOG.info("Deleting {} for {} region location in ZooKeeper", - TableName.valueOf("hbase", "meta"), replicaId); + LOG.info("Deleting hbase:meta for {} region location in ZooKeeper", replicaId); } try { // Just delete the node. Don't need any watches. diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java index 91ebbe983dbd..cd2b0b1a0149 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKDump.java @@ -32,7 +32,6 @@ import java.util.LinkedList; import java.util.List; import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; From 0974dd4f92546cb6c4646912f1a383437a346d08 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Tue, 27 Jan 2026 13:16:33 -0500 Subject: [PATCH 17/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: reformat --- .../hbase/TestClientClusterMetrics.java | 22 +++++++++---------- .../util/ProcessBasedLocalHBaseCluster.java | 2 +- .../hbase/IntegrationTestMetaReplicas.java | 2 +- .../hbase/mttr/IntegrationTestMTTR.java | 2 +- .../model/TestStorageClusterStatusModel.java | 5 +++-- 5 files changed, 17 insertions(+), 16 deletions(-) diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java index 2c66a0fc278a..2fa7fd8e3f18 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java @@ -221,16 +221,16 @@ public void testRegionStatesCount() throws Exception { ClusterMetrics metrics = ADMIN.getClusterMetrics(); Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2); - Assert.assertEquals(metrics.getTableRegionStatesCount().get(connection.getMetaTableName()) + Assert.assertEquals(metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()) .getRegionsInTransition(), 0); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getOpenRegions(), 1); + metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()).getOpenRegions(), 1); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getTotalRegions(), 1); + metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()).getTotalRegions(), 1); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getClosedRegions(), 0); + metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()).getClosedRegions(), 0); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getSplitRegions(), 0); + metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()).getSplitRegions(), 0); Assert.assertEquals( metrics.getTableRegionStatesCount().get(TABLE_NAME).getRegionsInTransition(), 0); Assert.assertEquals(metrics.getTableRegionStatesCount().get(TABLE_NAME).getOpenRegions(), 1); @@ -251,12 +251,12 @@ public void testRegionStatesWithSplit() throws Exception { ClusterMetrics metrics = ADMIN.getClusterMetrics(); Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2); - Assert.assertEquals(metrics.getTableRegionStatesCount().get(connection.getMetaTableName()) + Assert.assertEquals(metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()) .getRegionsInTransition(), 0); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getOpenRegions(), 1); + metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()).getOpenRegions(), 1); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getTotalRegions(), 1); + metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()).getTotalRegions(), 1); Assert.assertEquals( metrics.getTableRegionStatesCount().get(TABLE_NAME).getRegionsInTransition(), 0); Assert.assertEquals(metrics.getTableRegionStatesCount().get(TABLE_NAME).getOpenRegions(), 1); @@ -270,12 +270,12 @@ public void testRegionStatesWithSplit() throws Exception { metrics = ADMIN.getClusterMetrics(); Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2); - Assert.assertEquals(metrics.getTableRegionStatesCount().get(connection.getMetaTableName()) + Assert.assertEquals(metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()) .getRegionsInTransition(), 0); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getOpenRegions(), 1); + metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()).getOpenRegions(), 1); Assert.assertEquals( - metrics.getTableRegionStatesCount().get(connection.getMetaTableName()).getTotalRegions(), 1); + metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()).getTotalRegions(), 1); Assert.assertEquals( metrics.getTableRegionStatesCount().get(TABLE_NAME).getRegionsInTransition(), 0); Assert.assertEquals(metrics.getTableRegionStatesCount().get(TABLE_NAME).getOpenRegions(), 2); diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java index f40a4b35db13..2d470f47a179 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/util/ProcessBasedLocalHBaseCluster.java @@ -164,7 +164,7 @@ public void startHBase() throws IOException { int attemptsLeft = 10; while (attemptsLeft-- > 0) { try { - testUtil.getConnection().getTable(connection.getMetaTableName()); + testUtil.getConnection().getMetaTable(); } catch (Exception e) { LOG.info("Waiting for HBase to startup. Retries left: " + attemptsLeft, e); Threads.sleep(1000); diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java index 57a635a02c14..5eb6b00ec748 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestMetaReplicas.java @@ -56,7 +56,7 @@ public static void setUp() throws Exception { 1000); // Make sure there are three servers. util.initializeCluster(3); - HBaseTestingUtil.setReplicas(util.getAdmin(), connection.getMetaTableName(), 3); + HBaseTestingUtil.setReplicas(util.getAdmin(), util.getConnection().getMetaTableName(), 3); ZKWatcher zkw = util.getZooKeeperWatcher(); Configuration conf = util.getConfiguration(); String baseZNode = diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java index 3fe0154ab095..5a7b6260918d 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java @@ -184,7 +184,7 @@ private static void setupActions() throws IOException { // Set up the action that will move the regions of meta. moveMetaRegionsAction = new MoveRegionsOfTableAction(sleepTime, - MonkeyConstants.DEFAULT_MOVE_REGIONS_MAX_TIME, connection.getMetaTableName()); + MonkeyConstants.DEFAULT_MOVE_REGIONS_MAX_TIME, util.getConnection().getMetaTableName()); // Set up the action that will move the regions of our table. moveRegionAction = new MoveRegionsOfTableAction(sleepTime, diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java index 54a37a73310c..af900d02594e 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java @@ -23,6 +23,7 @@ import java.util.Iterator; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.RestTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -88,7 +89,7 @@ protected StorageClusterStatusModel buildTestModel() { model.addLiveNode("test1", 1245219839331L, 128, 1024).addRegion(Bytes.toBytes("hbase:root,,0"), 1, 1, 0, 0, 0, 1, 1, 2, 1, 1, 1, 1, 1); model.addLiveNode("test2", 1245239331198L, 512, 1024).addRegion( - Bytes.toBytes(connection.getMetaTableName() + ",,1246000043724"), 1, 1, 0, 0, 0, 1, 1, 2, 1, + Bytes.toBytes(TableName.valueOf("hbase:meta") + ",,1246000043724"), 1, 1, 0, 0, 0, 1, 1, 2, 1, 1, 1, 1, 1); return model; } @@ -128,7 +129,7 @@ protected void checkModel(StorageClusterStatusModel model) { regions = node.getRegions().iterator(); region = regions.next(); assertEquals(Bytes.toString(region.getName()), - connection.getMetaTableName() + ",,1246000043724"); + TableName.valueOf("hbase:meta") + ",,1246000043724"); assertEquals(1, region.getStores()); assertEquals(1, region.getStorefiles()); assertEquals(0, region.getStorefileSizeMB()); From 79166267197b1d71d24121c62887e047810c26ad Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Wed, 28 Jan 2026 18:53:08 -0500 Subject: [PATCH 18/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: address comments 1 --- .../hbase/client/ZKConnectionRegistry.java | 5 ++ .../org/apache/hadoop/hbase/HConstants.java | 3 +- .../hbase/TestClientClusterMetrics.java | 44 ++++++++--------- .../apache/hadoop/hbase/HBaseServerBase.java | 20 +++++--- .../apache/hadoop/hbase/master/HMaster.java | 49 ++++++++----------- .../hadoop/hbase/master/MasterServices.java | 6 +++ .../hbase/master/MetaTableNameStore.java | 32 +++--------- .../master/procedure/InitMetaProcedure.java | 26 ++-------- .../hbase/regionserver/HRegionServer.java | 16 ++++++ .../hbase/master/MockNoopMasterServices.java | 5 ++ 10 files changed, 100 insertions(+), 106 deletions(-) diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java index 72b3c388f716..310086164eb6 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java @@ -264,6 +264,11 @@ public CompletableFuture getActiveMaster() { "ZKConnectionRegistry.getActiveMaster"); } + /** + * Returns the meta table name. This implementation always returns the default "hbase:meta" + * because ZKConnectionRegistry is deprecated and does not support custom meta table names. Custom + * meta table name support requires using RPC-based connection registry. + */ @Override public CompletableFuture getMetaTableName() { return tracedFuture( diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 08b1fe06a7a2..9af711e7edfd 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -1209,7 +1209,8 @@ public enum OperationStatusCode { @Deprecated public static final List HBASE_NON_USER_TABLE_DIRS = Collections.unmodifiableList(Arrays.asList( - (String[]) ArrayUtils.addAll(new String[] { "hbase:meta" }, HBASE_NON_TABLE_DIRS.toArray()))); + (String[]) ArrayUtils.addAll(new String[] { TableName.META_TABLE_NAME.getNameAsString() }, + HBASE_NON_TABLE_DIRS.toArray()))); /** Health script related settings. */ public static final String HEALTH_SCRIPT_LOC = "hbase.node.health.script.location"; diff --git a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java index 2fa7fd8e3f18..c4302c0a9e60 100644 --- a/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java +++ b/hbase-diagnostics/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java @@ -221,16 +221,16 @@ public void testRegionStatesCount() throws Exception { ClusterMetrics metrics = ADMIN.getClusterMetrics(); Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2); - Assert.assertEquals(metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()) - .getRegionsInTransition(), 0); - Assert.assertEquals( - metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()).getOpenRegions(), 1); - Assert.assertEquals( - metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()).getTotalRegions(), 1); - Assert.assertEquals( - metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()).getClosedRegions(), 0); - Assert.assertEquals( - metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()).getSplitRegions(), 0); + Assert.assertEquals(metrics.getTableRegionStatesCount() + .get(UTIL.getConnection().getMetaTableName()).getRegionsInTransition(), 0); + Assert.assertEquals(metrics.getTableRegionStatesCount() + .get(UTIL.getConnection().getMetaTableName()).getOpenRegions(), 1); + Assert.assertEquals(metrics.getTableRegionStatesCount() + .get(UTIL.getConnection().getMetaTableName()).getTotalRegions(), 1); + Assert.assertEquals(metrics.getTableRegionStatesCount() + .get(UTIL.getConnection().getMetaTableName()).getClosedRegions(), 0); + Assert.assertEquals(metrics.getTableRegionStatesCount() + .get(UTIL.getConnection().getMetaTableName()).getSplitRegions(), 0); Assert.assertEquals( metrics.getTableRegionStatesCount().get(TABLE_NAME).getRegionsInTransition(), 0); Assert.assertEquals(metrics.getTableRegionStatesCount().get(TABLE_NAME).getOpenRegions(), 1); @@ -251,12 +251,12 @@ public void testRegionStatesWithSplit() throws Exception { ClusterMetrics metrics = ADMIN.getClusterMetrics(); Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2); - Assert.assertEquals(metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()) - .getRegionsInTransition(), 0); - Assert.assertEquals( - metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()).getOpenRegions(), 1); - Assert.assertEquals( - metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()).getTotalRegions(), 1); + Assert.assertEquals(metrics.getTableRegionStatesCount() + .get(UTIL.getConnection().getMetaTableName()).getRegionsInTransition(), 0); + Assert.assertEquals(metrics.getTableRegionStatesCount() + .get(UTIL.getConnection().getMetaTableName()).getOpenRegions(), 1); + Assert.assertEquals(metrics.getTableRegionStatesCount() + .get(UTIL.getConnection().getMetaTableName()).getTotalRegions(), 1); Assert.assertEquals( metrics.getTableRegionStatesCount().get(TABLE_NAME).getRegionsInTransition(), 0); Assert.assertEquals(metrics.getTableRegionStatesCount().get(TABLE_NAME).getOpenRegions(), 1); @@ -270,12 +270,12 @@ public void testRegionStatesWithSplit() throws Exception { metrics = ADMIN.getClusterMetrics(); Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2); - Assert.assertEquals(metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()) - .getRegionsInTransition(), 0); - Assert.assertEquals( - metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()).getOpenRegions(), 1); - Assert.assertEquals( - metrics.getTableRegionStatesCount().get(UTIL.getConnection().getMetaTableName()).getTotalRegions(), 1); + Assert.assertEquals(metrics.getTableRegionStatesCount() + .get(UTIL.getConnection().getMetaTableName()).getRegionsInTransition(), 0); + Assert.assertEquals(metrics.getTableRegionStatesCount() + .get(UTIL.getConnection().getMetaTableName()).getOpenRegions(), 1); + Assert.assertEquals(metrics.getTableRegionStatesCount() + .get(UTIL.getConnection().getMetaTableName()).getTotalRegions(), 1); Assert.assertEquals( metrics.getTableRegionStatesCount().get(TABLE_NAME).getRegionsInTransition(), 0); Assert.assertEquals(metrics.getTableRegionStatesCount().get(TABLE_NAME).getOpenRegions(), 2); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java index d5f481b63a6a..81e3c7b9b81d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java @@ -675,14 +675,6 @@ public String toString() { return getServerName().toString(); } - @Override - public org.apache.hadoop.hbase.TableName getMetaTableName() { - // For now, always return the default meta table name. - // Future implementations may support custom meta table names from configuration or storage. - return org.apache.hadoop.hbase.TableName - .valueOf(org.apache.hadoop.hbase.NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); - } - protected abstract CoprocessorHost getCoprocessorHost(); protected abstract boolean canCreateBaseZNode(); @@ -706,4 +698,16 @@ public org.apache.hadoop.hbase.TableName getMetaTableName() { protected abstract boolean cacheTableDescriptor(); protected abstract boolean clusterMode(); + + protected TableName getDefaultMetaTableName() { + return TableName.valueOf("hbase", "meta"); + } + + @Override + public TableName getMetaTableName() { + // For now, it is hbase:meta because we don't support custom meta table name. + // After adding support for custom meta table names, we can calculate this from conf and use it + // downstream to persist it in Master Region. + return getDefaultMetaTableName(); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 90ef41e9231f..15fcdc8d1a71 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -515,6 +515,8 @@ public class HMaster extends HBaseServerBase implements Maste */ private MetaTableNameStore metaTableNameStore; + private volatile TableName cachedMetaTableName; + /** * Initializes the HMaster. The steps are as follows: *

@@ -1022,18 +1024,7 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE masterRegion = MasterRegionFactory.create(this); rsListStorage = new MasterRegionServerList(masterRegion, this); - // Initialize meta table name store after master region is ready - metaTableNameStore = new MetaTableNameStore(masterRegion); - try { - if (metaTableNameStore.isStored()) { - TableName metaName = metaTableNameStore.load(); - LOG.info("Loaded meta table name from Master Local Region: {}", metaName); - } else { - LOG.info("Meta table name not yet stored (will be set during InitMetaProcedure)"); - } - } catch (IOException e) { - LOG.warn("Failed to load meta table name from Master Local Region, will use default", e); - } + cachedMetaTableName = initMetaTableName(); // Initialize the ServerManager and register it as a configuration observer this.serverManager = createServerManager(this, rsListStorage); @@ -1683,25 +1674,25 @@ public TableStateManager getTableStateManager() { */ @Override public TableName getMetaTableName() { - if (metaTableNameStore != null) { - try { - return metaTableNameStore.load(); - } catch (IOException e) { - LOG.warn("Failed to load meta table name from Master Local Region, using default", e); - } - } - // Fallback to base implementation (returns default "hbase:meta") - return super.getMetaTableName(); + return cachedMetaTableName; } - /** - * Internal accessor for procedures to get the meta table name store. This is not exposed via - * MasterServices interface to avoid interface pollution. Package visibility is insufficient as - * procedures are in a sub-package. - * @return the meta table name store, or null if not yet initialized - */ - public MetaTableNameStore getMetaTableNameStoreInternal() { - return metaTableNameStore; + private TableName initMetaTableName() { + metaTableNameStore = new MetaTableNameStore(masterRegion); + try { + TableName metaTableName = metaTableNameStore.load(); + // If metaTableNameStore is empty (bootstrap case), get meta table name from super, store it, + // and return. + if (Objects.isNull(metaTableName)) { + metaTableName = super.getDefaultMetaTableName(); + LOG.info("Bootstrap: storing default meta table name in master region: {}", metaTableName); + metaTableNameStore.store(metaTableName); + } + return metaTableName; + } catch (IOException e) { + LOG.info("Exception loading/storing meta table name from master region"); + throw new RuntimeException(e); + } } /* diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java index 745b962860bb..dd88d56c6c36 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java @@ -525,4 +525,10 @@ long flushTable(final TableName tableName, final List columnFamilies, * @return procedure id */ long rollAllWALWriters(long nonceGroup, long nonce) throws IOException; + + /** + * Return cluster's meta table name + * @return meta table name + */ + TableName getMetaTableName(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java index 11001d9da4a5..f356a10df4f4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java @@ -29,10 +29,8 @@ import org.slf4j.LoggerFactory; /** - * Stores and retrieves the meta table name for this cluster in the Master Local Region. - *

- * This provides cluster-specific storage for the meta table name. - *

+ * Stores and retrieves the meta table name for this cluster in the Master Local Region. This + * provides cluster-specific storage for the meta table name. */ @InterfaceAudience.Private public class MetaTableNameStore { @@ -49,12 +47,11 @@ public MetaTableNameStore(MasterRegion masterRegion) { } /** - * Store the meta table name in the Master Local Region. - *

- * This should be called once during cluster initialization (InitMetaProcedure). The stored value - * is cluster-specific and should not conflict with other clusters sharing the same HDFS. + * Store the meta table name in the Master Local Region. This should be called once during cluster + * initialization. The stored value is cluster-specific and should not conflict with other + * clusters sharing the same HDFS. * @param metaTableName the meta table name to store - * @throws IOException if the storage operation fails + * @throws IOException if the operation fails */ public void store(TableName metaTableName) throws IOException { LOG.info("Storing meta table name in Master Local Region: {}", metaTableName); @@ -67,7 +64,6 @@ public void store(TableName metaTableName) throws IOException { /** * Load the meta table name from the Master Local Region. - *

* @return the meta table name for this cluster * @throws IOException if the load operation fails */ @@ -90,22 +86,8 @@ public TableName load() throws IOException { LOG.debug("Loaded meta table name from Master Local Region: {}", cachedMetaTableName); return cachedMetaTableName; } - cachedMetaTableName = TableName.valueOf("hbase", "meta"); - LOG.info("No stored meta table name found in Master Local Region, using default: {}", - cachedMetaTableName); + LOG.info("No stored meta table name found in Master Local Region: {}", cachedMetaTableName); return cachedMetaTableName; } } - - /** - * Check if a meta table name has been stored in the Master Local Region. - * @return true if a meta table name is stored, false otherwise - * @throws IOException if the check operation fails - */ - public boolean isStored() throws IOException { - Get get = new Get(META_TABLE_NAME_ROW); - get.addColumn(INFO_FAMILY, NAME_QUALIFIER); - Result result = masterRegion.get(get); - return !result.isEmpty(); - } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java index c83e97bf743d..fc33e083a51b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java @@ -34,8 +34,6 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.io.hfile.HFile; -import org.apache.hadoop.hbase.master.HMaster; -import org.apache.hadoop.hbase.master.MetaTableNameStore; import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; @@ -79,11 +77,11 @@ public TableOperationType getTableOperationType() { return TableOperationType.CREATE; } - private static TableDescriptor writeFsLayout(Path rootDir, MasterProcedureEnv env) - throws IOException { + private static TableDescriptor writeFsLayout(Path rootDir, MasterProcedureEnv env, + TableName metaTableName) throws IOException { LOG.info("BOOTSTRAP: creating hbase:meta region"); FileSystem fs = rootDir.getFileSystem(env.getMasterConfiguration()); - Path tableDir = CommonFSUtils.getTableDir(rootDir, TableName.valueOf("hbase", "meta")); + Path tableDir = CommonFSUtils.getTableDir(rootDir, metaTableName); if (fs.exists(tableDir) && !deleteMetaTableDirectoryIfPartial(fs, tableDir)) { LOG.warn("Can not delete partial created meta table, continue..."); } @@ -107,24 +105,10 @@ protected Flow executeFromState(MasterProcedureEnv env, InitMetaState state) try { switch (state) { case INIT_META_WRITE_FS_LAYOUT: - // Store the meta table name in Master Local Region before creating the table - // This establishes the storage pattern for dynamic meta table name discovery - TableName metaTableName = TableName.valueOf("hbase", "meta"); - HMaster master = (HMaster) env.getMasterServices(); - MetaTableNameStore store = master.getMetaTableNameStoreInternal(); - - if (store != null && !store.isStored()) { - LOG.info("Storing meta table name in Master Local Region: {}", metaTableName); - store.store(metaTableName); - } else if (store == null) { - LOG.warn("MetaTableNameStore not initialized, skipping storage"); - } else { - LOG.info("Meta table name already stored, skipping"); - } - Configuration conf = env.getMasterConfiguration(); Path rootDir = CommonFSUtils.getRootDir(conf); - TableDescriptor td = writeFsLayout(rootDir, env); + TableDescriptor td = + writeFsLayout(rootDir, env, env.getMasterServices().getMetaTableName()); env.getMasterServices().getTableDescriptors().update(td, true); setNextState(InitMetaState.INIT_META_ASSIGN_META); return Flow.HAS_MORE_STATE; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index eea82ca511eb..f69e6230fc1b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -3672,6 +3672,22 @@ public List getMetaLocations() { return metaRegionLocationCache.getMetaRegionLocations(); } + /** + * RegionServers get the meta table name from Master via connection registry. + */ + @Override + public TableName getMetaTableName() { + if (asyncClusterConnection != null) { + try { + return asyncClusterConnection.getMetaTableName(); + } catch (Exception e) { + LOG.warn("Failed to get meta table name from Master", e); + } + } + // Bootstrap + return super.getMetaTableName(); + } + @Override protected NamedQueueRecorder createNamedQueueRecord() { return NamedQueueRecorder.getInstance(conf); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java index b04380ae450c..720ba0fba073 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockNoopMasterServices.java @@ -597,6 +597,11 @@ public long rollAllWALWriters(long nonceGroup, long nonce) throws IOException { return 0; } + @Override + public TableName getMetaTableName() { + return null; + } + @Override public KeyManagementService getKeyManagementService() { return this; From c0a96ffe7c77394cd592c16679e60f8d9de0c1d9 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Wed, 28 Jan 2026 22:33:20 -0500 Subject: [PATCH 19/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: fix unit tests --- .../TestConnectionFactoryApplyURIQueries.java | 21 ++++++++++++------- .../org/apache/hadoop/hbase/TableName.java | 2 +- .../TestSimpleRegionNormalizer.java | 2 +- .../TestGetClosestAtOrBefore.java | 2 +- .../regionserver/TestWALEntrySinkFilter.java | 2 +- 5 files changed, 17 insertions(+), 12 deletions(-) diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionFactoryApplyURIQueries.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionFactoryApplyURIQueries.java index 806c5edeb7fc..b82bd25af4e1 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionFactoryApplyURIQueries.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionFactoryApplyURIQueries.java @@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.After; @@ -59,6 +60,8 @@ public void setUp() { mockedConnectionRegistryFactory .when(() -> ConnectionRegistryFactory.create(any(), any(), any())).thenReturn(registry); when(registry.getClusterId()).thenReturn(CompletableFuture.completedFuture("cluster")); + when(registry.getMetaTableName()) + .thenReturn(CompletableFuture.completedFuture(TableName.valueOf("hbase:meta"))); } @After @@ -68,13 +71,15 @@ public void tearDown() { @Test public void testApplyURIQueries() throws Exception { - ConnectionFactory.createConnection(new URI("hbase+rpc://server:16010?a=1&b=2&c"), conf); - ArgumentCaptor captor = ArgumentCaptor.forClass(Configuration.class); - mockedConnectionRegistryFactory - .verify(() -> ConnectionRegistryFactory.create(any(), captor.capture(), any())); - Configuration c = captor.getValue(); - assertEquals("1", c.get("a")); - assertEquals("2", c.get("b")); - assertEquals("", c.get("c")); + try (Connection ignored = + ConnectionFactory.createConnection(new URI("hbase+rpc://server:16010?a=1&b=2&c"), conf)) { + ArgumentCaptor captor = ArgumentCaptor.forClass(Configuration.class); + mockedConnectionRegistryFactory + .verify(() -> ConnectionRegistryFactory.create(any(), captor.capture(), any())); + Configuration c = captor.getValue(); + assertEquals("1", c.get("a")); + assertEquals("2", c.get("b")); + assertEquals("", c.get("c")); + } } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java index eab8b9c9294e..e392717fa26f 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java @@ -71,7 +71,7 @@ public final class TableName implements Comparable { /** The hbase:meta table's name. */ @Deprecated - public static TableName META_TABLE_NAME = + public static final TableName META_TABLE_NAME = valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java index 27a75cc08d23..42657a91ae3b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java @@ -104,7 +104,7 @@ public void before() { @Test public void testNoNormalizationForMetaTable() { - TableName testTable = masterServices.getConnection().getMetaTableName(); + TableName testTable = TableName.valueOf("hbase:meta"); TableDescriptor testMetaTd = TableDescriptorBuilder.newBuilder(testTable).build(); List RegionInfo = new ArrayList<>(); Map regionSizes = new HashMap<>(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java index 5ff46259d505..053b250fefcc 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java @@ -92,7 +92,7 @@ public void testUsingMetaAndBinary() throws IOException { // Up flush size else we bind up when we use default catalog flush of 16k. TableDescriptors tds = new FSTableDescriptors(UTIL.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(UTIL.getConfiguration()); - TableDescriptor td = tds.get(UTIL.getConnection().getMetaTableName()); + TableDescriptor td = tds.get(TableName.valueOf("hbase:meta")); td = TableDescriptorBuilder.newBuilder(td).setMemStoreFlushSize(64 * 1024 * 1024).build(); HRegion mr = HBaseTestingUtil.createRegionAndWAL(RegionInfoBuilder.FIRST_META_REGIONINFO, rootdir, conf, td); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java index 11a0e98c5541..6e865967dda4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestWALEntrySinkFilter.java @@ -187,7 +187,7 @@ public static class DevNullAsyncClusterConnection extends DummyAsyncClusterConne private final Configuration conf; public DevNullAsyncClusterConnection(Configuration conf, Object registry, String clusterId, - SocketAddress localAddress, User user) { + TableName metaTableName, SocketAddress localAddress, User user) { this.conf = conf; } From fe30eecd7f9e1beb42fd9c4f38f9690552dfe2b2 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Thu, 29 Jan 2026 09:55:43 -0500 Subject: [PATCH 20/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: fix unit tests 1 --- .../hbase/client/TestAsyncRegionLocatorTracing.java | 3 ++- .../apache/hadoop/hbase/client/TestCompactFromClient.java | 1 + .../backup/example/TestZooKeeperTableArchiveClient.java | 5 +++++ .../hbase/regionserver/wal/TestLogRollingNoCluster.java | 6 +++--- .../replication/regionserver/TestReplicationSource.java | 2 +- .../regionserver/TestSerialReplicationChecker.java | 1 + .../java/org/apache/hadoop/hbase/wal/TestWALSplit.java | 8 +++++--- hbase-shell/src/main/ruby/hbase/table.rb | 2 +- 8 files changed, 19 insertions(+), 9 deletions(-) diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java index 5c533d2522d7..4c76f96f741f 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java @@ -85,7 +85,8 @@ public class TestAsyncRegionLocatorTracing { @Before public void setUp() throws IOException { - RegionInfo metaRegionInfo = RegionInfoBuilder.newBuilder(conn.getMetaTableName()).build(); + RegionInfo metaRegionInfo = + RegionInfoBuilder.newBuilder(TableName.valueOf("hbase:meta")).build(); locs = new RegionLocations( new HRegionLocation(metaRegionInfo, ServerName.valueOf("127.0.0.1", 12345, EnvironmentEdgeManager.currentTime())), diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java index f6093dfe9533..a16fefc3ca8c 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java @@ -92,6 +92,7 @@ public void testCompactTableWithNullLocations() throws Exception { AsyncConnectionImpl connection = mock(AsyncConnectionImpl.class)) { mockedMeta.when(() -> ClientMetaTableAccessor.getTableHRegionLocations(any(AsyncTable.class), any(TableName.class))).thenReturn(nullLocationsFuture); + when(connection.getMetaTableName()).thenReturn(TableName.valueOf("hbase:meta")); AsyncTable metaTable = mock(AsyncTable.class); when(connection.getTable(connection.getMetaTableName())).thenReturn(metaTable); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java index 509d74e0335c..36bde34136d8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java @@ -104,6 +104,11 @@ public MockRegistry(Configuration conf, User user) { public CompletableFuture getClusterId() { return CompletableFuture.completedFuture("clusterId"); } + + @Override + public CompletableFuture getMetaTableName() { + return CompletableFuture.completedFuture(org.apache.hadoop.hbase.TableName.META_TABLE_NAME); + } } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java index 4d7548da277a..4a89c064c304 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java @@ -95,7 +95,7 @@ public void testContendedLogRolling() throws Exception { FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(TEST_UTIL.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration()); TableDescriptor metaTableDescriptor = - fsTableDescriptors.get(TEST_UTIL.getConnection().getMetaTableName()); + fsTableDescriptors.get(org.apache.hadoop.hbase.TableName.valueOf("hbase:meta")); conf.set(FSHLogProvider.WRITER_IMPL, HighLatencySyncWriter.class.getName()); final WALFactory wals = new WALFactory(conf, TestLogRollingNoCluster.class.getName()); final WAL wal = wals.getWAL(null); @@ -159,7 +159,7 @@ public void run() { try { TableDescriptors tds = new FSTableDescriptors(TEST_UTIL.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration()); - TableDescriptor htd = tds.get(TEST_UTIL.getConnection().getMetaTableName()); + TableDescriptor htd = tds.get(org.apache.hadoop.hbase.TableName.valueOf("hbase:meta")); for (int i = 0; i < this.count; i++) { long now = EnvironmentEdgeManager.currentTime(); // Roll every ten edits @@ -176,7 +176,7 @@ public void run() { scopes.put(fam, 0); } final long txid = wal.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), - TEST_UTIL.getConnection().getMetaTableName(), now, mvcc, scopes), edit); + org.apache.hadoop.hbase.TableName.valueOf("hbase:meta"), now, mvcc, scopes), edit); Threads.sleep(ThreadLocalRandom.current().nextInt(5)); wal.sync(txid); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java index b4274ed0520d..8fdd8d9806e0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java @@ -196,7 +196,7 @@ public void testWALEntryFilter() throws IOException { assertTrue(wef.filter(e) == e); // Test system WAL edit. e = new WAL.Entry(new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, - TEST_UTIL.getConnection().getMetaTableName(), -1, -1, uuid), we); + org.apache.hadoop.hbase.TableName.valueOf("hbase:meta"), -1, -1, uuid), we); assertNull(wef.filter(e)); } finally { rs.terminate("Done"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java index 4fa03854d61e..9c085d810dbf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestSerialReplicationChecker.java @@ -115,6 +115,7 @@ public void setUp() throws IOException { when(source.getReplicationQueueStorage()).thenReturn(QUEUE_STORAGE); conn = mock(Connection.class); when(conn.isClosed()).thenReturn(false); + when(conn.getMetaTableName()).thenReturn(UTIL.getConnection().getMetaTableName()); doAnswer(new Answer() { @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java index c887fa4f4cb1..fd6da1e1d744 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java @@ -456,7 +456,8 @@ public void testRecoveredEditsPathForMeta() throws IOException { @Test public void testOldRecoveredEditsFileSidelined() throws IOException { Path p = createRecoveredEditsPathForRegion(); - Path tdir = CommonFSUtils.getTableDir(HBASEDIR, TEST_UTIL.getConnection().getMetaTableName()); + Path tdir = + CommonFSUtils.getTableDir(HBASEDIR, org.apache.hadoop.hbase.TableName.valueOf("hbase:meta")); Path regiondir = new Path(tdir, RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName()); fs.mkdirs(regiondir); Path parent = WALSplitUtil.getRegionDirRecoveredEditsDir(regiondir); @@ -469,8 +470,9 @@ public void testOldRecoveredEditsFileSidelined() throws IOException { private Path createRecoveredEditsPathForRegion() throws IOException { byte[] encoded = RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(); - Path p = WALSplitUtil.getRegionSplitEditsPath(TEST_UTIL.getConnection().getMetaTableName(), - encoded, 1, FILENAME_BEING_SPLIT, TMPDIRNAME, conf, ""); + Path p = + WALSplitUtil.getRegionSplitEditsPath(org.apache.hadoop.hbase.TableName.valueOf("hbase:meta"), + encoded, 1, FILENAME_BEING_SPLIT, TMPDIRNAME, conf, ""); return p; } diff --git a/hbase-shell/src/main/ruby/hbase/table.rb b/hbase-shell/src/main/ruby/hbase/table.rb index 458b254166e4..7154c862aec9 100644 --- a/hbase-shell/src/main/ruby/hbase/table.rb +++ b/hbase-shell/src/main/ruby/hbase/table.rb @@ -748,7 +748,7 @@ def get_all_columns # Checks if current table is one of the 'meta' tables def is_meta_table? - org.apache.hadoop.hbase.TableName.is_meta_table.equals(@table.getName) + org.apache.hadoop.hbase.TableName.isMetaTableName(@table.getName) end # Given a column specification in the format FAMILY[:QUALIFIER[:CONVERTER]] From ca6b94d72c176a16aee4e4facb0733afed6b99de Mon Sep 17 00:00:00 2001 From: Shanmukha Kota Date: Mon, 2 Feb 2026 10:36:18 -0500 Subject: [PATCH 21/21] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static: address reviews --- ...rgeClusterBalancingMetaTableIsolation.java | 4 +-- ...gTableIsolationAndReplicaDistribution.java | 4 +-- .../AbstractRpcBasedConnectionRegistry.java | 3 +- .../hbase/client/RegionInfoBuilder.java | 2 +- .../hbase/client/ZKConnectionRegistry.java | 5 +-- .../client/TestAsyncAdminRpcPriority.java | 2 +- .../client/TestAsyncConnectionTracing.java | 3 +- .../client/TestAsyncRegionLocatorTracing.java | 5 ++- .../client/TestAsyncTableRpcPriority.java | 2 +- .../hbase/client/TestAsyncTableTracing.java | 2 +- .../hbase/client/TestCompactFromClient.java | 2 +- .../TestConnectionFactoryApplyURIQueries.java | 2 +- .../hbase/client/TestMetricsConnection.java | 8 ++--- .../hbase/client/TestRegionInfoBuilder.java | 2 +- .../client/TestTableDescriptorBuilder.java | 4 +-- .../hadoop/hbase/CellComparatorImpl.java | 4 +-- .../hbase/InnerStoreCellComparator.java | 7 +--- .../org/apache/hadoop/hbase/TableName.java | 5 +-- .../hadoop/hbase/TestCellComparator.java | 36 +++++++++---------- .../org/apache/hadoop/hbase/TestKeyValue.java | 24 ++++++------- .../model/TestStorageClusterStatusModel.java | 7 ++-- .../apache/hadoop/hbase/HBaseServerBase.java | 2 +- .../client/AsyncClusterConnectionImpl.java | 3 +- .../client/ClusterConnectionFactory.java | 3 +- .../ShortCircuitConnectionRegistry.java | 3 +- .../apache/hadoop/hbase/master/HMaster.java | 22 +++++------- .../hbase/master/janitor/MetaFixer.java | 9 ++--- .../procedure/DeleteTableProcedure.java | 6 ++-- .../procedure/DisableTableProcedure.java | 6 ++-- .../procedure/HBCKServerCrashProcedure.java | 8 ++--- .../master/procedure/InitMetaProcedure.java | 5 ++- .../master/procedure/MasterProcedureEnv.java | 5 +++ .../procedure/MasterProcedureScheduler.java | 4 +-- .../MigrateNamespaceTableProcedure.java | 7 ++-- .../procedure/ModifyTableProcedure.java | 3 +- .../hbase/master/procedure/SchemaLocking.java | 8 ++--- .../hadoop/hbase/util/FSTableDescriptors.java | 2 +- .../apache/hadoop/hbase/util/HBaseFsck.java | 2 +- .../TestZooKeeperTableArchiveClient.java | 5 +-- ...talogReplicaLoadBalanceSimpleSelector.java | 2 +- .../hbase/client/TestFallbackToUseReplay.java | 2 +- .../coprocessor/TestMetaTableMetrics.java | 2 +- .../BalancerConditionalsTestUtil.java | 2 +- .../master/http/TestMasterStatusUtil.java | 3 +- .../master/http/TestMetaBrowserNoCluster.java | 2 +- .../TestSimpleRegionNormalizer.java | 2 +- .../hbase/master/procedure/TestHBCKSCP.java | 4 +-- .../TestGetClosestAtOrBefore.java | 2 +- .../hadoop/hbase/regionserver/TestHStore.java | 5 +-- .../wal/TestLogRollingNoCluster.java | 8 ++--- .../regionserver/TestReplicationSource.java | 2 +- .../hbase/util/TestFSTableDescriptors.java | 10 +++--- .../apache/hadoop/hbase/wal/TestWALSplit.java | 7 ++-- .../org/apache/hadoop/hbase/HBaseCluster.java | 2 +- 54 files changed, 133 insertions(+), 158 deletions(-) diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java index ccd28c486d8a..e4d2e168e2b6 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingMetaTableIsolation.java @@ -70,7 +70,7 @@ public static void setup() { // Create regions List allRegions = new ArrayList<>(); for (int i = 0; i < NUM_REGIONS; i++) { - TableName tableName = i < 3 ? TableName.valueOf("hbase:meta") : NON_META_TABLE_NAME; + TableName tableName = i < 3 ? TableName.META_TABLE_NAME : NON_META_TABLE_NAME; byte[] startKey = new byte[1]; startKey[0] = (byte) i; byte[] endKey = new byte[1]; @@ -98,7 +98,7 @@ public void testMetaTableIsolation() { } private boolean isMetaTableIsolated(BalancerClusterState cluster) { - return isTableIsolated(cluster, TableName.valueOf("hbase", "meta"), "Meta"); + return isTableIsolated(cluster, TableName.META_TABLE_NAME, "Meta"); } } diff --git a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java index 34f090b32b7b..d323f7ca73ae 100644 --- a/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java +++ b/hbase-balancer/src/test/java/org/apache/hadoop/hbase/master/balancer/TestLargeClusterBalancingTableIsolationAndReplicaDistribution.java @@ -74,7 +74,7 @@ public static void setup() { for (int i = 0; i < NUM_REGIONS; i++) { TableName tableName; if (i < 1) { - tableName = TableName.valueOf("hbase", "meta"); + tableName = TableName.META_TABLE_NAME; } else if (i < 10) { tableName = SYSTEM_TABLE_NAME; } else { @@ -120,7 +120,7 @@ public void testTableIsolationAndReplicaDistribution() { */ private boolean isMetaTableIsolated(BalancerClusterState cluster) { - return isTableIsolated(cluster, TableName.valueOf("hbase", "meta"), "Meta"); + return isTableIsolated(cluster, TableName.META_TABLE_NAME, "Meta"); } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java index 2cd9b78309c2..bbf8029d944e 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java @@ -34,7 +34,6 @@ import java.util.function.Predicate; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -262,7 +261,7 @@ public CompletableFuture getMetaTableName() { if (resp.hasTableName() && !resp.getTableName().isEmpty()) { return TableName.valueOf(resp.getTableName()); } else { - return TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); + return TableName.META_TABLE_NAME; } }), getClass().getSimpleName() + ".getMetaTableName"); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java index ec3dd84db39d..66c62755839b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java @@ -45,7 +45,7 @@ public class RegionInfoBuilder { // TODO: For now, hardcode to default. Future: lazy initialization based on config or make it use // conenction public static final RegionInfo FIRST_META_REGIONINFO = - new MutableRegionInfo(1L, TableName.valueOf("hbase", "meta"), RegionInfo.DEFAULT_REPLICA_ID); + new MutableRegionInfo(1L, TableName.META_TABLE_NAME, RegionInfo.DEFAULT_REPLICA_ID); private final TableName tableName; private byte[] startKey = HConstants.EMPTY_START_ROW; diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java index 310086164eb6..cc3b23392f2b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java @@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.ClusterId; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; @@ -271,9 +270,7 @@ public CompletableFuture getActiveMaster() { */ @Override public CompletableFuture getMetaTableName() { - return tracedFuture( - () -> CompletableFuture - .completedFuture(TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta")), + return tracedFuture(() -> CompletableFuture.completedFuture(TableName.META_TABLE_NAME), "ZKConnectionRegistry.getMetaTableName"); } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java index 575f7f69a94b..9d3045257aaf 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminRpcPriority.java @@ -144,7 +144,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { any()); User user = UserProvider.instantiate(CONF).getCurrent(); conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF, user), "test", - TableName.valueOf("hbase:meta"), null, user) { + TableName.META_TABLE_NAME, null, user) { @Override CompletableFuture getMasterStub() { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java index 29f9543cc67c..22fac8ecc9e5 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java @@ -73,8 +73,7 @@ public CompletableFuture getActiveMaster() { return CompletableFuture.completedFuture(masterServer); } }; - conn = - new AsyncConnectionImpl(CONF, registry, "test", TableName.valueOf("hbase:meta"), null, user); + conn = new AsyncConnectionImpl(CONF, registry, "test", TableName.META_TABLE_NAME, null, user); } @After diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java index 4c76f96f741f..180a95d08895 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java @@ -85,8 +85,7 @@ public class TestAsyncRegionLocatorTracing { @Before public void setUp() throws IOException { - RegionInfo metaRegionInfo = - RegionInfoBuilder.newBuilder(TableName.valueOf("hbase:meta")).build(); + RegionInfo metaRegionInfo = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build(); locs = new RegionLocations( new HRegionLocation(metaRegionInfo, ServerName.valueOf("127.0.0.1", 12345, EnvironmentEdgeManager.currentTime())), @@ -101,7 +100,7 @@ public void setUp() throws IOException { public CompletableFuture getMetaRegionLocations() { return CompletableFuture.completedFuture(locs); } - }, "test", TableName.valueOf("hbase:meta"), null, user); + }, "test", TableName.META_TABLE_NAME, null, user); } @After diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java index 7e61b299e1ea..87e1211d715b 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRpcPriority.java @@ -163,7 +163,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(stub).get(any(HBaseRpcController.class), any(GetRequest.class), any()); User user = UserProvider.instantiate(CONF).getCurrent(); conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF, user), "test", - TableName.valueOf("hbase:meta"), null, user) { + TableName.META_TABLE_NAME, null, user) { @Override AsyncRegionLocator getLocator() { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java index b1ffab020fbe..3ee9a6b97e54 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java @@ -210,7 +210,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { }).when(stub).get(any(HBaseRpcController.class), any(GetRequest.class), any()); final User user = UserProvider.instantiate(CONF).getCurrent(); conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF, user), "test", - TableName.valueOf("hbase:meta"), null, user) { + TableName.META_TABLE_NAME, null, user) { @Override AsyncRegionLocator getLocator() { diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java index a16fefc3ca8c..eca15781f0e0 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestCompactFromClient.java @@ -92,7 +92,7 @@ public void testCompactTableWithNullLocations() throws Exception { AsyncConnectionImpl connection = mock(AsyncConnectionImpl.class)) { mockedMeta.when(() -> ClientMetaTableAccessor.getTableHRegionLocations(any(AsyncTable.class), any(TableName.class))).thenReturn(nullLocationsFuture); - when(connection.getMetaTableName()).thenReturn(TableName.valueOf("hbase:meta")); + when(connection.getMetaTableName()).thenReturn(TableName.META_TABLE_NAME); AsyncTable metaTable = mock(AsyncTable.class); when(connection.getTable(connection.getMetaTableName())).thenReturn(metaTable); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionFactoryApplyURIQueries.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionFactoryApplyURIQueries.java index b82bd25af4e1..a31f0cfb900c 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionFactoryApplyURIQueries.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestConnectionFactoryApplyURIQueries.java @@ -61,7 +61,7 @@ public void setUp() { .when(() -> ConnectionRegistryFactory.create(any(), any(), any())).thenReturn(registry); when(registry.getClusterId()).thenReturn(CompletableFuture.completedFuture("cluster")); when(registry.getMetaTableName()) - .thenReturn(CompletableFuture.completedFuture(TableName.valueOf("hbase:meta"))); + .thenReturn(CompletableFuture.completedFuture(TableName.META_TABLE_NAME)); } @After diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java index 7558e0056169..6edbb3d47515 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java @@ -106,14 +106,14 @@ public void testMetricsConnectionScope() throws IOException { String scope = "testScope"; conf.setBoolean(MetricsConnection.CLIENT_SIDE_METRICS_ENABLED_KEY, true); - AsyncConnectionImpl impl = new AsyncConnectionImpl(conf, null, "foo", - TableName.valueOf("hbase:meta"), null, User.getCurrent()); + AsyncConnectionImpl impl = new AsyncConnectionImpl(conf, null, "foo", TableName.META_TABLE_NAME, + null, User.getCurrent()); Optional metrics = impl.getConnectionMetrics(); assertTrue("Metrics should be present", metrics.isPresent()); assertEquals(clusterId + "@" + Integer.toHexString(impl.hashCode()), metrics.get().getMetricScope()); conf.set(MetricsConnection.METRICS_SCOPE_KEY, scope); - impl = new AsyncConnectionImpl(conf, null, "foo", TableName.valueOf("hbase:meta"), null, + impl = new AsyncConnectionImpl(conf, null, "foo", TableName.META_TABLE_NAME, null, User.getCurrent()); metrics = impl.getConnectionMetrics(); @@ -134,7 +134,7 @@ public void testMetricsWithMultiConnections() throws IOException { AsyncConnectionImpl impl; List connList = new ArrayList(); for (int i = 0; i < num; i++) { - impl = new AsyncConnectionImpl(conf, null, null, TableName.valueOf("hbase:meta"), null, user); + impl = new AsyncConnectionImpl(conf, null, null, TableName.META_TABLE_NAME, null, user); connList.add(impl); } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java index f8e6b1533505..f74b79a0672e 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java @@ -141,7 +141,7 @@ public void testContainsRange() { @Test public void testContainsRangeForMetaTable() { TableDescriptor tableDesc = - TableDescriptorBuilder.newBuilder(TableName.valueOf("hbase:meta")).build(); + TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).build(); RegionInfo hri = RegionInfoBuilder.newBuilder(tableDesc.getTableName()).build(); byte[] startRow = HConstants.EMPTY_START_ROW; byte[] row1 = Bytes.toBytes("a,a,0"); diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java index b93ee2174c36..53f33845ef7d 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestTableDescriptorBuilder.java @@ -59,7 +59,7 @@ public class TestTableDescriptorBuilder { @Test(expected = IOException.class) public void testAddCoprocessorTwice() throws IOException { String cpName = "a.b.c.d"; - TableDescriptorBuilder.newBuilder(TableName.valueOf("hbase:meta")).setCoprocessor(cpName) + TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).setCoprocessor(cpName) .setCoprocessor(cpName).build(); } @@ -67,7 +67,7 @@ public void testAddCoprocessorTwice() throws IOException { public void testPb() throws DeserializationException, IOException { final int v = 123; TableDescriptor htd = - TableDescriptorBuilder.newBuilder(TableName.valueOf("hbase:meta")).setMaxFileSize(v) + TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).setMaxFileSize(v) .setDurability(Durability.ASYNC_WAL).setReadOnly(true).setRegionReplication(2).build(); byte[] bytes = TableDescriptorBuilder.toByteArray(htd); diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java index e2faf575975c..9d17de911085 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java @@ -793,7 +793,7 @@ public static CellComparator getCellComparator(byte[] tableName) { : CellComparatorImpl.COMPARATOR; } - private static boolean isMetaTable(byte[] tableName) { - return Bytes.startsWith(tableName, Bytes.toBytes("hbase:meta")); + static boolean isMetaTable(byte[] tableName) { + return Bytes.startsWith(tableName, TableName.META_TABLE_NAME.getName()); } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java index fe95aa7195e8..0c9dae240b07 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; @@ -76,12 +75,8 @@ public static CellComparator getInnerStoreCellComparator(TableName tableName) { */ public static CellComparator getInnerStoreCellComparator(byte[] tableName) { // Check if this is a meta table (hbase:meta or hbase:meta_*) - return isMetaTable(tableName) + return CellComparatorImpl.isMetaTable(tableName) ? MetaCellComparator.META_COMPARATOR : InnerStoreCellComparator.INNER_STORE_COMPARATOR; } - - private static boolean isMetaTable(byte[] tableName) { - return Bytes.startsWith(tableName, Bytes.toBytes("hbase:meta")); - } } diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java index e392717fa26f..3a28c1eecb85 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java @@ -95,8 +95,9 @@ public final class TableName implements Comparable { * both the default meta table and read replica meta tables. */ public static boolean isMetaTableName(final TableName tn) { - if (tn == null) return false; - if (!tn.getNamespaceAsString().equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR)) { + if ( + tn == null || !tn.getNamespaceAsString().equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR) + ) { return false; } String qualifier = tn.getQualifierAsString(); diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java index d4908903682a..d8c169140f46 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java @@ -196,32 +196,28 @@ public void testMetaComparisons2() { long now = EnvironmentEdgeManager.currentTime(); CellComparator c = MetaCellComparator.META_COMPARATOR; assertTrue(c.compare( - createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now)), - createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now))) == 0); - Cell a = createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now)); - Cell b = createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,2"), now)); + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta,a,,0,1"), now)), + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta,a,,0,1"), now))) + == 0); + Cell a = + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta,a,,0,1"), now)); + Cell b = + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta,a,,0,2"), now)); assertTrue(c.compare(a, b) < 0); assertTrue(c.compare( - createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,2"), now)), - createByteBufferKeyValueFromKeyValue( - new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now))) > 0); + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta,a,,0,2"), now)), + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta,a,,0,1"), now))) + > 0); assertTrue(c.compare( - createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now)), - createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now))) + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta,,1"), now)), + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta,,1"), now))) == 0); assertTrue(c.compare( - createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now)), - createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta" + ",,2"), now))) - < 0); + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta,,1"), now)), + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta,,2"), now))) < 0); assertTrue(c.compare( - createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta" + ",,2"), now)), - createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now))) - > 0); + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta,,2"), now)), + createByteBufferKeyValueFromKeyValue(new KeyValue(Bytes.toBytes("hbase:meta,,1"), now))) > 0); } @Test diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java index cfb02f44b1fd..4547aef0abf3 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java @@ -197,23 +197,23 @@ public void testKeyValueBorderCases() { private void metacomparisons(final CellComparatorImpl c) { long now = EnvironmentEdgeManager.currentTime(); - assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now), - new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now)) == 0); - KeyValue a = new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now); - KeyValue b = new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,2"), now); + assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta,a,,0,1"), now), + new KeyValue(Bytes.toBytes("hbase:meta,a,,0,1"), now)) == 0); + KeyValue a = new KeyValue(Bytes.toBytes("hbase:meta,a,,0,1"), now); + KeyValue b = new KeyValue(Bytes.toBytes("hbase:meta,a,,0,2"), now); assertTrue(c.compare(a, b) < 0); - assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,2"), now), - new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now)) > 0); + assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta,a,,0,2"), now), + new KeyValue(Bytes.toBytes("hbase:meta,a,,0,1"), now)) > 0); } private void comparisons(final CellComparatorImpl c) { long now = EnvironmentEdgeManager.currentTime(); - assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now), - new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now)) == 0); - assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now), - new KeyValue(Bytes.toBytes("hbase:meta" + ",,2"), now)) < 0); - assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta" + ",,2"), now), - new KeyValue(Bytes.toBytes("hbase:meta" + ",,1"), now)) > 0); + assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta,,1"), now), + new KeyValue(Bytes.toBytes("hbase:meta,,1"), now)) == 0); + assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta,,1"), now), + new KeyValue(Bytes.toBytes("hbase:meta,,2"), now)) < 0); + assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta,,2"), now), + new KeyValue(Bytes.toBytes("hbase:meta,,1"), now)) > 0); } @Test diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java index af900d02594e..8310232890dd 100644 --- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java +++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/model/TestStorageClusterStatusModel.java @@ -89,8 +89,8 @@ protected StorageClusterStatusModel buildTestModel() { model.addLiveNode("test1", 1245219839331L, 128, 1024).addRegion(Bytes.toBytes("hbase:root,,0"), 1, 1, 0, 0, 0, 1, 1, 2, 1, 1, 1, 1, 1); model.addLiveNode("test2", 1245239331198L, 512, 1024).addRegion( - Bytes.toBytes(TableName.valueOf("hbase:meta") + ",,1246000043724"), 1, 1, 0, 0, 0, 1, 1, 2, 1, - 1, 1, 1, 1); + Bytes.toBytes(TableName.META_TABLE_NAME + ",,1246000043724"), 1, 1, 0, 0, 0, 1, 1, 2, 1, 1, 1, + 1, 1); return model; } @@ -128,8 +128,7 @@ protected void checkModel(StorageClusterStatusModel model) { assertEquals(1024, node.getMaxHeapSizeMB()); regions = node.getRegions().iterator(); region = regions.next(); - assertEquals(Bytes.toString(region.getName()), - TableName.valueOf("hbase:meta") + ",,1246000043724"); + assertEquals(Bytes.toString(region.getName()), TableName.META_TABLE_NAME + ",,1246000043724"); assertEquals(1, region.getStores()); assertEquals(1, region.getStorefiles()); assertEquals(0, region.getStorefileSizeMB()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java index 81e3c7b9b81d..85df415f6598 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseServerBase.java @@ -700,7 +700,7 @@ public String toString() { protected abstract boolean clusterMode(); protected TableName getDefaultMetaTableName() { - return TableName.valueOf("hbase", "meta"); + return TableName.META_TABLE_NAME; } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java index 9d232e34aa31..4d01e98f2f7e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java @@ -59,8 +59,7 @@ class AsyncClusterConnectionImpl extends AsyncConnectionImpl implements AsyncClusterConnection { public AsyncClusterConnectionImpl(Configuration conf, ConnectionRegistry registry, - String clusterId, org.apache.hadoop.hbase.TableName metaTableName, SocketAddress localAddress, - User user) { + String clusterId, TableName metaTableName, SocketAddress localAddress, User user) { super(conf, registry, clusterId, metaTableName, localAddress, user, Collections.emptyMap()); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java index b1b469110fcf..a837a677ff5f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java @@ -22,6 +22,7 @@ import java.net.URI; import java.security.PrivilegedExceptionAction; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.FutureUtils; import org.apache.hadoop.hbase.util.ReflectionUtils; @@ -42,7 +43,7 @@ private ClusterConnectionFactory() { private static AsyncClusterConnection createAsyncClusterConnection(Configuration conf, ConnectionRegistry registry, SocketAddress localAddress, User user) throws IOException { String clusterId = FutureUtils.get(registry.getClusterId()); - org.apache.hadoop.hbase.TableName metaTableName = FutureUtils.get(registry.getMetaTableName()); + TableName metaTableName = FutureUtils.get(registry.getMetaTableName()); Class clazz = conf.getClass(HBASE_SERVER_CLUSTER_CONNECTION_IMPL, AsyncClusterConnectionImpl.class, AsyncClusterConnection.class); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitConnectionRegistry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitConnectionRegistry.java index e257bd029607..8863d6b8c54a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitConnectionRegistry.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitConnectionRegistry.java @@ -24,6 +24,7 @@ import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; import org.apache.yetus.audience.InterfaceAudience; /** @@ -69,7 +70,7 @@ public CompletableFuture getActiveMaster() { } @Override - public CompletableFuture getMetaTableName() { + public CompletableFuture getMetaTableName() { return CompletableFuture.completedFuture(endpoint.getMetaTableName()); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 15fcdc8d1a71..b7bc3badf629 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -1102,10 +1102,7 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE startupTaskGroup.addTask("Initializing meta table if this is a new deploy"); InitMetaProcedure initMetaProc = null; // Print out state of hbase:meta on startup; helps debugging. - if ( - !this.assignmentManager.getRegionStates() - .hasTableRegionStates(getConnection().getMetaTableName()) - ) { + if (!this.assignmentManager.getRegionStates().hasTableRegionStates(getMetaTableName())) { Optional optProc = procedureExecutor.getProcedures().stream() .filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny(); initMetaProc = optProc.orElseGet(() -> { @@ -1169,7 +1166,7 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE return; } - TableDescriptor metaDescriptor = tableDescriptors.get(getConnection().getMetaTableName()); + TableDescriptor metaDescriptor = tableDescriptors.get(getMetaTableName()); final ColumnFamilyDescriptor tableFamilyDesc = metaDescriptor.getColumnFamily(HConstants.TABLE_FAMILY); final ColumnFamilyDescriptor replBarrierFamilyDesc = @@ -1187,18 +1184,17 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE if (conf.get(HConstants.META_REPLICAS_NUM) != null) { int replicasNumInConf = conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM); - TableDescriptor metaDesc = tableDescriptors.get(getConnection().getMetaTableName()); + TableDescriptor metaDesc = tableDescriptors.get(getMetaTableName()); if (metaDesc.getRegionReplication() != replicasNumInConf) { // it is possible that we already have some replicas before upgrading, so we must set the // region replication number in meta TableDescriptor directly first, without creating a // ModifyTableProcedure, otherwise it may cause a double assign for the meta replicas. - int existingReplicasCount = assignmentManager.getRegionStates() - .getRegionsOfTable(getConnection().getMetaTableName()).size(); + int existingReplicasCount = + assignmentManager.getRegionStates().getRegionsOfTable(getMetaTableName()).size(); if (existingReplicasCount > metaDesc.getRegionReplication()) { LOG.info( - "Update replica count of {} from {}(in TableDescriptor)" + " to {}(existing ZNodes)", - getConnection().getMetaTableName(), metaDesc.getRegionReplication(), - existingReplicasCount); + "Update replica count of {} from {} (in TableDescriptor) to {} (existing ZNodes)", + getMetaTableName(), metaDesc.getRegionReplication(), existingReplicasCount); metaDesc = TableDescriptorBuilder.newBuilder(metaDesc) .setRegionReplication(existingReplicasCount).build(); tableDescriptors.update(metaDesc); @@ -1209,7 +1205,7 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE "The {} config is {} while the replica count in TableDescriptor is {}" + " for hbase:meta, altering...", HConstants.META_REPLICAS_NUM, replicasNumInConf, metaDesc.getRegionReplication(), - getConnection().getMetaTableName()); + getMetaTableName()); procedureExecutor.submitProcedure(new ModifyTableProcedure( procedureExecutor.getEnvironment(), TableDescriptorBuilder.newBuilder(metaDesc) .setRegionReplication(replicasNumInConf).build(), @@ -1439,7 +1435,7 @@ private void createMissingCFsInMetaDuringUpgrade(TableDescriptor metaDescriptor) TableDescriptor newMetaDesc = TableDescriptorBuilder.newBuilder(metaDescriptor) .setColumnFamily(FSTableDescriptors.getTableFamilyDescForMeta(conf)) .setColumnFamily(FSTableDescriptors.getReplBarrierFamilyDescForMeta()).build(); - long pid = this.modifyTable(getConnection().getMetaTableName(), () -> newMetaDesc, 0, 0, false); + long pid = this.modifyTable(getMetaTableName(), () -> newMetaDesc, 0, 0, false); waitForProcedureToComplete(pid, "Failed to add table and rep_barrier CFs to meta"); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java index d4ac3d8331cb..b7b0bcfaaab1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java @@ -204,18 +204,19 @@ private static List createMetaEntries(final MasterServices masterSer final List createMetaEntriesFailures = addMetaEntriesResults.stream() .filter(Either::hasRight).map(Either::getRight).collect(Collectors.toList()); LOG.debug("Added {}/{} entries to {}", createMetaEntriesSuccesses.size(), newRegionInfos.size(), - "hbase:meta"); + TableName.META_TABLE_NAME.getNameAsString()); if (!createMetaEntriesFailures.isEmpty()) { LOG.warn( "Failed to create entries in {}} for {}/{} RegionInfo descriptors. First" + " failure message included; full list of failures with accompanying stack traces is" + " available at log level DEBUG. message={}", - "hbase:meta", createMetaEntriesFailures.size(), addMetaEntriesResults.size(), - createMetaEntriesFailures.get(0).getMessage()); + TableName.META_TABLE_NAME.getNameAsString(), createMetaEntriesFailures.size(), + addMetaEntriesResults.size(), createMetaEntriesFailures.get(0).getMessage()); if (LOG.isDebugEnabled()) { createMetaEntriesFailures - .forEach(ioe -> LOG.debug("Attempt to fix region hole in {} failed.", "hbase:meta", ioe)); + .forEach(ioe -> LOG.debug("Attempt to fix region hole in {} failed.", + TableName.META_TABLE_NAME.getNameAsString(), ioe)); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java index 5b8f2571e337..b9a02c98dfbf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java @@ -393,9 +393,7 @@ private static void cleanRegionsInMeta(final MasterProcedureEnv env, final Table .setFilter(new KeyOnlyFilter()); long now = EnvironmentEdgeManager.currentTime(); List deletes = new ArrayList<>(); - try ( - Table metaTable = env.getMasterServices().getConnection() - .getTable(env.getMasterServices().getConnection().getMetaTableName()); + try (Table metaTable = env.getMasterServices().getConnection().getTable(env.getMetaTableName()); ResultScanner scanner = metaTable.getScanner(tableScan)) { for (;;) { Result result = scanner.next(); @@ -406,7 +404,7 @@ private static void cleanRegionsInMeta(final MasterProcedureEnv env, final Table } if (!deletes.isEmpty()) { LOG.warn("Deleting some vestigial " + deletes.size() + " rows of " + tableName + " from " - + env.getMasterServices().getConnection().getMetaTableName()); + + env.getMetaTableName()); metaTable.delete(deletes); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java index 6db264cca1c9..72931ad53ff5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java @@ -110,8 +110,8 @@ protected Flow executeFromState(final MasterProcedureEnv env, final DisableTable env.getMasterServices().getTableDescriptors().get(tableName).hasGlobalReplicationScope() ) { MasterFileSystem fs = env.getMasterFileSystem(); - try (BufferedMutator mutator = env.getMasterServices().getConnection() - .getBufferedMutator(env.getMasterServices().getConnection().getMetaTableName())) { + try (BufferedMutator mutator = + env.getMasterServices().getConnection().getBufferedMutator(env.getMetaTableName())) { for (RegionInfo region : env.getAssignmentManager().getRegionStates() .getRegionsOfTable(tableName)) { long maxSequenceId = WALSplitUtil.getMaxRegionSequenceId( @@ -230,7 +230,7 @@ public TableOperationType getTableOperationType() { */ private boolean prepareDisable(final MasterProcedureEnv env) throws IOException { boolean canTableBeDisabled = true; - if (tableName.equals(env.getMasterServices().getConnection().getMetaTableName())) { + if (tableName.equals(env.getMetaTableName())) { setFailure("master-disable-table", new ConstraintException("Cannot disable " + this.tableName)); canTableBeDisabled = false; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java index 3f2ddca33c96..91de3f9e6c5a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java @@ -102,16 +102,14 @@ List getRegionsOnCrashedServer(MasterProcedureEnv env) { MetaTableAccessor.scanMetaForTableRegions(env.getMasterServices().getConnection(), visitor, null); } catch (IOException ioe) { - LOG.warn("Failed scan of {} for 'Unknown Servers'", - env.getMasterServices().getConnection().getMetaTableName(), ioe); + LOG.warn("Failed scan of {} for 'Unknown Servers'", env.getMetaTableName(), ioe); return ris; } // create the server state node too env.getAssignmentManager().getRegionStates().createServer(getServerName()); LOG.info("Found {} mentions of {} in {} of OPEN/OPENING Regions: {}", - visitor.getReassigns().size(), getServerName(), - env.getMasterServices().getConnection().getMetaTableName(), visitor.getReassigns().stream() - .map(RegionInfo::getEncodedName).collect(Collectors.joining(","))); + visitor.getReassigns().size(), getServerName(), env.getMetaTableName(), visitor.getReassigns() + .stream().map(RegionInfo::getEncodedName).collect(Collectors.joining(","))); return visitor.getReassigns(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java index fc33e083a51b..ddaa188bf5a4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java @@ -69,7 +69,7 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure proc) { // ============================================================================ private MetaQueue getMetaQueue() { // For now, hardcode default. Future: pass metaTableName via constructor from Master - TableName metaTableName = TableName.valueOf("hbase", "meta"); + TableName metaTableName = TableName.META_TABLE_NAME; MetaQueue node = AvlTree.get(metaMap, metaTableName, META_QUEUE_KEY_COMPARATOR); if (node != null) { return node; @@ -1082,7 +1082,7 @@ public boolean waitMetaExclusiveLock(Procedure procedure) { } waitProcedure(lock, procedure); // TODO: Get dynamic name from MasterServices - logLockedResource(LockedResourceType.META, "hbase:meta"); + logLockedResource(LockedResourceType.META, TableName.META_TABLE_NAME.getNameAsString()); return true; } finally { schedUnlock(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java index 89cbcb0e43ba..c726b2a77de9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MigrateNamespaceTableProcedure.java @@ -64,8 +64,7 @@ private void migrate(MasterProcedureEnv env) throws IOException { try (Table nsTable = conn.getTable(TableName.NAMESPACE_TABLE_NAME); ResultScanner scanner = nsTable.getScanner( new Scan().addFamily(TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES).readAllVersions()); - BufferedMutator mutator = - conn.getBufferedMutator(env.getMasterServices().getConnection().getMetaTableName())) { + BufferedMutator mutator = conn.getBufferedMutator(env.getMetaTableName())) { for (Result result;;) { result = scanner.next(); if (result == null) { @@ -88,8 +87,8 @@ protected Flow executeFromState(MasterProcedureEnv env, MigrateNamespaceTablePro try { switch (state) { case MIGRATE_NAMESPACE_TABLE_ADD_FAMILY: - TableDescriptor metaTableDesc = env.getMasterServices().getTableDescriptors() - .get(env.getMasterServices().getConnection().getMetaTableName()); + TableDescriptor metaTableDesc = + env.getMasterServices().getTableDescriptors().get(env.getMetaTableName()); if (!metaTableDesc.hasColumnFamily(HConstants.NAMESPACE_FAMILY)) { TableDescriptor newMetaTableDesc = TableDescriptorBuilder.newBuilder(metaTableDesc) .setColumnFamily( diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java index 559b590baf46..f8b71b3f5bae 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java @@ -110,8 +110,7 @@ protected void preflightChecks(MasterProcedureEnv env, Boolean enabled) throws H for (byte[] family : UNDELETABLE_META_COLUMNFAMILIES) { if (!cfs.contains(family)) { throw new HBaseIOException( - "Delete of " + env.getMasterServices().getConnection().getMetaTableName() - + " column family " + Bytes.toString(family)); + "Delete of " + env.getMetaTableName() + " column family " + Bytes.toString(family)); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java index 0ba91224030f..9153191017b1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SchemaLocking.java @@ -175,9 +175,8 @@ List getLocks() { LockedResourceType.REGION); addToLockedResources(lockedResources, peerLocks, Function.identity(), LockedResourceType.PEER); // TODO: Get dynamic name from MasterServices - addToLockedResources(lockedResources, - ImmutableMap.of(TableName.valueOf("hbase", "meta"), metaLock), tn -> tn.getNameAsString(), - LockedResourceType.META); + addToLockedResources(lockedResources, ImmutableMap.of(TableName.META_TABLE_NAME, metaLock), + tn -> tn.getNameAsString(), LockedResourceType.META); addToLockedResources(lockedResources, globalLocks, Function.identity(), LockedResourceType.GLOBAL); return lockedResources; @@ -239,8 +238,7 @@ public String toString() { .append("regionLocks", filterUnlocked(regionLocks)) .append("peerLocks", filterUnlocked(peerLocks)) // TODO: Get dynamic name from MasterServices - .append("metaLocks", - filterUnlocked(ImmutableMap.of(TableName.valueOf("hbase", "meta"), metaLock))) + .append("metaLocks", filterUnlocked(ImmutableMap.of(TableName.META_TABLE_NAME, metaLock))) .append("globalLocks", filterUnlocked(globalLocks)).build(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java index 852145633365..f763c76e6c38 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java @@ -145,7 +145,7 @@ public static void tryUpdateMetaTableDescriptor(Configuration conf) throws IOExc private static TableName getMetaTableNameFromConf(Configuration conf) { // TODO: Support replica-specific meta table names from masterRegion - return TableName.valueOf("hbase", "meta"); + return TableName.META_TABLE_NAME; } public static TableDescriptor tryUpdateAndGetMetaTableDescriptor(Configuration conf, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index db50478a188f..19b7506cde29 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -663,7 +663,7 @@ public int onlineConsistencyRepair() throws IOException, KeeperException, Interr String errorMsg = connection.getMetaTableName() + " table is not consistent. "; if (shouldFixAssignments()) { errorMsg += "HBCK will try fixing it. Rerun once " + connection.getMetaTableName() - + " is back " + "to consistent state."; + + " is back to consistent state."; } else { errorMsg += "Run HBCK with proper fix options to fix " + connection.getMetaTableName() + " inconsistency."; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java index 36bde34136d8..03f58d602351 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.Stoppable; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; @@ -106,8 +107,8 @@ public CompletableFuture getClusterId() { } @Override - public CompletableFuture getMetaTableName() { - return CompletableFuture.completedFuture(org.apache.hadoop.hbase.TableName.META_TABLE_NAME); + public CompletableFuture getMetaTableName() { + return CompletableFuture.completedFuture(TableName.META_TABLE_NAME); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java index 2416cd0b0e2b..b5e2b7d93c27 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogReplicaLoadBalanceSimpleSelector.java @@ -78,7 +78,7 @@ public static void setUp() throws Exception { registry = ConnectionRegistryFactory.create(TEST_UTIL.getConfiguration(), User.getCurrent()); CONN = new AsyncConnectionImpl(conf, registry, registry.getClusterId().get(), - TableName.valueOf("hbase:meta"), null, User.getCurrent()); + TableName.META_TABLE_NAME, null, User.getCurrent()); } @AfterClass diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFallbackToUseReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFallbackToUseReplay.java index cda9104d4a0c..d138316f44e4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFallbackToUseReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFallbackToUseReplay.java @@ -99,7 +99,7 @@ public static void setUpBeforeClass() throws IOException { return null; }).when(stub).replay(any(), any(), any()); CONN = new AsyncClusterConnectionImpl(CONF, mock(ConnectionRegistry.class), "test", - TableName.valueOf("hbase:meta"), null, User.getCurrent()) { + TableName.META_TABLE_NAME, null, User.getCurrent()) { @Override AsyncRegionLocator getLocator() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMetaTableMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMetaTableMetrics.java index 1b66a085fa54..1b839637adf0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMetaTableMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMetaTableMetrics.java @@ -169,7 +169,7 @@ public void testMetaTableMetricsInJmx() throws Exception { @Test public void testConcurrentAccess() { try { - tablename = Bytes.toBytes("hbase:meta"); + tablename = TableName.META_TABLE_NAME.getName(); int numRows = 3000; int numRowsInTableBefore = UTIL.countRows(TableName.valueOf(tablename)); putData(numRows); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java index 4ea9450cdea1..8a7169b09309 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/BalancerConditionalsTestUtil.java @@ -146,7 +146,7 @@ static void validateReplicaDistribution(Connection connection, TableName tableNa static void validateRegionLocations(Map> tableToServers, TableName productTableName, boolean shouldBeBalanced) { ServerName metaServer = - tableToServers.get(TableName.valueOf("hbase", "meta")).stream().findFirst().orElseThrow(); + tableToServers.get(TableName.META_TABLE_NAME).stream().findFirst().orElseThrow(); ServerName quotaServer = tableToServers.get(QuotaUtil.QUOTA_TABLE_NAME).stream().findFirst().orElseThrow(); Set productServers = tableToServers.get(productTableName); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/http/TestMasterStatusUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/http/TestMasterStatusUtil.java index 0ec94c917193..e1ea8ff9cf56 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/http/TestMasterStatusUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/http/TestMasterStatusUtil.java @@ -136,8 +136,7 @@ public void testGetUserTablesFilterOutSystemTables() throws IOException { Map mockTables = new HashMap<>(); mockTables.put("foo", TableDescriptorBuilder.newBuilder(TableName.valueOf("foo")).build()); mockTables.put("bar", TableDescriptorBuilder.newBuilder(TableName.valueOf("bar")).build()); - mockTables.put("meta", - TableDescriptorBuilder.newBuilder(TableName.valueOf("hbase", "meta")).build()); + mockTables.put("meta", TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).build()); TableDescriptors tableDescriptors = Mockito.mock(TableDescriptors.class); Mockito.doReturn(tableDescriptors).when(master).getTableDescriptors(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/http/TestMetaBrowserNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/http/TestMetaBrowserNoCluster.java index 4bb60f4137e7..74375e9d53e0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/http/TestMetaBrowserNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/http/TestMetaBrowserNoCluster.java @@ -61,7 +61,7 @@ public void buildFirstPageQueryStringNoParams() { final HttpServletRequest request = new MockRequestBuilder().build(); final MetaBrowser metaBrowser = new MetaBrowser(connection, request); - assertEquals("hbase:meta", metaBrowser.getName()); + assertEquals(TableName.META_TABLE_NAME.getNameAsString(), metaBrowser.getName()); assertNull(metaBrowser.getScanLimit()); assertNull(metaBrowser.getScanRegionState()); assertNull(metaBrowser.getScanStart()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java index 42657a91ae3b..e931716e77ed 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/normalizer/TestSimpleRegionNormalizer.java @@ -104,7 +104,7 @@ public void before() { @Test public void testNoNormalizationForMetaTable() { - TableName testTable = TableName.valueOf("hbase:meta"); + TableName testTable = TableName.META_TABLE_NAME; TableDescriptor testMetaTd = TableDescriptorBuilder.newBuilder(testTable).build(); List RegionInfo = new ArrayList<>(); Map regionSizes = new HashMap<>(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java index d71948aff71d..a878af785783 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestHBCKSCP.java @@ -262,7 +262,7 @@ public String toString() { private static class PrimaryNotMetaRegionSelector extends RegionSelector { @Override boolean regionFilter(final RegionInfo info) { - return !Objects.equals(TableName.valueOf("hbase:meta"), info.getTable()) + return !Objects.equals(TableName.META_TABLE_NAME, info.getTable()) && Objects.equals(RegionInfo.DEFAULT_REPLICA_ID, info.getReplicaId()); } @@ -278,7 +278,7 @@ Exception regionFilterFailure() { private static class ReplicaNonMetaRegionSelector extends RegionSelector { @Override boolean regionFilter(RegionInfo info) { - return !Objects.equals(TableName.valueOf("hbase:meta"), info.getTable()) + return !Objects.equals(TableName.META_TABLE_NAME, info.getTable()) && !Objects.equals(RegionInfo.DEFAULT_REPLICA_ID, info.getReplicaId()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java index 053b250fefcc..a435b9d9b239 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java @@ -92,7 +92,7 @@ public void testUsingMetaAndBinary() throws IOException { // Up flush size else we bind up when we use default catalog flush of 16k. TableDescriptors tds = new FSTableDescriptors(UTIL.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(UTIL.getConfiguration()); - TableDescriptor td = tds.get(TableName.valueOf("hbase:meta")); + TableDescriptor td = tds.get(TableName.META_TABLE_NAME); td = TableDescriptorBuilder.newBuilder(td).setMemStoreFlushSize(64 * 1024 * 1024).build(); HRegion mr = HBaseTestingUtil.createRegionAndWAL(RegionInfoBuilder.FIRST_META_REGIONINFO, rootdir, conf, td); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java index 179297bd873f..acef0dc796df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java @@ -89,7 +89,6 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MemoryCompactionPolicy; -import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -1996,9 +1995,7 @@ public void testMaxPreadBytesConfiguredToBeLessThanZero() throws Exception { public void testInMemoryCompactionTypeWithLowerCase() throws IOException, InterruptedException { Configuration conf = HBaseConfiguration.create(); conf.set("hbase.systemtables.compacting.memstore.type", "eager"); - init(name.getMethodName(), conf, - TableDescriptorBuilder.newBuilder( - TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME, "meta".getBytes())), + init(name.getMethodName(), conf, TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME), ColumnFamilyDescriptorBuilder.newBuilder(family) .setInMemoryCompaction(MemoryCompactionPolicy.NONE).build()); assertTrue(((MemStoreCompactor) ((CompactingMemStore) store.memstore).compactor).toString() diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java index 4a89c064c304..dd4fe77c8a38 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java @@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableDescriptors; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.RegionInfoBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; @@ -94,8 +95,7 @@ public void testContendedLogRolling() throws Exception { CommonFSUtils.setRootDir(conf, dir); FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(TEST_UTIL.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration()); - TableDescriptor metaTableDescriptor = - fsTableDescriptors.get(org.apache.hadoop.hbase.TableName.valueOf("hbase:meta")); + TableDescriptor metaTableDescriptor = fsTableDescriptors.get(TableName.META_TABLE_NAME); conf.set(FSHLogProvider.WRITER_IMPL, HighLatencySyncWriter.class.getName()); final WALFactory wals = new WALFactory(conf, TestLogRollingNoCluster.class.getName()); final WAL wal = wals.getWAL(null); @@ -159,7 +159,7 @@ public void run() { try { TableDescriptors tds = new FSTableDescriptors(TEST_UTIL.getConfiguration()); FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration()); - TableDescriptor htd = tds.get(org.apache.hadoop.hbase.TableName.valueOf("hbase:meta")); + TableDescriptor htd = tds.get(TableName.META_TABLE_NAME); for (int i = 0; i < this.count; i++) { long now = EnvironmentEdgeManager.currentTime(); // Roll every ten edits @@ -176,7 +176,7 @@ public void run() { scopes.put(fam, 0); } final long txid = wal.appendData(hri, new WALKeyImpl(hri.getEncodedNameAsBytes(), - org.apache.hadoop.hbase.TableName.valueOf("hbase:meta"), now, mvcc, scopes), edit); + TableName.META_TABLE_NAME, now, mvcc, scopes), edit); Threads.sleep(ThreadLocalRandom.current().nextInt(5)); wal.sync(txid); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java index 8fdd8d9806e0..13b7101766c4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/regionserver/TestReplicationSource.java @@ -196,7 +196,7 @@ public void testWALEntryFilter() throws IOException { assertTrue(wef.filter(e) == e); // Test system WAL edit. e = new WAL.Entry(new WALKeyImpl(HConstants.EMPTY_BYTE_ARRAY, - org.apache.hadoop.hbase.TableName.valueOf("hbase:meta"), -1, -1, uuid), we); + org.apache.hadoop.hbase.TableName.META_TABLE_NAME, -1, -1, uuid), we); assertNull(wef.filter(e)); } finally { rs.terminate("Done"); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java index f4c698c2e392..5e2b4b52950f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSTableDescriptors.java @@ -279,8 +279,8 @@ public void testGetAll() throws IOException, InterruptedException { TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i)).build()); } // add hbase:meta - htds.createTableDescriptor( - TableDescriptorBuilder.newBuilder(TableName.valueOf("hbase", "meta")).build()); + htds + .createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).build()); assertEquals("getAll() didn't return all TableDescriptors, expected: " + (count + 1) + " got: " + htds.getAll().size(), count + 1, htds.getAll().size()); } @@ -298,8 +298,8 @@ public void testParallelGetAll() throws IOException, InterruptedException { TableDescriptorBuilder.newBuilder(TableName.valueOf(name + i)).build()); } // add hbase:meta - htds.createTableDescriptor( - TableDescriptorBuilder.newBuilder(TableName.valueOf("hbase", "meta")).build()); + htds + .createTableDescriptor(TableDescriptorBuilder.newBuilder(TableName.META_TABLE_NAME).build()); int getTableDescriptorSize = htds.getAll().size(); assertEquals("getAll() didn't return all TableDescriptors, expected: " + (count + 1) + " got: " @@ -326,7 +326,7 @@ public void testGetAllOrdering() throws Exception { // Remove hbase:meta from list. It shows up now since we made it dynamic. The schema // is written into the fs by the FSTableDescriptors constructor now where before it // didn't. - tables.remove(TableName.valueOf("hbase", "meta").getNameAsString()); + tables.remove(TableName.META_TABLE_NAME.getNameAsString()); assertEquals(4, tables.size()); String[] tableNamesOrdered = diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java index fd6da1e1d744..b33c4db2e177 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java @@ -457,7 +457,7 @@ public void testRecoveredEditsPathForMeta() throws IOException { public void testOldRecoveredEditsFileSidelined() throws IOException { Path p = createRecoveredEditsPathForRegion(); Path tdir = - CommonFSUtils.getTableDir(HBASEDIR, org.apache.hadoop.hbase.TableName.valueOf("hbase:meta")); + CommonFSUtils.getTableDir(HBASEDIR, org.apache.hadoop.hbase.TableName.META_TABLE_NAME); Path regiondir = new Path(tdir, RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName()); fs.mkdirs(regiondir); Path parent = WALSplitUtil.getRegionDirRecoveredEditsDir(regiondir); @@ -470,9 +470,8 @@ public void testOldRecoveredEditsFileSidelined() throws IOException { private Path createRecoveredEditsPathForRegion() throws IOException { byte[] encoded = RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(); - Path p = - WALSplitUtil.getRegionSplitEditsPath(org.apache.hadoop.hbase.TableName.valueOf("hbase:meta"), - encoded, 1, FILENAME_BEING_SPLIT, TMPDIRNAME, conf, ""); + Path p = WALSplitUtil.getRegionSplitEditsPath(org.apache.hadoop.hbase.TableName.META_TABLE_NAME, + encoded, 1, FILENAME_BEING_SPLIT, TMPDIRNAME, conf, ""); return p; } diff --git a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCluster.java b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCluster.java index dbcd3972afeb..27f3dd4f43ab 100644 --- a/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCluster.java +++ b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseCluster.java @@ -343,7 +343,7 @@ public boolean restoreClusterMetrics(ClusterMetrics desiredStatus) throws IOExce * Get the ServerName of region server serving the first hbase:meta region */ public ServerName getServerHoldingMeta() throws IOException { - return getServerHoldingRegion(TableName.valueOf("hbase:meta"), + return getServerHoldingRegion(TableName.META_TABLE_NAME, RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()); }