kgeisz commented on code in PR #7558:
URL: https://github.com/apache/hbase/pull/7558#discussion_r2739444453


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/client/AsyncClusterConnectionImpl.java:
##########
@@ -59,8 +59,9 @@
 class AsyncClusterConnectionImpl extends AsyncConnectionImpl implements 
AsyncClusterConnection {
 
   public AsyncClusterConnectionImpl(Configuration conf, ConnectionRegistry 
registry,
-    String clusterId, SocketAddress localAddress, User user) {
-    super(conf, registry, clusterId, localAddress, user, 
Collections.emptyMap());
+    String clusterId, org.apache.hadoop.hbase.TableName metaTableName, 
SocketAddress localAddress,

Review Comment:
   nit: `TableName` is imported on Line 29
   ```suggestion
       String clusterId, TableName metaTableName, SocketAddress localAddress,
   ```



##########
hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java:
##########
@@ -85,9 +90,17 @@ public final class TableName implements 
Comparable<TableName> {
   /** One globally disallowed name */
   public static final String DISALLOWED_TABLE_NAME = "zookeeper";
 
-  /** Returns True if <code>tn</code> is the hbase:meta table name. */
+  /**
+   * Returns True if <code>tn</code> 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) return false;
+    if 
(!tn.getNamespaceAsString().equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR))
 {
+      return false;
+    }

Review Comment:
   nit (not a big deal if you prefer the current style):
   ```suggestion
       if (tn == null || 
!tn.getNamespaceAsString().equals(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR))
 {
         return false;
       }
   ```



##########
hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java:
##########
@@ -197,32 +197,23 @@ public void testKeyValueBorderCases() {
 
   private void metacomparisons(final CellComparatorImpl c) {
     long now = EnvironmentEdgeManager.currentTime();
-    assertTrue(c.compare(
-      new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + 
",a,,0,1"), now),
-      new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + 
",a,,0,1"), now))
-        == 0);
-    KeyValue a =
-      new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + 
",a,,0,1"), now);
-    KeyValue b =
-      new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() + 
",a,,0,2"), now);
+    assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), 
now),

Review Comment:
   Same in this file:
   
   ```suggestion
       assertTrue(c.compare(new KeyValue(Bytes.toBytes("hbase:meta,a,,0,1"), 
now),
   ```



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/client/ClusterConnectionFactory.java:
##########
@@ -42,13 +42,14 @@ private ClusterConnectionFactory() {
   private static AsyncClusterConnection 
createAsyncClusterConnection(Configuration conf,
     ConnectionRegistry registry, SocketAddress localAddress, User user) throws 
IOException {
     String clusterId = FutureUtils.get(registry.getClusterId());
+    org.apache.hadoop.hbase.TableName metaTableName = 
FutureUtils.get(registry.getMetaTableName());

Review Comment:
   Edit: Is this happening because of maven spotless?  If so, then you can 
ignore this comment and the similar ones below.
   
   nit: add `import org.apache.hadoop.hbase.TableName;` at the top of the file 
and:
   
   
   ```suggestion
       TableName metaTableName = FutureUtils.get(registry.getMetaTableName());
   ```



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java:
##########
@@ -1092,7 +1102,10 @@ 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(getConnection().getMetaTableName())

Review Comment:
   Are we able to use `HMaster`'s `getMetaTableName()` instead of 
`getConnection().getMetaTableName()` here and below on later lines in the code? 
 I see `cachedMetaTableName` was initialized above on Line 1027.



##########
hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java:
##########
@@ -104,6 +104,11 @@ public MockRegistry(Configuration conf, User user) {
     public CompletableFuture<String> getClusterId() {
       return CompletableFuture.completedFuture("clusterId");
     }
+
+    @Override
+    public CompletableFuture<org.apache.hadoop.hbase.TableName> 
getMetaTableName() {
+      return 
CompletableFuture.completedFuture(org.apache.hadoop.hbase.TableName.META_TABLE_NAME);

Review Comment:
   nit: `import org.apache.hadoop.hbase.TableName` and:
   
   ```suggestion
       public CompletableFuture<TableName> getMetaTableName() {
         return CompletableFuture.completedFuture(TableName.META_TABLE_NAME);
   ```



##########
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java:
##########
@@ -71,25 +71,27 @@ public class TestRegionInfo {
   public void testIsStart() {
     assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isFirst());
     org.apache.hadoop.hbase.client.RegionInfo ri = 
org.apache.hadoop.hbase.client.RegionInfoBuilder
-      
.newBuilder(TableName.META_TABLE_NAME).setStartKey(Bytes.toBytes("not_start")).build();
+      .newBuilder(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable())

Review Comment:
   Why are we now using `FIRST_META_REGIONINFO` here?  I have seen this change 
in other parts of the PR as well.



##########
hbase-common/src/main/java/org/apache/hadoop/hbase/InnerStoreCellComparator.java:
##########
@@ -75,8 +75,13 @@ 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 isMetaTable(tableName)
       ? MetaCellComparator.META_COMPARATOR
       : InnerStoreCellComparator.INNER_STORE_COMPARATOR;
   }
+
+  private static boolean isMetaTable(byte[] tableName) {

Review Comment:
   nit: You defined `isMetaTable()` already in `CellComparatorImpl`.  
`InnerStoreCellComparator` extends `CellComparatorImpl`, so you do not need to 
redefine `isMetaTable()` here.



##########
hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java:
##########
@@ -197,38 +197,30 @@ public void testMetaComparisons2() {
     CellComparator c = MetaCellComparator.META_COMPARATOR;
     assertTrue(c.compare(
       createByteBufferKeyValueFromKeyValue(
-        new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString() 
+ ",a,,0,1"), now)),
+        new KeyValue(Bytes.toBytes("hbase:meta" + ",a,,0,1"), now)),

Review Comment:
   nit: Can't this just be:
   ```suggestion
           new KeyValue(Bytes.toBytes("hbase:meta,a,,0,1"), now)),
   ```
   Also similar for the other instances of this in this file.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java:
##########
@@ -1079,7 +1081,8 @@ public boolean waitMetaExclusiveLock(Procedure<?> 
procedure) {
         return false;
       }
       waitProcedure(lock, procedure);
-      logLockedResource(LockedResourceType.META, 
TableName.META_TABLE_NAME.getNameAsString());
+      // TODO: Get dynamic name from MasterServices
+      logLockedResource(LockedResourceType.META, "hbase:meta");

Review Comment:
   I see the comment, so hard-coding `hbase:meta` here is intentional? Just 
double checking.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/janitor/MetaFixer.java:
##########
@@ -203,19 +203,19 @@ private static List<RegionInfo> createMetaEntries(final 
MasterServices masterSer
         .flatMap(List::stream).collect(Collectors.toList());
     final List<IOException> 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(),
+      "hbase:meta");

Review Comment:
   Do we want `hbase:meta` hard-coded here? Same with Line 211 and Line 218.  
If hard-coding is what we want, then `hbase:meta` can be part of the string.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureScheduler.java:
##########
@@ -1079,7 +1081,8 @@ public boolean waitMetaExclusiveLock(Procedure<?> 
procedure) {
         return false;
       }
       waitProcedure(lock, procedure);
-      logLockedResource(LockedResourceType.META, 
TableName.META_TABLE_NAME.getNameAsString());
+      // TODO: Get dynamic name from MasterServices
+      logLockedResource(LockedResourceType.META, "hbase:meta");

Review Comment:
   I'm double checking like a previous comment I had: Is hard-coding 
`"hbase:meta"` intentional here? I have seen this TODO in other places, but the 
code was using `TableName.valueOf("hbase", "meta")`.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/client/ShortCircuitConnectionRegistry.java:
##########
@@ -68,6 +68,11 @@ public CompletableFuture<ServerName> getActiveMaster() {
     return future;
   }
 
+  @Override
+  public CompletableFuture<org.apache.hadoop.hbase.TableName> 
getMetaTableName() {

Review Comment:
   nit
   ```suggestion
     public CompletableFuture<TableName> getMetaTableName() {
   ```



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java:
##########
@@ -105,7 +107,8 @@ protected Flow executeFromState(MasterProcedureEnv env, 
InitMetaState state)
         case INIT_META_WRITE_FS_LAYOUT:
           Configuration conf = env.getMasterConfiguration();
           Path rootDir = CommonFSUtils.getRootDir(conf);
-          TableDescriptor td = writeFsLayout(rootDir, env);
+          TableDescriptor td =
+            writeFsLayout(rootDir, env, 
env.getMasterServices().getMetaTableName());

Review Comment:
   Similar to the previous comment here, but in this case it looks like you 
didn't need `.getConnection()`. Maybe the getter could be this instead?
   ```
   public TableName getMetaTableName() {
    return this.getMasterServices().getMetaTableName();
   }
   ```



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java:
##########
@@ -1174,16 +1187,18 @@ 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(getConnection().getMetaTableName());
       if (metaDesc.getRegionReplication() != replicasNumInConf) {
         // it is possible that we already have some replicas before upgrading, 
so we must set the
         // region replication number in meta TableDescriptor directly first, 
without creating a
         // ModifyTableProcedure, otherwise it may cause a double assign for 
the meta replicas.
-        int existingReplicasCount =
-          
assignmentManager.getRegionStates().getRegionsOfTable(TableName.META_TABLE_NAME).size();
+        int existingReplicasCount = assignmentManager.getRegionStates()
+          .getRegionsOfTable(getConnection().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)",

Review Comment:
   nit
   
   ```suggestion
               "Update replica count of {} from {} (in TableDescriptor)" + " to 
{} (existing ZNodes)",
   ```



##########
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.
+ */
[email protected]
+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);

Review Comment:
   nit: Maybe we should add a little more info here:
   
   ```suggestion
         LOG.info("No stored meta table name found in Master Local Region. 
Using cached meta table name: {}", cachedMetaTableName);
   ```



##########
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java:
##########
@@ -95,7 +94,8 @@ public void testContendedLogRolling() throws Exception {
     CommonFSUtils.setRootDir(conf, dir);
     FSTableDescriptors fsTableDescriptors = new 
FSTableDescriptors(TEST_UTIL.getConfiguration());
     
FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration());
-    TableDescriptor metaTableDescriptor = 
fsTableDescriptors.get(TableName.META_TABLE_NAME);
+    TableDescriptor metaTableDescriptor =
+      
fsTableDescriptors.get(org.apache.hadoop.hbase.TableName.valueOf("hbase:meta"));

Review Comment:
   nit (also on Line 162 and Line 179)
   
   ```suggestion
         fsTableDescriptors.get(TableName.valueOf("hbase:meta"));
   ```



##########
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.
+ */
[email protected]
+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;
+      }

Review Comment:
   Why is this needed in the `synchronized` block?  I see it is at the 
beginning of the `load()` method already.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java:
##########
@@ -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.";

Review Comment:
   nit:
   
   ```suggestion
             + " is back to consistent state.";
   ```



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java:
##########
@@ -111,7 +111,7 @@ protected Flow executeFromState(final MasterProcedureEnv 
env, final DisableTable
           ) {
             MasterFileSystem fs = env.getMasterFileSystem();
             try (BufferedMutator mutator = 
env.getMasterServices().getConnection()
-              .getBufferedMutator(TableName.META_TABLE_NAME)) {
+              
.getBufferedMutator(env.getMasterServices().getConnection().getMetaTableName()))
 {

Review Comment:
   I see `env.getMasterServices().getConnection().getMetaTableName()` is used 
in several places throughout the code.  Maybe we should add a getter to 
`MasterProcedureEnv` so this is more readable?
   ```
   public TableName getMetaTableName() {
    return this.getMasterServices().getConnection().getMetaTableName();
   }
   ```



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/tool/BulkLoadHFilesTool.java:
##########
@@ -660,21 +660,21 @@ private int getRegionIndex(List<Pair<byte[], byte[]>> 
startEndKeys, byte[] key)
   private void checkRegionIndexValid(int idx, List<Pair<byte[], byte[]>> 
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.");

Review Comment:
   nit: Hard-coded `hbase:meta` here and below on Line 670 and Line 676



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to