From f5de4ac1c503989615d39c8b1f5470a5477224f0 Mon Sep 17 00:00:00 2001 From: Kota-SH Date: Tue, 10 Feb 2026 09:32:28 -0600 Subject: [PATCH] HBASE-29691: Change TableName.META_TABLE_NAME from being a global static Change TableName.META_TABLE_NAME from a global static constant to a dynamically discovered value from ConnectionRegistry. Key changes: - Add ConnectionRegistry.getMetaTableName() method for dynamic discovery - Add MetaTableNameStore for persisting meta table name in master region - Update TableName to support dynamic meta table name - Update HMaster to integrate with MetaTableNameStore - Update all client and server code to use dynamic meta table name - Add protobuf changes for meta table name in Registry.proto Refactoring for the test classes to be handled separately. --- .../backup/impl/IncrementalBackupManager.java | 2 +- .../favored/FavoredNodeAssignmentHelper.java | 3 +- .../SnapshotOfRegionAssignmentFromMeta.java | 8 +- .../hadoop/hbase/ClientMetaTableAccessor.java | 2 +- .../AbstractRpcBasedConnectionRegistry.java | 16 ++++ .../hadoop/hbase/client/AsyncConnection.java | 20 ++++ .../hbase/client/AsyncConnectionImpl.java | 14 ++- .../client/AsyncNonMetaRegionLocator.java | 8 +- .../hbase/client/AsyncRegionLocator.java | 3 +- .../client/AsyncTableRegionLocatorImpl.java | 2 +- .../hadoop/hbase/client/Connection.java | 20 ++++ .../hbase/client/ConnectionFactory.java | 33 +++++-- .../client/ConnectionOverAsyncConnection.java | 5 + .../hbase/client/ConnectionRegistry.java | 10 ++ .../hbase/client/MutableRegionInfo.java | 4 +- .../hbase/client/RawAsyncHBaseAdmin.java | 7 +- .../hbase/client/RegionInfoBuilder.java | 2 + .../hbase/client/RegionInfoDisplay.java | 2 +- .../hbase/client/TableDescriptorBuilder.java | 2 +- .../hbase/client/ZKConnectionRegistry.java | 12 +++ .../hbase/security/token/ClientTokenUtil.java | 5 +- .../hbase/shaded/protobuf/ProtobufUtil.java | 2 +- .../client/DoNothingConnectionRegistry.java | 6 ++ .../client/TestAsyncAdminRpcPriority.java | 6 +- .../client/TestAsyncConnectionTracing.java | 3 +- .../client/TestAsyncRegionLocatorTracing.java | 24 +++-- .../client/TestAsyncTableRpcPriority.java | 24 ++--- .../hbase/client/TestAsyncTableTracing.java | 4 +- .../hbase/client/TestMetricsConnection.java | 8 +- .../hadoop/hbase/CellComparatorImpl.java | 7 +- .../hbase/InnerStoreCellComparator.java | 4 +- .../org/apache/hadoop/hbase/TableName.java | 22 ++++- .../mapreduce/TestHFileOutputFormat2.java | 5 + .../TestMultiTableInputFormatBase.java | 5 + .../mapreduce/TestTableInputFormatBase.java | 5 + .../src/main/protobuf/server/Registry.proto | 13 +++ .../hadoop/hbase/HBaseRpcServicesBase.java | 19 ++++ .../apache/hadoop/hbase/HBaseServerBase.java | 12 +++ .../hadoop/hbase/MetaTableAccessor.java | 8 +- .../client/AsyncClusterConnectionImpl.java | 4 +- .../client/ClusterConnectionFactory.java | 4 +- .../client/ConnectionRegistryEndpoint.java | 10 ++ .../hbase/client/SharedAsyncConnection.java | 5 + .../hadoop/hbase/client/SharedConnection.java | 5 + .../ShortCircuitConnectionRegistry.java | 6 ++ .../hbase/coprocessor/MetaTableMetrics.java | 6 +- .../apache/hadoop/hbase/master/HMaster.java | 57 ++++++++++-- ...sterAnnotationReadingPriorityFunction.java | 2 +- .../hadoop/hbase/master/MasterServices.java | 6 ++ .../hbase/master/MetaTableNameStore.java | 93 +++++++++++++++++++ .../master/RegionPlacementMaintainer.java | 10 +- .../hbase/master/TableNamespaceManager.java | 8 +- .../hbase/master/TableStateManager.java | 4 +- .../master/assignment/AssignmentManager.java | 6 +- .../MergeTableRegionsProcedure.java | 6 +- .../master/assignment/RegionStateStore.java | 19 ++-- .../assignment/SplitTableRegionProcedure.java | 6 +- .../cleaner/ReplicationBarrierCleaner.java | 5 +- .../hadoop/hbase/master/http/MetaBrowser.java | 2 +- .../hbase/master/janitor/CatalogJanitor.java | 8 +- .../hbase/master/janitor/MetaFixer.java | 13 +-- .../master/janitor/ReportMakingVisitor.java | 5 +- .../procedure/DeleteTableProcedure.java | 5 +- .../procedure/DisableTableProcedure.java | 6 +- .../procedure/HBCKServerCrashProcedure.java | 8 +- .../master/procedure/InitMetaProcedure.java | 10 +- .../master/procedure/MasterProcedureEnv.java | 5 + .../procedure/MasterProcedureScheduler.java | 7 +- .../hbase/master/procedure/MetaQueue.java | 4 +- .../MigrateNamespaceTableProcedure.java | 4 +- .../procedure/ModifyTableProcedure.java | 2 +- .../hbase/master/procedure/SchemaLocking.java | 2 + .../procedure/TruncateRegionProcedure.java | 2 +- .../hbase/regionserver/HRegionServer.java | 16 ++++ .../hbase/regionserver/RSRpcServices.java | 2 +- .../ReplicationBarrierFamilyFormat.java | 4 +- .../hbase/security/access/AccessChecker.java | 2 +- .../hadoop/hbase/tool/BulkLoadHFilesTool.java | 10 +- .../hadoop/hbase/util/FSTableDescriptors.java | 18 ++-- .../apache/hadoop/hbase/util/HBaseFsck.java | 90 ++++++++++-------- .../hadoop/hbase/util/HBaseFsckRepair.java | 3 +- .../apache/hadoop/hbase/util/RegionMover.java | 8 +- .../wal/BoundedRecoveredHFilesOutputSink.java | 4 +- .../hbase-webapps/master/catalogTables.jsp | 2 +- .../resources/hbase-webapps/master/table.jsp | 8 +- .../client/DummyAsyncClusterConnection.java | 5 + .../client/TestAsyncNonMetaRegionLocator.java | 12 +-- ...ncNonMetaRegionLocatorConcurrenyLimit.java | 5 +- .../hbase/client/TestAsyncRegionLocator.java | 5 +- ...stAsyncSingleRequestRpcRetryingCaller.java | 8 +- ...talogReplicaLoadBalanceSimpleSelector.java | 50 +++++----- .../hbase/client/TestFallbackToUseReplay.java | 4 +- .../hbase/master/MockNoopMasterServices.java | 5 + hbase-shell/src/main/ruby/hbase/table.rb | 2 +- .../hadoop/hbase/HBaseTestingUtility.java | 20 ++-- .../thrift/ThriftHBaseServiceHandler.java | 8 +- .../thrift2/client/ThriftConnection.java | 5 + .../hbase/zookeeper/MetaTableLocator.java | 4 +- 98 files changed, 711 insertions(+), 291 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.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 20884edf836e..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 @@ -169,7 +169,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: {}", 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 6c021bf622a5..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 @@ -37,7 +37,6 @@ 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.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; @@ -131,7 +130,7 @@ public static void updateMetaWithFavoredNodesInfo( puts.add(put); } } - try (Table table = connection.getTable(TableName.META_TABLE_NAME)) { + 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 02c18c73bfb5..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 @@ -170,9 +170,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", + connection.getMetaTableName()); // Scan hbase:meta to pick up user regions - try (Table metaTable = connection.getTable(TableName.META_TABLE_NAME); + try (Table metaTable = connection.getTable(connection.getMetaTableName()); ResultScanner scanner = metaTable.getScanner(HConstants.CATALOG_FAMILY)) { for (;;) { Result result = scanner.next(); @@ -187,7 +188,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", + connection.getMetaTableName()); } private void addRegion(RegionInfo regionInfo) { 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..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 (TableName.META_TABLE_NAME.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/AbstractRpcBasedConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java index 62c6951b4535..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 @@ -36,6 +36,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.hadoop.hbase.exceptions.ClientExceptionsUtil; import org.apache.hadoop.hbase.exceptions.MasterRegistryFetchException; import org.apache.hadoop.hbase.ipc.HBaseRpcController; @@ -55,6 +56,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 +253,19 @@ 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 (resp.hasTableName() && !resp.getTableName().isEmpty()) { + return TableName.valueOf(resp.getTableName()); + } else { + return TableName.META_TABLE_NAME; + } + }), 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..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 @@ -41,6 +41,17 @@ 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. + *

+ * @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 @@ -104,6 +115,15 @@ default AsyncTable getTable(TableName tableName, ExecutorSer return getTableBuilder(tableName, pool).build(); } + /** + * 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() { + return getTable(getMetaTableName()); + } + /** * Returns an {@link AsyncTableBuilder} for creating {@link AsyncTable}. *

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/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java index e26fb837b89d..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 @@ -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; @@ -238,14 +237,15 @@ private boolean tryComplete(LocateRequest req, CompletableFuture { + .createSelector(replicaSelectorClass, conn.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, ", META_TABLE_NAME, e); + LOG.error("Failed to get table {}'s region replication, ", conn.getMetaTableName(), + 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(conn.getMetaTableName()).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..5e640c45a354 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; @@ -217,7 +216,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(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 b7ec7fcd8725..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 @@ -63,7 +63,7 @@ public CompletableFuture> getAllRegionLocations() { .thenApply(locs -> Arrays.asList(locs.getRegionLocations())); } CompletableFuture> future = ClientMetaTableAccessor - .getTableHRegionLocations(conn.getTable(TableName.META_TABLE_NAME), 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/Connection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java index 8220189d9b51..bebfa6addc25 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,17 @@ 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. + *

+ * @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 @@ -95,6 +106,15 @@ 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). + * @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 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..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 @@ -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,15 @@ 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()}. + *

+ * @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/MutableRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java index d6d8e00f7822..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 @@ -120,7 +120,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, !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); @@ -232,7 +232,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.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 f4a474957a2f..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 @@ -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; @@ -405,7 +404,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin { AsyncAdminBuilderBase builder) { this.connection = connection; this.retryTimer = retryTimer; - this.metaTable = connection.getTable(META_TABLE_NAME); + this.metaTable = connection.getTable(connection.getMetaTableName()); this.rpcTimeoutNs = builder.rpcTimeoutNs; this.operationTimeoutNs = builder.operationTimeoutNs; this.pauseNs = builder.pauseNs; @@ -1012,7 +1011,7 @@ List> adminCall(controller, stub, @Override public CompletableFuture> getRegions(TableName tableName) { - if (tableName.equals(META_TABLE_NAME)) { + if (tableName.equals(connection.getMetaTableName())) { return connection.registry.getMetaRegionLocations() .thenApply(locs -> Stream.of(locs.getRegionLocations()).map(HRegionLocation::getRegion) .collect(Collectors.toList())); @@ -1303,7 +1302,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 (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/client/RegionInfoBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java index ef927fd3a55b..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 @@ -42,6 +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 public static final RegionInfo FIRST_META_REGIONINFO = new MutableRegionInfo(1L, TableName.META_TABLE_NAME, RegionInfo.DEFAULT_REPLICA_ID); 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..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 @@ -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(TableName.META_TABLE_NAME)) { + 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 eed1a40a2c2f..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 @@ -616,7 +616,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(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 ebb43723b8f8..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 @@ -37,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.TableName; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.security.User; @@ -262,6 +263,17 @@ 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(() -> CompletableFuture.completedFuture(TableName.META_TABLE_NAME), + "ZKConnectionRegistry.getMetaTableName"); + } + @Override public String getConnectionString() { final String serverList = zk.getConnectString(); 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..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.TableName; import org.apache.hadoop.hbase.client.AsyncConnection; import org.apache.hadoop.hbase.client.AsyncTable; import org.apache.hadoop.hbase.client.Connection; @@ -73,7 +72,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(conn.getMetaTableName()); table. coprocessorService( AuthenticationProtos.AuthenticationService::newStub, @@ -102,7 +101,7 @@ static Token obtainToken(Connection conn) throws try { injectFault(); - meta = conn.getTable(TableName.META_TABLE_NAME); + 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/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..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 @@ -3325,7 +3325,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.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/DoNothingConnectionRegistry.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/DoNothingConnectionRegistry.java index 30d69d4b3f9e..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 @@ -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 f65c7ccb6e75..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 @@ -143,8 +143,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", + TableName.META_TABLE_NAME, null, user) { @Override CompletableFuture getMasterStub() { @@ -195,7 +195,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(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/TestAsyncConnectionTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java index e56fffbb2642..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 @@ -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,7 +73,7 @@ public CompletableFuture getActiveMaster() { return CompletableFuture.completedFuture(masterServer); } }; - conn = new AsyncConnectionImpl(CONF, registry, "test", 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 a7df92999d08..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 @@ -100,7 +100,7 @@ public void setUp() throws IOException { public CompletableFuture getMetaRegionLocations() { return CompletableFuture.completedFuture(locs); } - }, "test", null, user); + }, "test", TableName.META_TABLE_NAME, null, user); } @After @@ -147,38 +147,36 @@ public void testClearCacheServerName() { @Test public void testClearCacheTableName() { - conn.getLocator().clearCache(TableName.META_TABLE_NAME); + conn.getLocator().clearCache(conn.getMetaTableName()); SpanData span = waitSpan("AsyncRegionLocator.clearCache"); assertThat(span, allOf(hasStatusWithCode(StatusCode.OK), hasKind(SpanKind.INTERNAL), buildConnectionAttributesMatcher(conn), - buildTableAttributesMatcher(TableName.META_TABLE_NAME))); + buildTableAttributesMatcher(conn.getMetaTableName()))); } @Test public void testGetRegionLocation() { - conn.getLocator().getRegionLocation(TableName.META_TABLE_NAME, 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(TableName.META_TABLE_NAME), - 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 public void testGetRegionLocations() { - conn.getLocator().getRegionLocations(TableName.META_TABLE_NAME, 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 = 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(TableName.META_TABLE_NAME), 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 cb5431c35d3e..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 @@ -162,8 +162,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", + TableName.META_TABLE_NAME, null, user) { @Override AsyncRegionLocator getLocator() { @@ -237,7 +237,7 @@ public void testGetSystemTable() { @Test public void testGetMetaTable() { - conn.getTable(TableName.META_TABLE_NAME).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()); } @@ -268,7 +268,7 @@ public void testPutSystemTable() { @Test public void testPutMetaTable() { - conn.getTable(TableName.META_TABLE_NAME).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()); } @@ -296,7 +296,7 @@ public void testDeleteSystemTable() { @Test public void testDeleteMetaTable() { - conn.getTable(TableName.META_TABLE_NAME).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()); } @@ -327,7 +327,7 @@ public void testAppendSystemTable() { @Test public void testAppendMetaTable() { - conn.getTable(TableName.META_TABLE_NAME).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()); } @@ -355,7 +355,7 @@ public void testIncrementSystemTable() { @Test public void testIncrementMetaTable() { - conn.getTable(TableName.META_TABLE_NAME) + 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()); } @@ -393,7 +393,7 @@ public void testCheckAndPutSystemTable() { @Test public void testCheckAndPutMetaTable() { - conn.getTable(TableName.META_TABLE_NAME).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(); @@ -426,7 +426,7 @@ public void testCheckAndDeleteSystemTable() { @Test public void testCheckAndDeleteMetaTable() { - conn.getTable(TableName.META_TABLE_NAME).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(); @@ -467,7 +467,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(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(); @@ -555,7 +555,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(conn.getMetaTableName(), renewFuture, Optional.empty()); } private void testForTable(TableName tableName, CompletableFuture renewFuture, @@ -598,7 +598,7 @@ public void testBatchSystemTable() { @Test public void testBatchMetaTable() { - conn.getTable(TableName.META_TABLE_NAME).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/TestAsyncTableTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java index 2cecc974b6ef..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 @@ -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", + 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..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,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", 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", TableName.META_TABLE_NAME, null, + User.getCurrent()); metrics = impl.getConnectionMetrics(); assertTrue("Metrics should be present", metrics.isPresent()); @@ -132,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, null, user); + impl = new AsyncConnectionImpl(conf, null, null, TableName.META_TABLE_NAME, 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 0e6a53ca7c47..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 @@ -787,8 +787,13 @@ 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()) + // Check if this is a meta table (hbase:meta or hbase:meta_*) + return isMetaTable(tableName) ? MetaCellComparator.META_COMPARATOR : CellComparatorImpl.COMPARATOR; } + + 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 7f6e87ebf911..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; @@ -75,7 +74,8 @@ 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()) + // Check if this is a meta table (hbase:meta or hbase:meta_*) + return CellComparatorImpl.isMetaTable(tableName) ? MetaCellComparator.META_COMPARATOR : InnerStoreCellComparator.INNER_STORE_COMPARATOR; } 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..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 @@ -25,6 +25,8 @@ import org.apache.commons.lang3.ArrayUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; @@ -45,6 +47,8 @@ @InterfaceAudience.Public 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<>(); @@ -66,6 +70,7 @@ public final class TableName implements Comparable { + NAMESPACE_DELIM + ")?)" + "(?:" + VALID_TABLE_QUALIFIER_REGEX + "))"; /** The hbase:meta table's name. */ + @Deprecated public static final TableName META_TABLE_NAME = valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta"); @@ -85,9 +90,18 @@ 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(TableName.META_TABLE_NAME); + if ( + tn == null || !tn.getNamespaceAsString().equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR) + ) { + return false; + } + String qualifier = tn.getQualifierAsString(); + return qualifier.equals("meta") || qualifier.startsWith("meta_"); } /** @@ -288,8 +302,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 " + META_TABLE_NAME); + throw new IllegalArgumentException(OLD_META_STR + " no longer exists. The table has been " + + "renamed to hbase:meta or hbase:meta_suffix in conf"); } if (Bytes.equals(NamespaceDescriptor.DEFAULT_NAMESPACE_NAME, namespace)) { 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..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,6 +1205,11 @@ 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/TestMultiTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java index 7c136fa2a19f..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,6 +145,11 @@ 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..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,6 +231,11 @@ 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-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..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 @@ -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..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 @@ -698,4 +698,16 @@ public String toString() { protected abstract boolean cacheTableDescriptor(); protected abstract boolean clusterMode(); + + protected TableName getDefaultMetaTableName() { + return TableName.META_TABLE_NAME; + } + + @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/MetaTableAccessor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java index 05b049e27dbc..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(TableName.META_TABLE_NAME); + 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(TableName.META_TABLE_NAME)) { + 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(TableName.META_TABLE_NAME)) { + 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 hbase:meta", state); + LOG.info("Updated {} in {}", state, connection.getMetaTableName()); } /** 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..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,8 @@ 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, 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..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,13 +43,14 @@ private ClusterConnectionFactory() { private static AsyncClusterConnection createAsyncClusterConnection(Configuration conf, ConnectionRegistry registry, SocketAddress localAddress, User user) throws IOException { String clusterId = FutureUtils.get(registry.getClusterId()); + 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..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; /** @@ -68,6 +69,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/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java index 3cac1f319dae..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 @@ -133,7 +133,7 @@ private String getRegionIdFromOp(Row op) { } private boolean isMetaTableOp(ObserverContext e) { - return TableName.META_TABLE_NAME.equals(e.getEnvironment().getRegionInfo().getTable()); + return TableName.isMetaTableName(e.getEnvironment().getRegionInfo().getTable()); } private void clientMetricRegisterAndMark() { @@ -267,8 +267,8 @@ public void start(CoprocessorEnvironment env) throws IOException { if ( env instanceof RegionCoprocessorEnvironment && ((RegionCoprocessorEnvironment) env).getRegionInfo().getTable() != null - && ((RegionCoprocessorEnvironment) env).getRegionInfo().getTable() - .equals(TableName.META_TABLE_NAME) + && 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 c997f1c6e822..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 @@ -509,6 +509,14 @@ 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; + + private volatile TableName cachedMetaTableName; + /** * Initializes the HMaster. The steps are as follows: *

@@ -1016,6 +1024,8 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE masterRegion = MasterRegionFactory.create(this); rsListStorage = new MasterRegionServerList(masterRegion, this); + cachedMetaTableName = initMetaTableName(); + // Initialize the ServerManager and register it as a configuration observer this.serverManager = createServerManager(this, rsListStorage); this.configurationManager.registerObserver(this.serverManager); @@ -1092,7 +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(TableName.META_TABLE_NAME)) { + if (!this.assignmentManager.getRegionStates().hasTableRegionStates(getMetaTableName())) { Optional optProc = procedureExecutor.getProcedures().stream() .filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny(); initMetaProc = optProc.orElseGet(() -> { @@ -1156,7 +1166,7 @@ private void finishActiveMasterInitialization() throws IOException, InterruptedE return; } - TableDescriptor metaDescriptor = tableDescriptors.get(TableName.META_TABLE_NAME); + TableDescriptor metaDescriptor = tableDescriptors.get(getMetaTableName()); final ColumnFamilyDescriptor tableFamilyDesc = metaDescriptor.getColumnFamily(HConstants.TABLE_FAMILY); final ColumnFamilyDescriptor replBarrierFamilyDesc = @@ -1174,16 +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(TableName.META_TABLE_NAME); + 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(TableName.META_TABLE_NAME).size(); + assignmentManager.getRegionStates().getRegionsOfTable(getMetaTableName()).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)", + getMetaTableName(), metaDesc.getRegionReplication(), existingReplicasCount); metaDesc = TableDescriptorBuilder.newBuilder(metaDesc) .setRegionReplication(existingReplicasCount).build(); tableDescriptors.update(metaDesc); @@ -1193,7 +1204,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(), + getMetaTableName()); procedureExecutor.submitProcedure(new ModifyTableProcedure( procedureExecutor.getEnvironment(), TableDescriptorBuilder.newBuilder(metaDesc) .setRegionReplication(replicasNumInConf).build(), @@ -1423,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(TableName.META_TABLE_NAME, () -> 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"); } @@ -1652,6 +1664,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() { + return cachedMetaTableName; + } + + 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); + } + } + /* * 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 @@ -2601,7 +2640,7 @@ private void startActiveMasterManager(int infoPort) throws KeeperException { } private static boolean isCatalogTable(final TableName tableName) { - return tableName.equals(TableName.META_TABLE_NAME); + 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 464dfaca7035..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 @@ -84,7 +84,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 (TableName.isMetaTableName(tn)) { return META_TRANSITION_QOS; } } 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 new file mode 100644 index 000000000000..f356a10df4f4 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaTableNameStore.java @@ -0,0 +1,93 @@ +/* + * 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. + */ +@InterfaceAudience.Private +public class MetaTableNameStore { + private static final Logger LOG = LoggerFactory.getLogger(MetaTableNameStore.class); + 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; + 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. 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 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)); + cachedMetaTableName = metaTableName; + LOG.info("Successfully stored meta table name: {}", metaTableName); + } + + /** + * 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 + */ + public TableName load() throws IOException { + if (cachedMetaTableName != null) { + return cachedMetaTableName; + } + + synchronized (this) { + if (cachedMetaTableName != null) { + return cachedMetaTableName; + } + 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; + } + LOG.info("No stored meta table name found in Master Local Region: {}", cachedMetaTableName); + return cachedMetaTableName; + } + } +} 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..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 @@ -605,7 +605,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", connection.getMetaTableName()); Map> assignmentMap = plan.getAssignmentMap(); Map> planToUpdate = new HashMap<>(assignmentMap.size()); Map regionToRegionInfoMap = @@ -690,14 +690,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", + connection.getMetaTableName()); // 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", + connection.getMetaTableName()); } /** 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..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 @@ -79,7 +79,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(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; @@ -106,7 +106,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().getMetaTable(); ResultScanner scanner = table.getScanner(HConstants.NAMESPACE_FAMILY)) { for (Result result;;) { result = scanner.next(); @@ -204,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(TableName.META_TABLE_NAME)) { + try (Table table = conn.getMetaTable()) { table.put(put); } } @@ -212,7 +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(TableName.META_TABLE_NAME)) { + 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/TableStateManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/TableStateManager.java index 6ad32623be1a..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 @@ -86,7 +86,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(master.getConnection().getMetaTableName())) { // Can't delete the hbase:meta table. return; } @@ -147,7 +147,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(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 7bfa79e21484..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 @@ -354,7 +354,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 {} {}", master.getConnection().getMetaTableName(), regionNode); }, result); } } @@ -1962,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; hbase:meta=" + 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 c370fed9d9c0..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 @@ -718,8 +718,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.", + 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 5987fc7537b4..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 @@ -189,9 +189,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 hbase:meta row=") - .append(regionInfo.getEncodedName()).append(", regionState=").append(state); + final StringBuilder info = new StringBuilder("pid=").append(pid).append(" updating ") + .append(master.getConnection().getMetaTableName()).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 +283,8 @@ 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(master.getConnection().getMetaTableName()); future = table.put(put); } FutureUtils.addListener(future, (r, e) -> { @@ -329,8 +330,8 @@ private void multiMutate(RegionInfo ri, List mutations) throws IOExcep } } MutateRowsRequest request = builder.build(); - AsyncTable table = - master.getConnection().toAsyncConnection().getTable(TableName.META_TABLE_NAME); + AsyncTable table = master.getConnection().toAsyncConnection() + .getTable(master.getConnection().getMetaTableName()); CompletableFuture future = table. coprocessorService(MultiRowMutationService::newStub, (stub, controller, done) -> stub.mutateRows(controller, request, done), row); @@ -338,7 +339,7 @@ MutateRowsResponse> coprocessorService(MultiRowMutationService::newStub, } private Table getMetaTable() throws IOException { - return master.getConnection().getTable(TableName.META_TABLE_NAME); + return master.getConnection().getTable(master.getConnection().getMetaTableName()); } private Result getRegionCatalogResult(RegionInfo region) throws IOException { @@ -476,7 +477,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 { @@ -504,7 +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(TableName.META_TABLE_NAME)) { + try (Table table = master.getConnection().getTable(master.getConnection().getMetaTableName())) { table.delete(delete); } LOG.info( @@ -566,7 +566,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) 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..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 @@ -903,8 +903,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(), env.getMasterServices().getConnection().getMetaTableName()); 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..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 @@ -80,9 +80,8 @@ public synchronized void chore() { long deletedLastPushedSeqIds = 0; TableName tableName = null; List peerIds = null; - try (Table metaTable = conn.getTable(TableName.META_TABLE_NAME); - 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 9f5ff857d4d8..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 @@ -156,7 +156,7 @@ public TableName getScanTable() { public Results getResults() { final AsyncTable asyncTable = - connection.getTable(TableName.META_TABLE_NAME); + 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 0d3ddb43abd4..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 @@ -105,7 +105,8 @@ 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", + services.getConnection().getMetaTableName(), e); return false; } return true; @@ -145,7 +146,8 @@ 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", services.getConnection().getMetaTableName(), + e); } } @@ -484,7 +486,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(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/MetaFixer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java index 1244d5bf3525..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 @@ -203,19 +203,20 @@ 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(), + TableName.META_TABLE_NAME.getNameAsString()); 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(), - 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 hbase:meta failed.", 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/janitor/ReportMakingVisitor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/ReportMakingVisitor.java index c712f1cba672..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 @@ -137,8 +137,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(), + 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 e199f6d5971d..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,8 +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(TableName.META_TABLE_NAME); + try (Table metaTable = env.getMasterServices().getConnection().getTable(env.getMetaTableName()); ResultScanner scanner = metaTable.getScanner(tableScan)) { for (;;) { Result result = scanner.next(); @@ -405,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 " - + TableName.META_TABLE_NAME); + + 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 e8999b886afd..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(TableName.META_TABLE_NAME)) { + 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(TableName.META_TABLE_NAME)) { + 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 43d69361c2d2..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,14 +102,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'", env.getMetaTableName(), 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(), 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 2d54eaf6c58c..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 @@ -67,6 +67,8 @@ 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); + // For now, hardcode default. Future: pass metaTableName via constructor from Master + TableName metaTableName = TableName.META_TABLE_NAME; + 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; } @@ -1079,6 +1081,7 @@ public boolean waitMetaExclusiveLock(Procedure procedure) { return false; } waitProcedure(lock, procedure); + // TODO: Get dynamic name from MasterServices logLockedResource(LockedResourceType.META, TableName.META_TABLE_NAME.getNameAsString()); 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 3d313c9ac3ab..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,8 @@ @InterfaceAudience.Private class MetaQueue extends Queue { - protected MetaQueue(LockStatus lockStatus) { - super(TableName.META_TABLE_NAME, 1, lockStatus); + protected 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 dc9eac4c879d..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,7 +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(TableName.META_TABLE_NAME)) { + BufferedMutator mutator = conn.getBufferedMutator(env.getMetaTableName())) { for (Result result;;) { result = scanner.next(); if (result == null) { @@ -88,7 +88,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(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 3450f3059106..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,7 +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 hbase:meta 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 642df36d535f..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 @@ -174,6 +174,7 @@ List getLocks() { addToLockedResources(lockedResources, regionLocks, Function.identity(), LockedResourceType.REGION); addToLockedResources(lockedResources, peerLocks, Function.identity(), LockedResourceType.PEER); + // TODO: Get dynamic name from MasterServices addToLockedResources(lockedResources, ImmutableMap.of(TableName.META_TABLE_NAME, metaLock), tn -> tn.getNameAsString(), LockedResourceType.META); addToLockedResources(lockedResources, globalLocks, Function.identity(), @@ -236,6 +237,7 @@ public String toString() { .append("tableLocks", filterUnlocked(tableLocks)) .append("regionLocks", filterUnlocked(regionLocks)) .append("peerLocks", filterUnlocked(peerLocks)) + // TODO: Get dynamic name from MasterServices .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/master/procedure/TruncateRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateRegionProcedure.java index ef11e68217a5..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 @@ -231,7 +231,7 @@ public void toStringClassDetails(StringBuilder sb) { } private boolean prepareTruncate() throws IOException { - if (getTableName().equals(TableName.META_TABLE_NAME)) { + 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/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/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java index ba838e2f16ca..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 @@ -1928,7 +1928,7 @@ public OpenRegionResponse openRegion(final RpcController controller, tableName = ProtobufUtil.toTableName(ri.getTableName()); } } - if (!TableName.META_TABLE_NAME.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/replication/ReplicationBarrierFamilyFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/ReplicationBarrierFamilyFormat.java index 8bf32baada22..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 @@ -192,7 +192,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(conn.getMetaTableName()); ResultScanner scanner = table.getScanner(scan)) { for (Result result;;) { result = scanner.next(); @@ -215,7 +215,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(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 57d156ab1c2e..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 @@ -544,7 +544,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 (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 4d6f57e22edc..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 @@ -660,21 +660,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 " + + "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 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 " + + "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 hbase:meta." - + "Please use hbck tool to fix it first."); + + " 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..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 @@ -143,24 +143,30 @@ public static void tryUpdateMetaTableDescriptor(Configuration conf) throws IOExc CommonFSUtils.getRootDir(conf)); } + private static TableName getMetaTableNameFromConf(Configuration conf) { + // TODO: Support replica-specific meta table names from masterRegion + return TableName.META_TABLE_NAME; + } + 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. - Optional> opt = getTableDescriptorFromFs(fs, - CommonFSUtils.getTableDir(rootdir, TableName.META_TABLE_NAME), false); + TableName metaTableName = getMetaTableNameFromConf(conf); + Optional> opt = + getTableDescriptorFromFs(fs, 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 hbase:meta table descriptor {}", 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 hbase:meta table descriptor"); + throw new IOException("Failed update " + metaTableName + " table descriptor"); } - LOG.info("Updated hbase:meta table descriptor to {}", p); + LOG.info("Updated {} table descriptor to {}", metaTableName, p); return td; } @@ -198,7 +204,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(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 c3eafa7c11d1..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 @@ -540,7 +540,7 @@ public void run() { connection = ConnectionFactory.createConnection(getConf()); admin = connection.getAdmin(); - meta = connection.getTable(TableName.META_TABLE_NAME); + 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)); } @@ -660,17 +660,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 = connection.getMetaTableName() + " 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 " + connection.getMetaTableName() + + " 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 " + 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 hbase:meta table"); + LOG.info("Loading regionsinfo from the {} table", connection.getMetaTableName()); boolean success = loadMetaEntries(); if (!success) return -1; @@ -1219,8 +1221,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 hbase:meta: " - + emptyRegionInfoQualifiers.size()); + errors.print("Number of empty REGIONINFO_QUALIFIER rows in " + connection.getMetaTableName() + + ": " + emptyRegionInfoQualifiers.size()); if (details) { for (Result r : emptyRegionInfoQualifiers) { errors.print(" " + r); @@ -1371,7 +1373,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.", connection.getMetaTableName()); for (Result region : emptyRegionInfoQualifiers) { deleteMetaRegion(region.getRow()); errors.getErrorList().remove(ERROR_CODE.EMPTY_META_CELL); @@ -1574,8 +1576,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(connection.getMetaTableName(), + new TableState(connection.getMetaTableName(), TableState.State.ENABLED)); } /** @@ -1604,7 +1606,7 @@ public void loadHdfsRegionDirs() throws IOException, InterruptedException { TableName tableName = CommonFSUtils.getTableName(path); if ( (!checkMetaOnly && isTableIncluded(tableName)) - || tableName.equals(TableName.META_TABLE_NAME) + || tableName.equals(connection.getMetaTableName()) ) { tableDirs.add(fs.getFileStatus(path)); } @@ -1649,7 +1651,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(connection.getMetaTableName())) { locs = locator.getRegionLocations(HConstants.EMPTY_START_ROW, true); } if (locs == null || locs.isEmpty()) { @@ -2019,9 +2021,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.", + connection.getMetaTableName()); continue; } // close the region -- close files and remove assignment @@ -2140,8 +2144,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 " + + connection.getMetaTableName() + " but " + "deployed on " + + Joiner.on(", ").join(hbi.getDeployedOn())); if (shouldFixAssignments()) { undeployRegions(hbi); } @@ -2155,8 +2160,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 " + connection.getMetaTableName() + + " or deployed on any region server"); // restore region consistency of an adopted orphan if (shouldFixMeta()) { if (!hbi.isHdfsRegioninfoPresent()) { @@ -2196,7 +2202,8 @@ else if (!inMeta && !inHdfs && !isDeployed) { } } } - LOG.info("Patching hbase:meta with .regioninfo: " + hbi.getHdfsHRI()); + 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(), @@ -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(), + connection.getMetaTableName()); 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 " + 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 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 " + 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 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(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. @@ -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, + connection.getMetaTableName() + ", 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 " + connection.getMetaTableName() + + ", 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()); + 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 hbase:meta.."); + errors.print("Trying to undeploy excess replica, replicaId: " + entry.getKey() + " of " + + connection.getMetaTableName() + ".."); 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."); + connection.getMetaTableName() + ", 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 " + connection.getMetaTableName() + ".."); 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 " + connection.getMetaTableName()); 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 " + connection.getMetaTableName()); 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 " + connection.getMetaTableName() + " 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..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 @@ -28,7 +28,6 @@ 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.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.AsyncClusterConnection; @@ -149,7 +148,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(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 c1f98edd75ab..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 @@ -586,13 +586,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 {}", conn.getMetaTableName(), + 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: {}", conn.getMetaTableName(), + 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..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 @@ -17,8 +17,6 @@ */ package org.apache.hadoop.hbase.wal; -import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME; - import java.io.IOException; import java.io.InterruptedIOException; import java.util.HashMap; @@ -78,7 +76,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 = 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 1f1aad61ba70..9f79df924014 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/catalogTables.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/catalogTables.jsp @@ -56,7 +56,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 d88d968e199f..2a300e2629bc 100644 --- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp @@ -195,8 +195,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(TableName.META_TABLE_NAME).getRegionReplication(); + int numMetaReplicas = master.getTableDescriptors() + .get(master.getConnection().getMetaTableName()).getRegionReplication(); Map frags = null; if (showFragmentation) { frags = FSUtils.getTableFragmentation(master); @@ -317,7 +317,7 @@

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

Table Regions

@@ -653,7 +653,7 @@
- +
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..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 @@ -44,6 +44,11 @@ public Configuration getConfiguration() { return null; } + @Override + public TableName getMetaTableName() { + return null; + } + @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 0bfbd18eb32f..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 @@ -106,10 +106,10 @@ 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); - TEST_UTIL.waitFor(30000, - () -> TEST_UTIL.getMiniHBaseCluster().getRegions(TableName.META_TABLE_NAME).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) { @@ -129,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(), null, User.getCurrent()); + conn = new AsyncConnectionImpl(c, registry, registry.getClusterId().get(), + 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 4529c07dfd13..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(), 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/TestAsyncRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocator.java index 5e858eb8d9d6..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(), 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 baa4ee74ade9..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(), null, User.getCurrent()); + CONN = + new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry, registry.getClusterId().get(), + TEST_UTIL.getConnection().getMetaTableName(), null, User.getCurrent()); } @AfterClass @@ -165,7 +166,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(), + TEST_UTIL.getConnection().getMetaTableName(), 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 184b4ba0d3cc..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 @@ -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; @@ -72,14 +71,14 @@ public static void setUp() throws Exception { admin.balancerSwitch(false, true); // Enable hbase:meta replication. - HBaseTestingUtil.setReplicas(admin, TableName.META_TABLE_NAME, numOfMetaReplica); - TEST_UTIL.waitFor(30000, - () -> TEST_UTIL.getMiniHBaseCluster().getRegions(TableName.META_TABLE_NAME).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(), null, - User.getCurrent()); + CONN = new AsyncConnectionImpl(conf, registry, registry.getClusterId().get(), + TableName.META_TABLE_NAME, null, User.getCurrent()); } @AfterClass @@ -94,18 +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, META_TABLE_NAME, 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); - } - 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]; @@ -116,20 +116,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, TableName.META_TABLE_NAME, 1); - TEST_UTIL.waitFor(30000, - () -> TEST_UTIL.getMiniHBaseCluster().getRegions(TableName.META_TABLE_NAME).size() == 1); + HBaseTestingUtil.setReplicas(admin, TEST_UTIL.getConnection().getMetaTableName(), 1); + TEST_UTIL.waitFor(30000, () -> TEST_UTIL.getMiniHBaseCluster() + .getRegions(TEST_UTIL.getConnection().getMetaTableName()).size() == 1); CatalogReplicaLoadBalanceSelector metaSelectorWithNoReplica = - CatalogReplicaLoadBalanceSelectorFactory.createSelector(replicaSelectorClass, META_TABLE_NAME, - CONN, () -> { + 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 {}'s region replication, ", META_TABLE_NAME, 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/TestFallbackToUseReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFallbackToUseReplay.java index 44c096f0b8a9..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 @@ -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", + TableName.META_TABLE_NAME, null, User.getCurrent()) { @Override AsyncRegionLocator getLocator() { 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; diff --git a/hbase-shell/src/main/ruby/hbase/table.rb b/hbase-shell/src/main/ruby/hbase/table.rb index aed5e1c26e59..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::META_TABLE_NAME.equals(@table.getName) + org.apache.hadoop.hbase.TableName.isMetaTableName(@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/HBaseTestingUtility.java b/hbase-testing-util/src/main/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 698e433bb744..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(TableName.META_TABLE_NAME); + 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(TableName.META_TABLE_NAME); + 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(TableName.META_TABLE_NAME); + 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(TableName.META_TABLE_NAME); + 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(TableName.META_TABLE_NAME); + 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(TableName.META_TABLE_NAME).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(TableName.META_TABLE_NAME); + 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(TableName.META_TABLE_NAME)) { + 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(TableName.META_TABLE_NAME); + 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(TableName.META_TABLE_NAME)) { + 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/thrift/ThriftHBaseServiceHandler.java b/hbase-thrift/src/main/java/org/apache/hadoop/hbase/thrift/ThriftHBaseServiceHandler.java index 1900c6c0f8da..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,12 +1072,12 @@ private void closeTable(Table table) throws IOError { public TRegionInfo getRegionInfo(ByteBuffer searchRow) throws IOError { try { byte[] row = getBytes(searchRow); - Result startRowResult = - getReverseScanResult(TableName.META_TABLE_NAME.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 " + TableName.META_TABLE_NAME + ", 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-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"); 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..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 @@ -182,8 +182,8 @@ public static void setMetaLocation(ZKWatcher zookeeper, ServerName serverName, i if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) { LOG.debug("hbase:meta region location doesn't exist, create it"); } else { - LOG.debug( - "hbase:meta region location doesn't exist for replicaId=" + replicaId + ", create it"); + LOG.debug("hbase:meta region location doesn't exist for replicaId={}, create it", + replicaId); } ZKUtil.createAndWatch(zookeeper, zookeeper.getZNodePaths().getZNodeForReplica(replicaId), data);