You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/09/23 09:59:45 UTC

[GitHub] [hbase] francisliu opened a new pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

francisliu opened a new pull request #2445:
URL: https://github.com/apache/hbase/pull/2445


   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506769592



##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
+    throws IOException {
+    byte[] row = regionName;
+    RegionInfo parsedInfo = null;
+    try {
+      parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
+      row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);
+    } catch (Exception parseEx) {
+      return null;
+      // Ignore. This is used with tableName passed as regionName.
+    }
+    Get get = new Get(row);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    Result r;
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(parsedInfo.getTable()))) {
+      r = t.get(get);
+    }
+    RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+    return locations == null ? null :
+      locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionInfo region information
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
+    throws IOException {
+    return CatalogFamilyFormat.getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
+      regionInfo, regionInfo.getReplicaId());
+  }
+
+  /**
+   * @return Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta.
+   */
+  public static Result getCatalogFamilyRow(Connection connection, RegionInfo ri)
+    throws IOException {
+    Get get = new Get(CatalogFamilyFormat.getMetaKeyForRegion(ri));
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(ri.getTable()))) {
+      return t.get(get);
+    }
+  }
+
+  public static TableName getCatalogTableForTable(TableName tableName) {
+    if (TableName.ROOT_TABLE_NAME.equals(tableName)) {
+      throw new IllegalStateException("Can't get catalog table for hbase:root table");
+    }
+    if (TableName.META_TABLE_NAME.equals(tableName)) {
+      return TableName.ROOT_TABLE_NAME;
+    }
+    return TableName.META_TABLE_NAME;
+  }
+
+  public static boolean isCatalogTable(TableName tableName) {
+    return tableName.equals(TableName.ROOT_TABLE_NAME) ||
+      tableName.equals(TableName.META_TABLE_NAME);
+  }
+
+  /**
+   * Gets the result in hbase:meta for the specified region.
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return result of the specified region
+   */
+  public static Result getRegionResult(Connection connection,
+      byte[] regionName) throws IOException {
+    Table catalogTable = null;
+    if (Bytes.equals(RegionInfoBuilder.ROOT_REGIONINFO.getRegionName(), regionName)) {
+      throw new IllegalStateException("This method cannot be used for hbase:root region");
+    }
+    catalogTable = getCatalogHTable(connection,
+      getCatalogTableForTable(
+        CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName).getTable()));
+    Get get = new Get(regionName);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = catalogTable) {
+      return t.get(get);
+    }
+  }
+
+  /**
+   * Scans META table for a row whose key contains the specified <B>regionEncodedName</B>, returning
+   * a single related <code>Result</code> instance if any row is found, null otherwise.
+   * @param connection the connection to query META table.
+   * @param regionEncodedName the region encoded name to look for at META.
+   * @return <code>Result</code> instance with the row related info in META, null otherwise.
+   * @throws IOException if any errors occur while querying META.
+   */
+  public static Result scanByRegionEncodedName(Connection connection, String regionEncodedName)
+    throws IOException {
+    RowFilter rowFilter =
+      new RowFilter(CompareOperator.EQUAL, new SubstringComparator(regionEncodedName));
+    Scan scan = getCatalogScan(connection, 1);
+    scan.setFilter(rowFilter);
+    Result res = null;
+    try (ResultScanner resultScanner =
+      getCatalogHTable(connection, TableName.META_TABLE_NAME).getScanner(scan)) {
+      res = resultScanner.next();
+    }
+    if (res == null) {
+      scan = getCatalogScan(connection, 1);
+      scan.setFilter(rowFilter);
+      try (ResultScanner resultScanner =
+        getCatalogHTable(connection, TableName.ROOT_TABLE_NAME).getScanner(scan)) {
+        res = resultScanner.next();
+      }
+    }
+    return res;
+  }
+
+  /**
+   * Lists all of the regions currently in META.
+   * @param connection to connect with
+   * @param excludeOfflinedSplitParents False if we are to include offlined/splitparents regions,
+   *          true and we'll leave out offlined regions from returned list
+   * @return List of all user-space regions.
+   */
+  @VisibleForTesting
+  public static List<RegionInfo> getAllRegions(Connection connection,
+    boolean excludeOfflinedSplitParents) throws IOException {
+    List<Pair<RegionInfo, ServerName>> result;
+
+    result = getTableRegionsAndLocations(connection, null, excludeOfflinedSplitParents);
+
+    return getListOfRegionInfos(result);
+
+  }
+
+  /**
+   * Gets all of the regions of the specified table. Do not use this method to get meta table
+   * regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @return Ordered list of {@link RegionInfo}.
+   */
+  public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName)
+  throws IOException {
+    return getTableRegions(connection, tableName, false);
+  }
+
+  /**
+   * Gets all of the regions of the specified table. Do not use this method to get meta table
+   * regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @param excludeOfflinedSplitParents If true, do not include offlined split parents in the
+   *          return.
+   * @return Ordered list of {@link RegionInfo}.
+   */
+  public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName,
+    final boolean excludeOfflinedSplitParents) throws IOException {
+    List<Pair<RegionInfo, ServerName>> result =
+      getTableRegionsAndLocations(connection, tableName, excludeOfflinedSplitParents);
+    return getListOfRegionInfos(result);
+  }
+
+  private static List<RegionInfo>
+    getListOfRegionInfos(final List<Pair<RegionInfo, ServerName>> pairs) {
+    if (pairs == null || pairs.isEmpty()) {
+      return Collections.emptyList();
+    }
+    List<RegionInfo> result = new ArrayList<>(pairs.size());
+    for (Pair<RegionInfo, ServerName> pair : pairs) {
+      result.add(pair.getFirst());
+    }
+    return result;
+  }
+
+  /**
+   * This method creates a Scan object that will only scan catalog rows that belong to the specified
+   * table. It doesn't specify any columns. This is a better alternative to just using a start row
+   * and scan until it hits a new table since that requires parsing the HRI to get the table name.
+   * @param tableName bytes of table's name
+   * @return configured Scan object
+   * @deprecated This is internal so please remove it when we get a chance.
+   */
+  @Deprecated
+  public static Scan getScanForTableName(Connection connection, TableName tableName) {
+    // Start key is just the table name with delimiters
+    byte[] startKey = ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION);
+    // Stop key appends the smallest possible char to the table name
+    byte[] stopKey = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);
+
+    Scan scan = getCatalogScan(connection, -1);
+    scan.withStartRow(startKey);
+    scan.withStopRow(stopKey);
+    return scan;
+  }
+
+  private static Scan getCatalogScan(Connection connection, int rowUpperLimit) {
+    Scan scan = new Scan();
+    int scannerCaching = connection.getConfiguration().getInt(HConstants.HBASE_META_SCANNER_CACHING,
+      HConstants.DEFAULT_HBASE_META_SCANNER_CACHING);
+    if (connection.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS,
+      HConstants.DEFAULT_USE_META_REPLICAS)) {
+      scan.setConsistency(Consistency.TIMELINE);
+    }
+    if (rowUpperLimit > 0) {
+      scan.setLimit(rowUpperLimit);
+      scan.setReadType(Scan.ReadType.PREAD);
+    }
+    scan.setCaching(scannerCaching);
+    return scan;
+  }
+
+  /**
+   * Do not use this method to get meta table regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @return Return list of regioninfos and server.
+   */
+  public static List<Pair<RegionInfo, ServerName>>
+    getTableRegionsAndLocations(Connection connection, TableName tableName) throws IOException {
+    return getTableRegionsAndLocations(connection, tableName, true);
+  }
+
+  /**
+   * Do not use this method to get meta table regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table to work with, can be null for getting all regions
+   * @param excludeOfflinedSplitParents don't return split parents
+   * @return Return list of regioninfos and server addresses.
+   */
+  // What happens here when 1M regions in hbase:meta? This won't scale?
+  public static List<Pair<RegionInfo, ServerName>> getTableRegionsAndLocations(
+    Connection connection, @Nullable final TableName tableName,
+    final boolean excludeOfflinedSplitParents) throws IOException {
+    if (tableName != null && tableName.equals(TableName.ROOT_TABLE_NAME)) {
+      throw new IOException(
+        "This method can't be used to locate root regions; use RootTableLocator instead");
+    }
+    // Make a version of CollectingVisitor that collects RegionInfo and ServerAddress
+    ClientMetaTableAccessor.CollectRegionLocationsVisitor visitor =
+      new ClientMetaTableAccessor.CollectRegionLocationsVisitor(excludeOfflinedSplitParents);
+    byte[] startRow = ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION);
+    byte[] stopRow = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);
+    TableName parentTable = TableName.META_TABLE_NAME;
+    if (TableName.META_TABLE_NAME.equals(tableName)) {
+      parentTable = TableName.ROOT_TABLE_NAME;
+      startRow = null;
+      stopRow = null;
+    }
+
+    scanCatalog(connection,
+      parentTable,
+      startRow,
+      stopRow,
+      QueryType.REGION,
+      Integer.MAX_VALUE,
+      visitor);
+    return visitor.getResults();
+  }
+
+  public static void fullScanMetaAndPrint(Connection connection) throws IOException {

Review comment:
       Yeah keeping legacy apis I don't need to change just as part of this PR to avoid doing pervasive renames.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506767689



##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
+    throws IOException {
+    byte[] row = regionName;
+    RegionInfo parsedInfo = null;
+    try {
+      parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
+      row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);
+    } catch (Exception parseEx) {
+      return null;
+      // Ignore. This is used with tableName passed as regionName.

Review comment:
       Yeah this did happen if I remember right. I'll update the javadoc then? Not sure I understand what you by is it normal?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] francisliu commented on pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
francisliu commented on pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#issuecomment-720392056


   Rebased and addressed @saintstack  comments. I still need to check .94 for root comparator tests as well as write a doc.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506773956



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
##########
@@ -2391,42 +2406,91 @@ public void run(Timeout timeout) throws Exception {
       return failedFuture(new IllegalArgumentException("Passed region name can't be null"));
     }
     try {
+      TableName parentTable;
       CompletableFuture<Optional<HRegionLocation>> future;
       if (RegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) {
         String encodedName = Bytes.toString(regionNameOrEncodedRegionName);
-        if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) {
-          // old format encodedName, should be meta region
+
+        //TODO francis do we really need to support encoded name for root?
+        boolean isRoot = false;
+        for (int i = 0; i< numRootReplicas; i++) {
+          RegionInfo info =
+            RegionReplicaUtil.getRegionInfoForReplica(RegionInfoBuilder.ROOT_REGIONINFO, i);

Review comment:
       There was support for meta region when it was "root". I'm just keeping that feature parity. Should I skip support?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506770139



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ClientMetaTableAccessor.java
##########
@@ -193,15 +193,15 @@ private ClientMetaTableAccessor() {
    * @param tableName table we're looking for, can be null for getting all regions
    * @param excludeOfflinedSplitParents don't return split parents
    * @return the list of regioninfos and server. The return value will be wrapped by a

Review comment:
       Yes it does. Will do.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#issuecomment-720446891






----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506779022



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -1926,6 +2225,27 @@ public void markRegionAsMerged(final RegionInfo child, final ServerName serverNa
     }
     TableDescriptor td = master.getTableDescriptors().get(child.getTable());
     regionStateStore.mergeRegions(child, mergeParents, serverName, td);
+
+    //Split meta assignment event
+    if (td.isMetaTable()) {
+      ProcedureEvent<?> parentEvent[] = new ProcedureEvent[mergeParents.length];
+      for (int i=0; i<parentEvent.length; i++) {
+        parentEvent[i] = metaAssignEventMap.get(mergeParents[i].getStartKey());
+      }
+
+      metaAssignEventMap.put(child.getStartKey(),
+        new ProcedureEvent<>("meta assign: " + child.getRegionNameAsString()));
+
+      //wake the procedures waiting on parent event, the procedures will awaken
+      //and wait on newly created child event
+      for (int i= mergeParents.length-1; i>=1; i--) {
+        metaAssignEventMap.remove(mergeParents[i].getStartKey());
+        parentEvent[i].wake(getProcedureScheduler());
+      }
+      //for the first key we don't remove since we already replaced it
+      parentEvent[0].wake(getProcedureScheduler());
+    }
+
     if (shouldAssignFavoredNodes(child)) {
       getFavoredNodePromoter().generateFavoredNodesForMergedRegion(child, mergeParents);
     }

Review comment:
       That's true today tho? Or I'm missing something? Let me know so I can see if that scenario is covered in UT.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
##########
@@ -257,46 +327,62 @@ private void cleanupSplitDir(MasterProcedureEnv env) {
         // old hbase:meta tenancy on this server; clean these up if any before trying to remove the
         // WAL directory of this server or we will fail. See archiveMetaLog comment for more details
         // on this condition.
-        env.getMasterServices().getMasterWalManager().archiveMetaLog(this.serverName);
+        env.getMasterServices().getMasterWalManager().archiveCatalogLog(this.serverName, false);
       }
       splitWALManager.deleteWALDir(serverName);
     } catch (IOException e) {
       LOG.warn("Remove WAL directory for {} failed, ignore...{}", serverName, e.getMessage());
     }
   }
 
-  private boolean isSplittingDone(MasterProcedureEnv env, boolean splitMeta) {
+  private boolean isSplittingDone(MasterProcedureEnv env, SplitWALManager.SplitType splitType) {
     SplitWALManager splitWALManager = env.getMasterServices().getSplitWALManager();
     try {
-      int wals = splitWALManager.getWALsToSplit(serverName, splitMeta).size();
-      LOG.debug("Check if {} WAL splitting is done? wals={}, meta={}", serverName, wals, splitMeta);
+      int wals = splitWALManager.getWALsToSplit(serverName, splitType).size();
+      LOG.debug("Check if {} WAL splitting is done? wals={}, SplitType={}",
+        serverName, wals, splitType);
       return wals == 0;
     } catch (IOException e) {
       LOG.warn("Get WALs of {} failed, retry...", serverName, e);
       return false;
     }
   }
 
-  private Procedure[] createSplittingWalProcedures(MasterProcedureEnv env, boolean splitMeta)
+  private Procedure[] createSplittingWalProcedures(MasterProcedureEnv env,
+    SplitWALManager.SplitType splitType)
       throws IOException {
-    LOG.info("Splitting WALs {}, isMeta: {}", this, splitMeta);
+    LOG.info("Splitting WALs {}, SplitType: {}", this, splitType);
     SplitWALManager splitWALManager = env.getMasterServices().getSplitWALManager();
-    List<Procedure> procedures = splitWALManager.splitWALs(serverName, splitMeta);
+    List<Procedure> procedures = splitWALManager.splitWALs(serverName, splitType);
     return procedures.toArray(new Procedure[procedures.size()]);
   }
 
   private boolean filterDefaultMetaRegions() {
     if (regionsOnCrashedServer == null) {
       return false;
     }
-    regionsOnCrashedServer.removeIf(this::isDefaultMetaRegion);
+    regionsOnCrashedServer.removeIf((x) -> isDefaultMetaRegion(x) || isDefaultRootRegion(x));
     return !regionsOnCrashedServer.isEmpty();
   }
 
+  private boolean isDefaultRootRegion(RegionInfo hri) {
+    return hri.isRootRegion() && RegionReplicaUtil.isDefaultReplica(hri);
+  }
+
   private boolean isDefaultMetaRegion(RegionInfo hri) {
     return hri.isMetaRegion() && RegionReplicaUtil.isDefaultReplica(hri);
   }
 
+  private void zkCoordinatedSplitRootLogs(MasterProcedureEnv env) throws IOException {
+    LOG.debug("Splitting root WALs {}", this);
+    MasterWalManager mwm = env.getMasterServices().getMasterWalManager();
+    AssignmentManager am = env.getMasterServices().getAssignmentManager();
+    am.getRegionStates().rootLogSplitting(serverName);
+    mwm.splitRootLog(serverName);
+    am.getRegionStates().rootLogSplit(serverName);
+    LOG.debug("Done splitting root WALs {}", this);
+  }

Review comment:
       thanks will update.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506775857



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
##########
@@ -2531,6 +2531,7 @@ public GetTableStateResponse setTableStateInMeta(RpcController controller,
    *
    * @return previous states of the regions
    */
+  //TODO francis support root here

Review comment:
       Yeah hbck related stuff I have not finished.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#issuecomment-710731328


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  5s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 19s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 36s |  master passed  |
   | +1 :green_heart: |  compile  |   3m  5s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 14s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   5m 28s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 30s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m  1s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m  1s |  the patch passed  |
   | -1 :x: |  shadedjars  |   4m 39s |  patch has 10 errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 15s |  hbase-balancer generated 2 new + 0 unchanged - 2 fixed = 2 total (was 2)  |
   | -0 :warning: |  javadoc  |   3m  0s |  root generated 2 new + 172 unchanged - 2 fixed = 174 total (was 174)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  16m  1s |  root in the patch failed.  |
   |  |   |  58m 14s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2445 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 213a12227b06 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e6c78f4037 |
   | Default Java | 2020-01-14 |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-jdk11-hadoop3-check/output/patch-shadedjars.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-jdk11-hadoop3-check/output/diff-javadoc-javadoc-hbase-balancer.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-jdk11-hadoop3-check/output/diff-javadoc-javadoc-root.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/testReport/ |
   | Max. process+thread count | 780 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-zookeeper hbase-balancer hbase-server hbase-backup . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506776132



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
##########
@@ -2986,6 +2988,7 @@ public GetMastersResponse getMasters(RpcController rpcController, GetMastersRequ
     return resp.build();
   }
 
+  //TODO francis this needs to be get RootLocation now?
   @Override
   public GetMetaRegionLocationsResponse getMetaRegionLocations(RpcController rpcController,

Review comment:
       Yeah I have. Looks like I need to having something like this for backward compatibility but I also need to have a getRootregionLocations() one. Should I just rename this and worry about re-adding the existing api for compatibility later? 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506770269



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonRootRegionLocator.java
##########
@@ -68,9 +70,9 @@
  * The asynchronous locator for regions other than meta.
  */
 @InterfaceAudience.Private
-class AsyncNonMetaRegionLocator {
+class AsyncNonRootRegionLocator {

Review comment:
       Yes. I'll update the javadoc.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506775087



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
##########
@@ -158,4 +174,14 @@ default int compareRows(ByteBuffer row, Cell cell) {
    *   Do not pollute with types other than BBKV if can be helped; the Comparator will slow.
    */
   Comparator getSimpleComparator();
+
+  static CellComparator getComparator(TableName tableName) {
+    if (tableName.equals(TableName.ROOT_TABLE_NAME)) {
+      return RootCellComparator.ROOT_COMPARATOR;
+    }
+    if (tableName.equals(TableName.META_TABLE_NAME)) {
+      return MetaCellComparator.META_COMPARATOR;
+    }
+    return CellComparatorImpl.COMPARATOR;

Review comment:
       No that's just artifact from the PoC iteration will clean it up.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#issuecomment-708338719


   Thanks for the review @saintstack . Let me go through it tomorrow. Will write up an overview as well as clean things up a bit. It does split meta see unit tests TestSplitMetaBasicOperations.java and TestSplitMetaAssignmentOperations.java. 


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506778311



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -430,11 +553,19 @@ private boolean isCarryingRegion(final ServerName serverName, final RegionInfo r
     return(node != null && serverName.equals(node.getRegionLocation()));
   }
 
-  private RegionInfo getMetaForRegion(final RegionInfo regionInfo) {
-    //if (regionInfo.isMetaRegion()) return regionInfo;
-    // TODO: handle multiple meta. if the region provided is not meta lookup
-    // which meta the region belongs to.
-    return RegionInfoBuilder.FIRST_META_REGIONINFO;
+  /**
+   * Check hbase:meta is up and ready for reading. For use during Master startup only.
+   * @return True if meta is UP and online and startup can progress. Otherwise, meta is not online
+   *   and we will hold here until operator intervention.
+   */
+  @VisibleForTesting
+  public boolean waitForMetaOnline() {

Review comment:
       This is only during master startup/initialization. We can do as a follow-up as that'd be a big change. Off-hand: As currently RegionStates is mainly used after it has read all the contents of meta. That would be a change in expectation if it would now be possible that RegionStates may not know about certain regions. Eg it would be possible that SCP will only recover regions that RegionStates knows about. And we would need a mechanism to cover for when other regions of meta are loaded down the road. 
   
   Alternatively we can just speedup/parallelize meta assignment until that is no longer good enough then consider what you mentioned? Note also that as part of startup/initialization. We would need to scan meta to bootstrap RegionStates. Internally we parallelized the scanning of meta during startup. I'll remember to add that in this patch  too.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#issuecomment-716471729


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   6m 30s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 20s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 59s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 32s |  master passed  |
   | +1 :green_heart: |  spotbugs  |  19m  4s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 48s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 36s |  root: The patch generated 110 new + 962 unchanged - 5 fixed = 1072 total (was 967)  |
   | -0 :warning: |  whitespace  |   0m  0s |  The patch has 1 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  hadoopcheck  |  18m 50s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   8m  6s |  the patch passed  |
   | -1 :x: |  spotbugs  |   1m 21s |  hbase-client generated 3 new + 0 unchanged - 0 fixed = 3 total (was 0)  |
   | -1 :x: |  spotbugs  |   2m 38s |  hbase-server generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -1 :x: |  spotbugs  |  11m 10s |  root generated 4 new + 0 unchanged - 0 fixed = 4 total (was 0)  |
   ||| _ Other Tests _ |
   | -1 :x: |  asflicense  |   1m 25s |  The patch generated 1 ASF License warnings.  |
   |  |   | 100m 10s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-client |
   |  |  Nullcheck of other at line 833 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.isOverlap(RegionInfo)  At RegionInfo.java:833 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.isOverlap(RegionInfo)  At RegionInfo.java:[line 832] |
   |  |  Nullcheck of rhs at line 113 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.lambda$static$0(RegionInfo, RegionInfo)  At RegionInfo.java:113 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.lambda$static$0(RegionInfo, RegionInfo)  At RegionInfo.java:[line 112] |
   |  |  Nullcheck of regionA at line 448 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.areAdjacent(RegionInfo, RegionInfo)  At RegionInfo.java:448 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.areAdjacent(RegionInfo, RegionInfo)  At RegionInfo.java:[line 447] |
   | FindBugs | module:hbase-server |
   |  |  Possible null pointer dereference of region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:[line 149] |
   | FindBugs | module:root |
   |  |  Nullcheck of other at line 833 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.isOverlap(RegionInfo)  At RegionInfo.java:833 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.isOverlap(RegionInfo)  At RegionInfo.java:[line 832] |
   |  |  Nullcheck of rhs at line 113 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.lambda$static$0(RegionInfo, RegionInfo)  At RegionInfo.java:113 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.lambda$static$0(RegionInfo, RegionInfo)  At RegionInfo.java:[line 112] |
   |  |  Nullcheck of regionA at line 448 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.areAdjacent(RegionInfo, RegionInfo)  At RegionInfo.java:448 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.areAdjacent(RegionInfo, RegionInfo)  At RegionInfo.java:[line 447] |
   |  |  Possible null pointer dereference of region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:[line 149] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2445 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle cc hbaseprotoc prototool |
   | uname | Linux 717d8fa39628 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 164cc5a3dc |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | whitespace | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-general-check/output/whitespace-eol.txt |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-general-check/output/new-spotbugs-hbase-client.html |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-general-check/output/new-spotbugs-hbase-server.html |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-general-check/output/new-spotbugs-root.html |
   | asflicense | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-general-check/output/patch-asflicense-problems.txt |
   | Max. process+thread count | 122 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-zookeeper hbase-balancer hbase-server hbase-backup . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506775008



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
##########
@@ -250,6 +260,7 @@
     DEFAULT_VALUES.put(PRIORITY, String.valueOf(DEFAULT_PRIORITY));
     DEFAULT_VALUES.keySet().stream()
             .map(s -> new Bytes(Bytes.toBytes(s))).forEach(RESERVED_KEYWORDS::add);
+    RESERVED_KEYWORDS.add(IS_ROOT_KEY);
     RESERVED_KEYWORDS.add(IS_META_KEY);

Review comment:
       Yeah...




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506775609



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
##########
@@ -124,7 +124,7 @@
 import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
-import org.apache.hadoop.hbase.master.procedure.InitMetaProcedure;
+import org.apache.hadoop.hbase.master.procedure.InitRootProcedure;

Review comment:
       Looks like it's still [here](https://github.com/apache/hbase/blob/master/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java)? Maybe you mean MetaBootstrap?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506774118



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
##########
@@ -2391,42 +2406,91 @@ public void run(Timeout timeout) throws Exception {
       return failedFuture(new IllegalArgumentException("Passed region name can't be null"));
     }
     try {
+      TableName parentTable;
       CompletableFuture<Optional<HRegionLocation>> future;
       if (RegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) {
         String encodedName = Bytes.toString(regionNameOrEncodedRegionName);
-        if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) {
-          // old format encodedName, should be meta region
+
+        //TODO francis do we really need to support encoded name for root?
+        boolean isRoot = false;
+        for (int i = 0; i< numRootReplicas; i++) {
+          RegionInfo info =
+            RegionReplicaUtil.getRegionInfoForReplica(RegionInfoBuilder.ROOT_REGIONINFO, i);
+          if (Bytes.equals(info.getRegionName(), regionNameOrEncodedRegionName)) {
+            isRoot = true;
+            break;
+          }
+        }
+        if (isRoot) {
           future = connection.registry.getMetaRegionLocations()
             .thenApply(locs -> Stream.of(locs.getRegionLocations())
               .filter(loc -> loc.getRegion().getEncodedName().equals(encodedName)).findFirst());
+          parentTable = null;
+        } else if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) {
+          future = ClientMetaTableAccessor.getRegionLocationWithEncodedName(rootTable,
+            regionNameOrEncodedRegionName);
+          parentTable = ROOT_TABLE_NAME;
         } else {
           future = ClientMetaTableAccessor.getRegionLocationWithEncodedName(metaTable,
             regionNameOrEncodedRegionName);
+          parentTable = META_TABLE_NAME;
         }
       } else {
         RegionInfo regionInfo =
           CatalogFamilyFormat.parseRegionInfoFromRegionName(regionNameOrEncodedRegionName);
-        if (regionInfo.isMetaRegion()) {
+        if (regionInfo.isRootRegion()) {
           future = connection.registry.getMetaRegionLocations()
             .thenApply(locs -> Stream.of(locs.getRegionLocations())
               .filter(loc -> loc.getRegion().getReplicaId() == regionInfo.getReplicaId())
               .findFirst());
+          parentTable = null;
+          //TODO francis it won't reach here once meta is split
+        } else if (regionInfo.isMetaRegion())   {
+          parentTable = ROOT_TABLE_NAME;
+          future =
+            ClientMetaTableAccessor.getRegionLocation(rootTable, regionNameOrEncodedRegionName);
         } else {
+          parentTable = META_TABLE_NAME;
           future =
             ClientMetaTableAccessor.getRegionLocation(metaTable, regionNameOrEncodedRegionName);
         }
       }
 
+      final TableName finalParentTable = parentTable;
       CompletableFuture<HRegionLocation> returnedFuture = new CompletableFuture<>();
       addListener(future, (location, err) -> {
         if (err != null) {
           returnedFuture.completeExceptionally(err);
           return;
         }
         if (!location.isPresent() || location.get().getRegion() == null) {
-          returnedFuture.completeExceptionally(
-            new UnknownRegionException("Invalid region name or encoded region name: " +
-              Bytes.toStringBinary(regionNameOrEncodedRegionName)));
+          if (META_TABLE_NAME.equals(finalParentTable)) {
+            if (LOG.isDebugEnabled()) {

Review comment:
       I was trying to avoid the Bytes.toString() binary execution. That's inconsequential?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506778434



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -511,6 +654,39 @@ public boolean isMetaLoaded() {
     return metaLoadEvent.isReady();
   }
 
+
+  /**
+   * @return True if region is online and scannable else false if an error or shutdown (Otherwise
+   *   we just block in here holding up all forward-progess).

Review comment:
       I can't take credit for this radical-ness. I just moved this api from HMaster to here. :-)




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506767909



##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
+    throws IOException {
+    byte[] row = regionName;
+    RegionInfo parsedInfo = null;
+    try {
+      parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
+      row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);
+    } catch (Exception parseEx) {
+      return null;
+      // Ignore. This is used with tableName passed as regionName.
+    }
+    Get get = new Get(row);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    Result r;
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(parsedInfo.getTable()))) {
+      r = t.get(get);
+    }
+    RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+    return locations == null ? null :
+      locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionInfo region information
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
+    throws IOException {
+    return CatalogFamilyFormat.getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
+      regionInfo, regionInfo.getReplicaId());
+  }
+
+  /**
+   * @return Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta.

Review comment:
       For both will rename.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506768383



##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
+    throws IOException {
+    byte[] row = regionName;
+    RegionInfo parsedInfo = null;
+    try {
+      parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
+      row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);
+    } catch (Exception parseEx) {
+      return null;
+      // Ignore. This is used with tableName passed as regionName.
+    }
+    Get get = new Get(row);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    Result r;
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(parsedInfo.getTable()))) {
+      r = t.get(get);
+    }
+    RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+    return locations == null ? null :
+      locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionInfo region information
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
+    throws IOException {
+    return CatalogFamilyFormat.getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
+      regionInfo, regionInfo.getReplicaId());
+  }
+
+  /**
+   * @return Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta.
+   */
+  public static Result getCatalogFamilyRow(Connection connection, RegionInfo ri)
+    throws IOException {
+    Get get = new Get(CatalogFamilyFormat.getMetaKeyForRegion(ri));
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(ri.getTable()))) {
+      return t.get(get);
+    }
+  }
+
+  public static TableName getCatalogTableForTable(TableName tableName) {
+    if (TableName.ROOT_TABLE_NAME.equals(tableName)) {
+      throw new IllegalStateException("Can't get catalog table for hbase:root table");
+    }
+    if (TableName.META_TABLE_NAME.equals(tableName)) {
+      return TableName.ROOT_TABLE_NAME;
+    }
+    return TableName.META_TABLE_NAME;
+  }
+
+  public static boolean isCatalogTable(TableName tableName) {
+    return tableName.equals(TableName.ROOT_TABLE_NAME) ||
+      tableName.equals(TableName.META_TABLE_NAME);
+  }
+
+  /**
+   * Gets the result in hbase:meta for the specified region.
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return result of the specified region
+   */
+  public static Result getRegionResult(Connection connection,
+      byte[] regionName) throws IOException {
+    Table catalogTable = null;
+    if (Bytes.equals(RegionInfoBuilder.ROOT_REGIONINFO.getRegionName(), regionName)) {
+      throw new IllegalStateException("This method cannot be used for hbase:root region");
+    }
+    catalogTable = getCatalogHTable(connection,
+      getCatalogTableForTable(
+        CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName).getTable()));
+    Get get = new Get(regionName);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = catalogTable) {
+      return t.get(get);
+    }
+  }
+
+  /**
+   * Scans META table for a row whose key contains the specified <B>regionEncodedName</B>, returning

Review comment:
       Will do.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506772362



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java
##########
@@ -126,7 +127,9 @@ 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,
+      //1 is region id of FIRST_META_REGION_INFO
+      !(this.tableName.equals(TableName.META_TABLE_NAME) && regionId == 1));

Review comment:
       Post split meta the region ids will be in the non-legacy format. This special case here is to handle the primordial meta region which is in legacy format. So we need to catch if the primordial meta region is being instantiated here and make sure it is created in the legacy format. 
   
   Note I think I need to handle primordial meta region replicas here as well. (Do I need to support that?)




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506775224



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
##########
@@ -300,8 +300,12 @@ public static CellComparator getCellComparator(TableName tableName) {
    * @return CellComparator to use going off the {@code tableName} passed.
    */
   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())?
-      MetaCellComparator.META_COMPARATOR: CellComparatorImpl.COMPARATOR;
+    if (Bytes.equals(tableName, TableName.ROOT_TABLE_NAME.toBytes())) {
+      return RootCellComparator.ROOT_COMPARATOR;
+    }
+    if (Bytes.equals(tableName, TableName.META_TABLE_NAME.toBytes())) {
+      return MetaCellComparator.META_COMPARATOR;
+    }
+    return CellComparatorImpl.COMPARATOR;

Review comment:
       yes will consolidate.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506776199



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
##########
@@ -257,46 +327,62 @@ private void cleanupSplitDir(MasterProcedureEnv env) {
         // old hbase:meta tenancy on this server; clean these up if any before trying to remove the
         // WAL directory of this server or we will fail. See archiveMetaLog comment for more details
         // on this condition.
-        env.getMasterServices().getMasterWalManager().archiveMetaLog(this.serverName);
+        env.getMasterServices().getMasterWalManager().archiveCatalogLog(this.serverName, false);
       }
       splitWALManager.deleteWALDir(serverName);
     } catch (IOException e) {
       LOG.warn("Remove WAL directory for {} failed, ignore...{}", serverName, e.getMessage());
     }
   }
 
-  private boolean isSplittingDone(MasterProcedureEnv env, boolean splitMeta) {
+  private boolean isSplittingDone(MasterProcedureEnv env, SplitWALManager.SplitType splitType) {
     SplitWALManager splitWALManager = env.getMasterServices().getSplitWALManager();
     try {
-      int wals = splitWALManager.getWALsToSplit(serverName, splitMeta).size();
-      LOG.debug("Check if {} WAL splitting is done? wals={}, meta={}", serverName, wals, splitMeta);
+      int wals = splitWALManager.getWALsToSplit(serverName, splitType).size();
+      LOG.debug("Check if {} WAL splitting is done? wals={}, SplitType={}",
+        serverName, wals, splitType);
       return wals == 0;
     } catch (IOException e) {
       LOG.warn("Get WALs of {} failed, retry...", serverName, e);
       return false;
     }
   }
 
-  private Procedure[] createSplittingWalProcedures(MasterProcedureEnv env, boolean splitMeta)
+  private Procedure[] createSplittingWalProcedures(MasterProcedureEnv env,
+    SplitWALManager.SplitType splitType)
       throws IOException {
-    LOG.info("Splitting WALs {}, isMeta: {}", this, splitMeta);
+    LOG.info("Splitting WALs {}, SplitType: {}", this, splitType);
     SplitWALManager splitWALManager = env.getMasterServices().getSplitWALManager();
-    List<Procedure> procedures = splitWALManager.splitWALs(serverName, splitMeta);
+    List<Procedure> procedures = splitWALManager.splitWALs(serverName, splitType);
     return procedures.toArray(new Procedure[procedures.size()]);
   }
 
   private boolean filterDefaultMetaRegions() {
     if (regionsOnCrashedServer == null) {
       return false;
     }
-    regionsOnCrashedServer.removeIf(this::isDefaultMetaRegion);
+    regionsOnCrashedServer.removeIf((x) -> isDefaultMetaRegion(x) || isDefaultRootRegion(x));
     return !regionsOnCrashedServer.isEmpty();
   }
 
+  private boolean isDefaultRootRegion(RegionInfo hri) {
+    return hri.isRootRegion() && RegionReplicaUtil.isDefaultReplica(hri);
+  }
+
   private boolean isDefaultMetaRegion(RegionInfo hri) {
     return hri.isMetaRegion() && RegionReplicaUtil.isDefaultReplica(hri);
   }
 
+  private void zkCoordinatedSplitRootLogs(MasterProcedureEnv env) throws IOException {
+    LOG.debug("Splitting root WALs {}", this);
+    MasterWalManager mwm = env.getMasterServices().getMasterWalManager();
+    AssignmentManager am = env.getMasterServices().getAssignmentManager();
+    am.getRegionStates().rootLogSplitting(serverName);
+    mwm.splitRootLog(serverName);
+    am.getRegionStates().rootLogSplit(serverName);
+    LOG.debug("Done splitting root WALs {}", this);
+  }

Review comment:
       will make sure to update things then. 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506770075



##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java
##########
@@ -132,7 +132,7 @@ public static void updateMetaWithFavoredNodesInfo(
         puts.add(put);
       }
     }
-    MetaTableAccessor.putsToMetaTable(connection, puts);
+    MetaTableAccessor.putsToCatalogTable(connection, TableName.META_TABLE_NAME, puts);

Review comment:
       Yes in places that don't include any real changes. This one is a mistake tho. It should've stuck with putsToMetaTable().




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#issuecomment-710722619


   @saintstack , I did a rebase and force pushed the updateated branch. I'll address your comments. 


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506775438



##########
File path: hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
##########
@@ -205,6 +243,37 @@ public void testMetaComparisons2() {
             Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",,1"), now))) > 0);
   }
 
+  @Test
+  public void testRootComparisons2() {

Review comment:
       Yeah I guess we should then. Let me check the legacy 0.94 unit tests. Would that mean there'd be millions for meta comparator too? 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r503607888



##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.

Review comment:
       Comment says meta. Is it any catalog table?

##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {

Review comment:
       ClientMetaTableAccessor needs rename?

##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
+    throws IOException {
+    byte[] row = regionName;
+    RegionInfo parsedInfo = null;
+    try {
+      parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
+      row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);
+    } catch (Exception parseEx) {
+      return null;
+      // Ignore. This is used with tableName passed as regionName.
+    }
+    Get get = new Get(row);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    Result r;
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(parsedInfo.getTable()))) {
+      r = t.get(get);
+    }
+    RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+    return locations == null ? null :
+      locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionInfo region information
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
+    throws IOException {
+    return CatalogFamilyFormat.getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
+      regionInfo, regionInfo.getReplicaId());
+  }
+
+  /**
+   * @return Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta.

Review comment:
       For hbase:meta only?

##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
+    throws IOException {
+    byte[] row = regionName;
+    RegionInfo parsedInfo = null;
+    try {
+      parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
+      row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);
+    } catch (Exception parseEx) {
+      return null;
+      // Ignore. This is used with tableName passed as regionName.

Review comment:
       Does this happen? Should the javadoc allow this as a case? Is it normal that we get regioninfo or tablename?

##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
+    throws IOException {
+    byte[] row = regionName;
+    RegionInfo parsedInfo = null;
+    try {
+      parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
+      row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);
+    } catch (Exception parseEx) {
+      return null;
+      // Ignore. This is used with tableName passed as regionName.
+    }
+    Get get = new Get(row);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    Result r;
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(parsedInfo.getTable()))) {
+      r = t.get(get);
+    }
+    RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+    return locations == null ? null :
+      locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionInfo region information
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
+    throws IOException {
+    return CatalogFamilyFormat.getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
+      regionInfo, regionInfo.getReplicaId());
+  }
+
+  /**
+   * @return Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta.
+   */
+  public static Result getCatalogFamilyRow(Connection connection, RegionInfo ri)
+    throws IOException {
+    Get get = new Get(CatalogFamilyFormat.getMetaKeyForRegion(ri));
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(ri.getTable()))) {
+      return t.get(get);
+    }
+  }
+
+  public static TableName getCatalogTableForTable(TableName tableName) {
+    if (TableName.ROOT_TABLE_NAME.equals(tableName)) {
+      throw new IllegalStateException("Can't get catalog table for hbase:root table");
+    }
+    if (TableName.META_TABLE_NAME.equals(tableName)) {
+      return TableName.ROOT_TABLE_NAME;
+    }
+    return TableName.META_TABLE_NAME;
+  }
+
+  public static boolean isCatalogTable(TableName tableName) {
+    return tableName.equals(TableName.ROOT_TABLE_NAME) ||
+      tableName.equals(TableName.META_TABLE_NAME);
+  }
+
+  /**
+   * Gets the result in hbase:meta for the specified region.
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return result of the specified region
+   */
+  public static Result getRegionResult(Connection connection,
+      byte[] regionName) throws IOException {
+    Table catalogTable = null;
+    if (Bytes.equals(RegionInfoBuilder.ROOT_REGIONINFO.getRegionName(), regionName)) {
+      throw new IllegalStateException("This method cannot be used for hbase:root region");
+    }
+    catalogTable = getCatalogHTable(connection,
+      getCatalogTableForTable(
+        CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName).getTable()));
+    Get get = new Get(regionName);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = catalogTable) {
+      return t.get(get);
+    }
+  }
+
+  /**
+   * Scans META table for a row whose key contains the specified <B>regionEncodedName</B>, returning

Review comment:
       Looks like it does both if nothing found in hbase:meta. Worth mention in javadoc.

##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
+    throws IOException {
+    byte[] row = regionName;
+    RegionInfo parsedInfo = null;
+    try {
+      parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
+      row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);
+    } catch (Exception parseEx) {
+      return null;
+      // Ignore. This is used with tableName passed as regionName.
+    }
+    Get get = new Get(row);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    Result r;
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(parsedInfo.getTable()))) {
+      r = t.get(get);
+    }
+    RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+    return locations == null ? null :
+      locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionInfo region information
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
+    throws IOException {
+    return CatalogFamilyFormat.getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
+      regionInfo, regionInfo.getReplicaId());
+  }
+
+  /**
+   * @return Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta.
+   */
+  public static Result getCatalogFamilyRow(Connection connection, RegionInfo ri)
+    throws IOException {
+    Get get = new Get(CatalogFamilyFormat.getMetaKeyForRegion(ri));
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(ri.getTable()))) {
+      return t.get(get);
+    }
+  }
+
+  public static TableName getCatalogTableForTable(TableName tableName) {
+    if (TableName.ROOT_TABLE_NAME.equals(tableName)) {
+      throw new IllegalStateException("Can't get catalog table for hbase:root table");
+    }
+    if (TableName.META_TABLE_NAME.equals(tableName)) {
+      return TableName.ROOT_TABLE_NAME;
+    }
+    return TableName.META_TABLE_NAME;
+  }
+
+  public static boolean isCatalogTable(TableName tableName) {
+    return tableName.equals(TableName.ROOT_TABLE_NAME) ||
+      tableName.equals(TableName.META_TABLE_NAME);
+  }
+
+  /**
+   * Gets the result in hbase:meta for the specified region.
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return result of the specified region
+   */
+  public static Result getRegionResult(Connection connection,
+      byte[] regionName) throws IOException {
+    Table catalogTable = null;
+    if (Bytes.equals(RegionInfoBuilder.ROOT_REGIONINFO.getRegionName(), regionName)) {
+      throw new IllegalStateException("This method cannot be used for hbase:root region");
+    }
+    catalogTable = getCatalogHTable(connection,
+      getCatalogTableForTable(
+        CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName).getTable()));
+    Get get = new Get(regionName);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = catalogTable) {
+      return t.get(get);
+    }
+  }
+
+  /**
+   * Scans META table for a row whose key contains the specified <B>regionEncodedName</B>, returning
+   * a single related <code>Result</code> instance if any row is found, null otherwise.
+   * @param connection the connection to query META table.
+   * @param regionEncodedName the region encoded name to look for at META.
+   * @return <code>Result</code> instance with the row related info in META, null otherwise.
+   * @throws IOException if any errors occur while querying META.
+   */
+  public static Result scanByRegionEncodedName(Connection connection, String regionEncodedName)
+    throws IOException {
+    RowFilter rowFilter =
+      new RowFilter(CompareOperator.EQUAL, new SubstringComparator(regionEncodedName));
+    Scan scan = getCatalogScan(connection, 1);
+    scan.setFilter(rowFilter);
+    Result res = null;
+    try (ResultScanner resultScanner =
+      getCatalogHTable(connection, TableName.META_TABLE_NAME).getScanner(scan)) {
+      res = resultScanner.next();
+    }
+    if (res == null) {
+      scan = getCatalogScan(connection, 1);
+      scan.setFilter(rowFilter);
+      try (ResultScanner resultScanner =
+        getCatalogHTable(connection, TableName.ROOT_TABLE_NAME).getScanner(scan)) {
+        res = resultScanner.next();
+      }
+    }
+    return res;
+  }
+
+  /**
+   * Lists all of the regions currently in META.
+   * @param connection to connect with
+   * @param excludeOfflinedSplitParents False if we are to include offlined/splitparents regions,
+   *          true and we'll leave out offlined regions from returned list
+   * @return List of all user-space regions.
+   */
+  @VisibleForTesting
+  public static List<RegionInfo> getAllRegions(Connection connection,
+    boolean excludeOfflinedSplitParents) throws IOException {
+    List<Pair<RegionInfo, ServerName>> result;
+
+    result = getTableRegionsAndLocations(connection, null, excludeOfflinedSplitParents);
+
+    return getListOfRegionInfos(result);
+
+  }
+
+  /**
+   * Gets all of the regions of the specified table. Do not use this method to get meta table
+   * regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @return Ordered list of {@link RegionInfo}.
+   */
+  public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName)
+  throws IOException {
+    return getTableRegions(connection, tableName, false);
+  }
+
+  /**
+   * Gets all of the regions of the specified table. Do not use this method to get meta table
+   * regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @param excludeOfflinedSplitParents If true, do not include offlined split parents in the
+   *          return.
+   * @return Ordered list of {@link RegionInfo}.
+   */
+  public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName,
+    final boolean excludeOfflinedSplitParents) throws IOException {
+    List<Pair<RegionInfo, ServerName>> result =
+      getTableRegionsAndLocations(connection, tableName, excludeOfflinedSplitParents);
+    return getListOfRegionInfos(result);
+  }
+
+  private static List<RegionInfo>
+    getListOfRegionInfos(final List<Pair<RegionInfo, ServerName>> pairs) {
+    if (pairs == null || pairs.isEmpty()) {
+      return Collections.emptyList();
+    }
+    List<RegionInfo> result = new ArrayList<>(pairs.size());
+    for (Pair<RegionInfo, ServerName> pair : pairs) {
+      result.add(pair.getFirst());
+    }
+    return result;
+  }
+
+  /**
+   * This method creates a Scan object that will only scan catalog rows that belong to the specified
+   * table. It doesn't specify any columns. This is a better alternative to just using a start row
+   * and scan until it hits a new table since that requires parsing the HRI to get the table name.
+   * @param tableName bytes of table's name
+   * @return configured Scan object
+   * @deprecated This is internal so please remove it when we get a chance.
+   */
+  @Deprecated
+  public static Scan getScanForTableName(Connection connection, TableName tableName) {
+    // Start key is just the table name with delimiters
+    byte[] startKey = ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION);
+    // Stop key appends the smallest possible char to the table name
+    byte[] stopKey = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);
+
+    Scan scan = getCatalogScan(connection, -1);
+    scan.withStartRow(startKey);
+    scan.withStopRow(stopKey);
+    return scan;
+  }
+
+  private static Scan getCatalogScan(Connection connection, int rowUpperLimit) {
+    Scan scan = new Scan();
+    int scannerCaching = connection.getConfiguration().getInt(HConstants.HBASE_META_SCANNER_CACHING,
+      HConstants.DEFAULT_HBASE_META_SCANNER_CACHING);
+    if (connection.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS,
+      HConstants.DEFAULT_USE_META_REPLICAS)) {
+      scan.setConsistency(Consistency.TIMELINE);
+    }
+    if (rowUpperLimit > 0) {
+      scan.setLimit(rowUpperLimit);
+      scan.setReadType(Scan.ReadType.PREAD);
+    }
+    scan.setCaching(scannerCaching);
+    return scan;
+  }
+
+  /**
+   * Do not use this method to get meta table regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @return Return list of regioninfos and server.
+   */
+  public static List<Pair<RegionInfo, ServerName>>
+    getTableRegionsAndLocations(Connection connection, TableName tableName) throws IOException {
+    return getTableRegionsAndLocations(connection, tableName, true);
+  }
+
+  /**
+   * Do not use this method to get meta table regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table to work with, can be null for getting all regions
+   * @param excludeOfflinedSplitParents don't return split parents
+   * @return Return list of regioninfos and server addresses.
+   */
+  // What happens here when 1M regions in hbase:meta? This won't scale?
+  public static List<Pair<RegionInfo, ServerName>> getTableRegionsAndLocations(
+    Connection connection, @Nullable final TableName tableName,
+    final boolean excludeOfflinedSplitParents) throws IOException {
+    if (tableName != null && tableName.equals(TableName.ROOT_TABLE_NAME)) {
+      throw new IOException(
+        "This method can't be used to locate root regions; use RootTableLocator instead");
+    }
+    // Make a version of CollectingVisitor that collects RegionInfo and ServerAddress
+    ClientMetaTableAccessor.CollectRegionLocationsVisitor visitor =
+      new ClientMetaTableAccessor.CollectRegionLocationsVisitor(excludeOfflinedSplitParents);
+    byte[] startRow = ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION);
+    byte[] stopRow = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);
+    TableName parentTable = TableName.META_TABLE_NAME;
+    if (TableName.META_TABLE_NAME.equals(tableName)) {
+      parentTable = TableName.ROOT_TABLE_NAME;
+      startRow = null;
+      stopRow = null;
+    }
+
+    scanCatalog(connection,
+      parentTable,
+      startRow,
+      stopRow,
+      QueryType.REGION,
+      Integer.MAX_VALUE,
+      visitor);
+    return visitor.getResults();
+  }
+
+  public static void fullScanMetaAndPrint(Connection connection) throws IOException {

Review comment:
       Just meta table?

##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);

Review comment:
       Is it scanMeta or scanCatalog?

##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.

Review comment:
       ditto on the comment

##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>

Review comment:
       ditto

##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.

Review comment:
       ditto

##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region

Review comment:
       Or null if none found?

##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
+    throws IOException {
+    byte[] row = regionName;
+    RegionInfo parsedInfo = null;
+    try {
+      parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
+      row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);
+    } catch (Exception parseEx) {
+      return null;
+      // Ignore. This is used with tableName passed as regionName.
+    }
+    Get get = new Get(row);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    Result r;
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(parsedInfo.getTable()))) {
+      r = t.get(get);
+    }
+    RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+    return locations == null ? null :
+      locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionInfo region information
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
+    throws IOException {
+    return CatalogFamilyFormat.getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
+      regionInfo, regionInfo.getReplicaId());
+  }
+
+  /**
+   * @return Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta.
+   */
+  public static Result getCatalogFamilyRow(Connection connection, RegionInfo ri)
+    throws IOException {
+    Get get = new Get(CatalogFamilyFormat.getMetaKeyForRegion(ri));
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(ri.getTable()))) {
+      return t.get(get);
+    }
+  }
+
+  public static TableName getCatalogTableForTable(TableName tableName) {
+    if (TableName.ROOT_TABLE_NAME.equals(tableName)) {
+      throw new IllegalStateException("Can't get catalog table for hbase:root table");
+    }
+    if (TableName.META_TABLE_NAME.equals(tableName)) {
+      return TableName.ROOT_TABLE_NAME;
+    }
+    return TableName.META_TABLE_NAME;
+  }
+
+  public static boolean isCatalogTable(TableName tableName) {
+    return tableName.equals(TableName.ROOT_TABLE_NAME) ||
+      tableName.equals(TableName.META_TABLE_NAME);
+  }
+
+  /**
+   * Gets the result in hbase:meta for the specified region.
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return result of the specified region
+   */
+  public static Result getRegionResult(Connection connection,
+      byte[] regionName) throws IOException {
+    Table catalogTable = null;
+    if (Bytes.equals(RegionInfoBuilder.ROOT_REGIONINFO.getRegionName(), regionName)) {
+      throw new IllegalStateException("This method cannot be used for hbase:root region");
+    }
+    catalogTable = getCatalogHTable(connection,

Review comment:
       We should close this table when done as per the method's javadoc ... try-with-resources?

##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
+    throws IOException {
+    byte[] row = regionName;
+    RegionInfo parsedInfo = null;
+    try {
+      parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
+      row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);
+    } catch (Exception parseEx) {
+      return null;
+      // Ignore. This is used with tableName passed as regionName.
+    }
+    Get get = new Get(row);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    Result r;
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(parsedInfo.getTable()))) {
+      r = t.get(get);
+    }
+    RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+    return locations == null ? null :
+      locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionInfo region information
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
+    throws IOException {
+    return CatalogFamilyFormat.getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
+      regionInfo, regionInfo.getReplicaId());
+  }
+
+  /**
+   * @return Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta.
+   */
+  public static Result getCatalogFamilyRow(Connection connection, RegionInfo ri)
+    throws IOException {
+    Get get = new Get(CatalogFamilyFormat.getMetaKeyForRegion(ri));
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(ri.getTable()))) {
+      return t.get(get);
+    }
+  }
+
+  public static TableName getCatalogTableForTable(TableName tableName) {
+    if (TableName.ROOT_TABLE_NAME.equals(tableName)) {
+      throw new IllegalStateException("Can't get catalog table for hbase:root table");
+    }
+    if (TableName.META_TABLE_NAME.equals(tableName)) {
+      return TableName.ROOT_TABLE_NAME;
+    }
+    return TableName.META_TABLE_NAME;
+  }
+
+  public static boolean isCatalogTable(TableName tableName) {
+    return tableName.equals(TableName.ROOT_TABLE_NAME) ||
+      tableName.equals(TableName.META_TABLE_NAME);
+  }
+
+  /**
+   * Gets the result in hbase:meta for the specified region.
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return result of the specified region
+   */
+  public static Result getRegionResult(Connection connection,
+      byte[] regionName) throws IOException {
+    Table catalogTable = null;
+    if (Bytes.equals(RegionInfoBuilder.ROOT_REGIONINFO.getRegionName(), regionName)) {
+      throw new IllegalStateException("This method cannot be used for hbase:root region");
+    }
+    catalogTable = getCatalogHTable(connection,
+      getCatalogTableForTable(
+        CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName).getTable()));
+    Get get = new Get(regionName);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = catalogTable) {
+      return t.get(get);
+    }
+  }
+
+  /**
+   * Scans META table for a row whose key contains the specified <B>regionEncodedName</B>, returning
+   * a single related <code>Result</code> instance if any row is found, null otherwise.
+   * @param connection the connection to query META table.
+   * @param regionEncodedName the region encoded name to look for at META.
+   * @return <code>Result</code> instance with the row related info in META, null otherwise.
+   * @throws IOException if any errors occur while querying META.
+   */
+  public static Result scanByRegionEncodedName(Connection connection, String regionEncodedName)
+    throws IOException {
+    RowFilter rowFilter =
+      new RowFilter(CompareOperator.EQUAL, new SubstringComparator(regionEncodedName));
+    Scan scan = getCatalogScan(connection, 1);
+    scan.setFilter(rowFilter);
+    Result res = null;
+    try (ResultScanner resultScanner =
+      getCatalogHTable(connection, TableName.META_TABLE_NAME).getScanner(scan)) {
+      res = resultScanner.next();
+    }
+    if (res == null) {
+      scan = getCatalogScan(connection, 1);
+      scan.setFilter(rowFilter);
+      try (ResultScanner resultScanner =
+        getCatalogHTable(connection, TableName.ROOT_TABLE_NAME).getScanner(scan)) {
+        res = resultScanner.next();
+      }
+    }
+    return res;
+  }
+
+  /**
+   * Lists all of the regions currently in META.
+   * @param connection to connect with
+   * @param excludeOfflinedSplitParents False if we are to include offlined/splitparents regions,
+   *          true and we'll leave out offlined regions from returned list
+   * @return List of all user-space regions.
+   */
+  @VisibleForTesting
+  public static List<RegionInfo> getAllRegions(Connection connection,
+    boolean excludeOfflinedSplitParents) throws IOException {
+    List<Pair<RegionInfo, ServerName>> result;
+
+    result = getTableRegionsAndLocations(connection, null, excludeOfflinedSplitParents);
+
+    return getListOfRegionInfos(result);
+
+  }
+
+  /**
+   * Gets all of the regions of the specified table. Do not use this method to get meta table
+   * regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @return Ordered list of {@link RegionInfo}.
+   */
+  public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName)
+  throws IOException {
+    return getTableRegions(connection, tableName, false);
+  }
+
+  /**
+   * Gets all of the regions of the specified table. Do not use this method to get meta table
+   * regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @param excludeOfflinedSplitParents If true, do not include offlined split parents in the
+   *          return.
+   * @return Ordered list of {@link RegionInfo}.
+   */
+  public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName,
+    final boolean excludeOfflinedSplitParents) throws IOException {
+    List<Pair<RegionInfo, ServerName>> result =
+      getTableRegionsAndLocations(connection, tableName, excludeOfflinedSplitParents);
+    return getListOfRegionInfos(result);
+  }
+
+  private static List<RegionInfo>
+    getListOfRegionInfos(final List<Pair<RegionInfo, ServerName>> pairs) {
+    if (pairs == null || pairs.isEmpty()) {
+      return Collections.emptyList();
+    }
+    List<RegionInfo> result = new ArrayList<>(pairs.size());
+    for (Pair<RegionInfo, ServerName> pair : pairs) {
+      result.add(pair.getFirst());
+    }
+    return result;
+  }
+
+  /**
+   * This method creates a Scan object that will only scan catalog rows that belong to the specified
+   * table. It doesn't specify any columns. This is a better alternative to just using a start row

Review comment:
       I don't get the second sentence.

##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
+    throws IOException {
+    byte[] row = regionName;
+    RegionInfo parsedInfo = null;
+    try {
+      parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
+      row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);
+    } catch (Exception parseEx) {
+      return null;
+      // Ignore. This is used with tableName passed as regionName.
+    }
+    Get get = new Get(row);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    Result r;
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(parsedInfo.getTable()))) {
+      r = t.get(get);
+    }
+    RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+    return locations == null ? null :
+      locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionInfo region information
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
+    throws IOException {
+    return CatalogFamilyFormat.getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
+      regionInfo, regionInfo.getReplicaId());
+  }
+
+  /**
+   * @return Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta.
+   */
+  public static Result getCatalogFamilyRow(Connection connection, RegionInfo ri)
+    throws IOException {
+    Get get = new Get(CatalogFamilyFormat.getMetaKeyForRegion(ri));
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(ri.getTable()))) {
+      return t.get(get);
+    }
+  }
+
+  public static TableName getCatalogTableForTable(TableName tableName) {
+    if (TableName.ROOT_TABLE_NAME.equals(tableName)) {
+      throw new IllegalStateException("Can't get catalog table for hbase:root table");
+    }
+    if (TableName.META_TABLE_NAME.equals(tableName)) {
+      return TableName.ROOT_TABLE_NAME;
+    }
+    return TableName.META_TABLE_NAME;
+  }
+
+  public static boolean isCatalogTable(TableName tableName) {
+    return tableName.equals(TableName.ROOT_TABLE_NAME) ||
+      tableName.equals(TableName.META_TABLE_NAME);
+  }
+
+  /**
+   * Gets the result in hbase:meta for the specified region.
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return result of the specified region
+   */
+  public static Result getRegionResult(Connection connection,
+      byte[] regionName) throws IOException {
+    Table catalogTable = null;
+    if (Bytes.equals(RegionInfoBuilder.ROOT_REGIONINFO.getRegionName(), regionName)) {
+      throw new IllegalStateException("This method cannot be used for hbase:root region");
+    }
+    catalogTable = getCatalogHTable(connection,
+      getCatalogTableForTable(
+        CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName).getTable()));
+    Get get = new Get(regionName);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = catalogTable) {
+      return t.get(get);
+    }
+  }
+
+  /**
+   * Scans META table for a row whose key contains the specified <B>regionEncodedName</B>, returning

Review comment:
       Only hbase:meta?

##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/favored/FavoredNodeAssignmentHelper.java
##########
@@ -132,7 +132,7 @@ public static void updateMetaWithFavoredNodesInfo(
         puts.add(put);
       }
     }
-    MetaTableAccessor.putsToMetaTable(connection, puts);
+    MetaTableAccessor.putsToCatalogTable(connection, TableName.META_TABLE_NAME, puts);

Review comment:
       So, you keep referring to MetaTableAccessor so as to keep the patch small?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/ClientMetaTableAccessor.java
##########
@@ -193,15 +193,15 @@ private ClientMetaTableAccessor() {
    * @param tableName table we're looking for, can be null for getting all regions
    * @param excludeOfflinedSplitParents don't return split parents
    * @return the list of regioninfos and server. The return value will be wrapped by a

Review comment:
       This class needs a rename?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllRootRegionsOnlineException.java
##########
@@ -25,19 +25,19 @@
  * Thrown when an operation requires the root and all meta regions to be online
  */
 @InterfaceAudience.Public
-public class NotAllMetaRegionsOnlineException extends DoNotRetryIOException {
+public class NotAllRootRegionsOnlineException extends DoNotRetryIOException {

Review comment:
       Should this be named ROOTNotOnlineException  or RootRegionNotOnlineException(singular)

##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
+    throws IOException {
+    byte[] row = regionName;
+    RegionInfo parsedInfo = null;
+    try {
+      parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
+      row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);
+    } catch (Exception parseEx) {
+      return null;
+      // Ignore. This is used with tableName passed as regionName.
+    }
+    Get get = new Get(row);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    Result r;
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(parsedInfo.getTable()))) {
+      r = t.get(get);
+    }
+    RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+    return locations == null ? null :
+      locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionInfo region information
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
+    throws IOException {
+    return CatalogFamilyFormat.getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
+      regionInfo, regionInfo.getReplicaId());
+  }
+
+  /**
+   * @return Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta.
+   */
+  public static Result getCatalogFamilyRow(Connection connection, RegionInfo ri)
+    throws IOException {
+    Get get = new Get(CatalogFamilyFormat.getMetaKeyForRegion(ri));
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(ri.getTable()))) {
+      return t.get(get);
+    }
+  }
+
+  public static TableName getCatalogTableForTable(TableName tableName) {
+    if (TableName.ROOT_TABLE_NAME.equals(tableName)) {
+      throw new IllegalStateException("Can't get catalog table for hbase:root table");
+    }
+    if (TableName.META_TABLE_NAME.equals(tableName)) {
+      return TableName.ROOT_TABLE_NAME;
+    }
+    return TableName.META_TABLE_NAME;
+  }
+
+  public static boolean isCatalogTable(TableName tableName) {
+    return tableName.equals(TableName.ROOT_TABLE_NAME) ||
+      tableName.equals(TableName.META_TABLE_NAME);
+  }
+
+  /**
+   * Gets the result in hbase:meta for the specified region.
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return result of the specified region
+   */
+  public static Result getRegionResult(Connection connection,
+      byte[] regionName) throws IOException {
+    Table catalogTable = null;
+    if (Bytes.equals(RegionInfoBuilder.ROOT_REGIONINFO.getRegionName(), regionName)) {
+      throw new IllegalStateException("This method cannot be used for hbase:root region");
+    }
+    catalogTable = getCatalogHTable(connection,
+      getCatalogTableForTable(
+        CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName).getTable()));
+    Get get = new Get(regionName);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = catalogTable) {
+      return t.get(get);
+    }
+  }
+
+  /**
+   * Scans META table for a row whose key contains the specified <B>regionEncodedName</B>, returning
+   * a single related <code>Result</code> instance if any row is found, null otherwise.
+   * @param connection the connection to query META table.
+   * @param regionEncodedName the region encoded name to look for at META.
+   * @return <code>Result</code> instance with the row related info in META, null otherwise.
+   * @throws IOException if any errors occur while querying META.
+   */
+  public static Result scanByRegionEncodedName(Connection connection, String regionEncodedName)
+    throws IOException {
+    RowFilter rowFilter =
+      new RowFilter(CompareOperator.EQUAL, new SubstringComparator(regionEncodedName));
+    Scan scan = getCatalogScan(connection, 1);
+    scan.setFilter(rowFilter);
+    Result res = null;
+    try (ResultScanner resultScanner =
+      getCatalogHTable(connection, TableName.META_TABLE_NAME).getScanner(scan)) {
+      res = resultScanner.next();
+    }
+    if (res == null) {
+      scan = getCatalogScan(connection, 1);
+      scan.setFilter(rowFilter);
+      try (ResultScanner resultScanner =
+        getCatalogHTable(connection, TableName.ROOT_TABLE_NAME).getScanner(scan)) {
+        res = resultScanner.next();
+      }
+    }
+    return res;
+  }
+
+  /**
+   * Lists all of the regions currently in META.
+   * @param connection to connect with
+   * @param excludeOfflinedSplitParents False if we are to include offlined/splitparents regions,
+   *          true and we'll leave out offlined regions from returned list
+   * @return List of all user-space regions.
+   */
+  @VisibleForTesting
+  public static List<RegionInfo> getAllRegions(Connection connection,
+    boolean excludeOfflinedSplitParents) throws IOException {
+    List<Pair<RegionInfo, ServerName>> result;
+
+    result = getTableRegionsAndLocations(connection, null, excludeOfflinedSplitParents);
+
+    return getListOfRegionInfos(result);
+
+  }
+
+  /**
+   * Gets all of the regions of the specified table. Do not use this method to get meta table
+   * regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @return Ordered list of {@link RegionInfo}.
+   */
+  public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName)
+  throws IOException {
+    return getTableRegions(connection, tableName, false);
+  }
+
+  /**
+   * Gets all of the regions of the specified table. Do not use this method to get meta table
+   * regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @param excludeOfflinedSplitParents If true, do not include offlined split parents in the
+   *          return.
+   * @return Ordered list of {@link RegionInfo}.
+   */
+  public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName,
+    final boolean excludeOfflinedSplitParents) throws IOException {
+    List<Pair<RegionInfo, ServerName>> result =
+      getTableRegionsAndLocations(connection, tableName, excludeOfflinedSplitParents);
+    return getListOfRegionInfos(result);
+  }
+
+  private static List<RegionInfo>
+    getListOfRegionInfos(final List<Pair<RegionInfo, ServerName>> pairs) {
+    if (pairs == null || pairs.isEmpty()) {
+      return Collections.emptyList();
+    }
+    List<RegionInfo> result = new ArrayList<>(pairs.size());
+    for (Pair<RegionInfo, ServerName> pair : pairs) {
+      result.add(pair.getFirst());
+    }
+    return result;
+  }
+
+  /**
+   * This method creates a Scan object that will only scan catalog rows that belong to the specified
+   * table. It doesn't specify any columns. This is a better alternative to just using a start row
+   * and scan until it hits a new table since that requires parsing the HRI to get the table name.
+   * @param tableName bytes of table's name
+   * @return configured Scan object
+   * @deprecated This is internal so please remove it when we get a chance.
+   */
+  @Deprecated
+  public static Scan getScanForTableName(Connection connection, TableName tableName) {
+    // Start key is just the table name with delimiters
+    byte[] startKey = ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION);
+    // Stop key appends the smallest possible char to the table name
+    byte[] stopKey = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);
+
+    Scan scan = getCatalogScan(connection, -1);
+    scan.withStartRow(startKey);
+    scan.withStopRow(stopKey);
+    return scan;
+  }
+
+  private static Scan getCatalogScan(Connection connection, int rowUpperLimit) {
+    Scan scan = new Scan();
+    int scannerCaching = connection.getConfiguration().getInt(HConstants.HBASE_META_SCANNER_CACHING,
+      HConstants.DEFAULT_HBASE_META_SCANNER_CACHING);
+    if (connection.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS,
+      HConstants.DEFAULT_USE_META_REPLICAS)) {
+      scan.setConsistency(Consistency.TIMELINE);
+    }
+    if (rowUpperLimit > 0) {
+      scan.setLimit(rowUpperLimit);
+      scan.setReadType(Scan.ReadType.PREAD);
+    }
+    scan.setCaching(scannerCaching);
+    return scan;
+  }
+
+  /**
+   * Do not use this method to get meta table regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @return Return list of regioninfos and server.
+   */
+  public static List<Pair<RegionInfo, ServerName>>
+    getTableRegionsAndLocations(Connection connection, TableName tableName) throws IOException {
+    return getTableRegionsAndLocations(connection, tableName, true);
+  }
+
+  /**
+   * Do not use this method to get meta table regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table to work with, can be null for getting all regions
+   * @param excludeOfflinedSplitParents don't return split parents
+   * @return Return list of regioninfos and server addresses.
+   */
+  // What happens here when 1M regions in hbase:meta? This won't scale?
+  public static List<Pair<RegionInfo, ServerName>> getTableRegionsAndLocations(
+    Connection connection, @Nullable final TableName tableName,
+    final boolean excludeOfflinedSplitParents) throws IOException {
+    if (tableName != null && tableName.equals(TableName.ROOT_TABLE_NAME)) {
+      throw new IOException(
+        "This method can't be used to locate root regions; use RootTableLocator instead");
+    }
+    // Make a version of CollectingVisitor that collects RegionInfo and ServerAddress
+    ClientMetaTableAccessor.CollectRegionLocationsVisitor visitor =
+      new ClientMetaTableAccessor.CollectRegionLocationsVisitor(excludeOfflinedSplitParents);
+    byte[] startRow = ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION);
+    byte[] stopRow = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);
+    TableName parentTable = TableName.META_TABLE_NAME;
+    if (TableName.META_TABLE_NAME.equals(tableName)) {
+      parentTable = TableName.ROOT_TABLE_NAME;
+      startRow = null;
+      stopRow = null;
+    }
+
+    scanCatalog(connection,
+      parentTable,
+      startRow,
+      stopRow,
+      QueryType.REGION,
+      Integer.MAX_VALUE,
+      visitor);
+    return visitor.getResults();
+  }
+
+  public static void fullScanMetaAndPrint(Connection connection) throws IOException {
+    ClientMetaTableAccessor.Visitor v = r -> {
+      if (r == null || r.isEmpty()) {
+        return true;
+      }
+      LOG.info("fullScanMetaAndPrint.Current Meta Row: " + r);
+      TableState state = CatalogFamilyFormat.getTableState(r);
+      if (state != null) {
+        LOG.info("fullScanMetaAndPrint.Table State={}" + state);
+      } else {
+        RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+        if (locations == null) {
+          return true;
+        }
+        for (HRegionLocation loc : locations.getRegionLocations()) {
+          if (loc != null) {
+            LOG.info("fullScanMetaAndPrint.HRI Print={}", loc.getRegion());
+          }
+        }
+      }
+      return true;
+    };
+    scanMeta(connection, null, null, QueryType.ALL, v);
+  }
+
+  public static void scanMetaForTableRegions(Connection connection,
+    ClientMetaTableAccessor.Visitor visitor, TableName tableName) throws IOException {
+    scanCatalogForTableRegions(connection, QueryType.REGION, Integer.MAX_VALUE, visitor, tableName);
+  }
+
+  public static void scanCatalogForTableRegions(Connection connection, QueryType type, int maxRows,
+    final ClientMetaTableAccessor.Visitor visitor, TableName table) throws IOException {
+    scanCatalog(connection, getCatalogTableForTable(table),
+      ClientMetaTableAccessor.getTableStartRowForMeta(table, type),
+      ClientMetaTableAccessor.getTableStopRowForMeta(table, type),
+      type, maxRows, visitor);
+  }
+
+  public static void scanMeta(Connection connection, @Nullable final byte[] startRow,
+    @Nullable final byte[] stopRow, QueryType type, final ClientMetaTableAccessor.Visitor visitor)
+    throws IOException {
+    scanCatalog(connection, TableName.META_TABLE_NAME, startRow, stopRow, type, Integer.MAX_VALUE,
+        visitor);
+  }
+
+  /**
+   * Performs a scan of META table for given table starting from given row.
+   * @param connection connection we're using
+   * @param visitor visitor to call
+   * @param tableName table withing we scan
+   * @param row start scan from this row
+   * @param rowLimit max number of rows to return
+   */
+  public static void scanMeta(Connection connection, final ClientMetaTableAccessor.Visitor visitor,
+    final TableName tableName, final byte[] row, final int rowLimit) throws IOException {
+    byte[] startRow = null;
+    byte[] stopRow = null;
+    if (tableName != null) {
+      startRow = ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION);
+      if (row != null) {
+        RegionInfo closestRi = getClosestRegionInfo(connection, tableName, row);
+        startRow =
+          RegionInfo.createRegionName(tableName, closestRi.getStartKey(), HConstants.ZEROES, false);
+      }
+      stopRow = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);
+    }
+    scanCatalog(connection, TableName.META_TABLE_NAME, startRow, stopRow, QueryType.REGION,
+        rowLimit, visitor);
+  }
+
+  /**
+   * Performs a scan of META table.
+   * @param connection connection we're using
+   * @param startRow Where to start the scan. Pass null if want to begin scan at first row.
+   * @param stopRow Where to stop the scan. Pass null if want to scan all rows from the start one
+   * @param type scanned part of meta
+   * @param maxRows maximum rows to return
+   * @param visitor Visitor invoked against each row.
+   */
+  public static void scanCatalog(Connection connection, TableName catalogTable,
+      @Nullable final byte[] startRow, @Nullable final byte[] stopRow, QueryType type, int maxRows,
+      final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTable, startRow, stopRow, type, null, maxRows, visitor);
+  }
+
+  public static void scanCatalog(Connection connection, TableName catalogTableName,
+    @Nullable final byte[] startRow, @Nullable final byte[] stopRow,
+    QueryType type, @Nullable Filter filter, int maxRows,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE;
+    Scan scan = getCatalogScan(connection, rowUpperLimit);
+
+    for (byte[] family : type.getFamilies()) {
+      scan.addFamily(family);
+    }
+    if (startRow != null) {
+      scan.withStartRow(startRow);
+    }
+    if (stopRow != null) {
+      scan.withStopRow(stopRow);
+    }
+    if (filter != null) {
+      scan.setFilter(filter);
+    }
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Scanning META" + " starting at row=" + Bytes.toStringBinary(startRow) +
+        " stopping at row=" + Bytes.toStringBinary(stopRow) + " for max=" + rowUpperLimit +
+        " with caching=" + scan.getCaching());
+    }
+
+    int currentRow = 0;
+    try (Table catalogTable = getCatalogHTable(connection, catalogTableName)) {
+      try (ResultScanner scanner = catalogTable.getScanner(scan)) {
+        Result data;
+        while ((data = scanner.next()) != null) {
+          if (data.isEmpty()) {
+            continue;
+          }
+          // Break if visit returns false.
+          if (!visitor.visit(data)) {
+            break;
+          }
+          if (++currentRow >= rowUpperLimit) {
+            break;
+          }
+        }
+      }
+    }
+    if (visitor instanceof Closeable) {
+      try {
+        ((Closeable) visitor).close();
+      } catch (Throwable t) {
+        ExceptionUtil.rethrowIfInterrupt(t);
+        LOG.debug("Got exception in closing the meta scanner visitor", t);
+      }
+    }
+  }
+
+  /**
+   * @return Get closest metatable region row to passed <code>row</code>
+   */
+  @NonNull
+  private static RegionInfo getClosestRegionInfo(Connection connection,
+    @NonNull final TableName tableName, @NonNull final byte[] row) throws IOException {
+    byte[] searchRow = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
+    Scan scan = getCatalogScan(connection, 1);
+    scan.setReversed(true);
+    scan.withStartRow(searchRow);
+    try (ResultScanner resultScanner =
+        getCatalogHTable(connection, getCatalogTableForTable(tableName)).getScanner(scan)) {
+      Result result = resultScanner.next();
+      if (result == null) {
+        throw new TableNotFoundException("Cannot find row in META " + " for table: " + tableName +
+          ", row=" + Bytes.toStringBinary(row));
+      }
+      RegionInfo regionInfo = CatalogFamilyFormat.getRegionInfo(result);
+      if (regionInfo == null) {
+        throw new IOException("RegionInfo was null or empty in Meta for " + tableName + ", row=" +
+          Bytes.toStringBinary(row));
+      }
+      return regionInfo;
+    }
+  }
+
+  /**
+   * Returns the {@link ServerName} from catalog table {@link Result} where the region is
+   * transitioning on. It should be the same as
+   * {@link CatalogFamilyFormat#getServerName(Result,int)} if the server is at OPEN state.
+   * @param r Result to pull the transitioning server name from
+   * @return A ServerName instance or {@link CatalogFamilyFormat#getServerName(Result,int)} if
+   *         necessary fields not found or empty.
+   */
+  @Nullable
+  public static ServerName getTargetServerName(final Result r, final int replicaId) {
+    final Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY,
+      CatalogFamilyFormat.getServerNameColumn(replicaId));
+    if (cell == null || cell.getValueLength() == 0) {
+      RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+      if (locations != null) {
+        HRegionLocation location = locations.getRegionLocation(replicaId);
+        if (location != null) {
+          return location.getServerName();
+        }
+      }
+      return null;
+    }
+    return ServerName.parseServerName(
+      Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
+  }
+
+  /**
+   * Returns the daughter regions by reading the corresponding columns of the catalog table Result.
+   * @param data a Result object from the catalog table scan
+   * @return pair of RegionInfo or PairOfSameType(null, null) if region is not a split parent
+   */
+  public static PairOfSameType<RegionInfo> getDaughterRegions(Result data) {
+    RegionInfo splitA = CatalogFamilyFormat.getRegionInfo(data, HConstants.SPLITA_QUALIFIER);
+    RegionInfo splitB = CatalogFamilyFormat.getRegionInfo(data, HConstants.SPLITB_QUALIFIER);
+    return new PairOfSameType<>(splitA, splitB);
+  }
+
+  /**
+   * Fetch table state for given table from META table
+   * @param conn connection to use
+   * @param tableName table to fetch state for
+   */
+  @Nullable
+  public static TableState getTableState(Connection conn, TableName tableName)
+    throws IOException {
+    if (tableName.equals(TableName.ROOT_TABLE_NAME) ||
+      tableName.equals(TableName.META_TABLE_NAME)) {
+      return new TableState(tableName, TableState.State.ENABLED);
+    }
+    Table metaHTable = getCatalogHTable(conn, TableName.META_TABLE_NAME);
+    Get get = new Get(tableName.getName()).addColumn(HConstants.TABLE_FAMILY,
+      HConstants.TABLE_STATE_QUALIFIER);
+    Result result = metaHTable.get(get);
+    return CatalogFamilyFormat.getTableState(result);
+  }
+
+  /**
+   * Fetch table states from META table
+   * @param conn connection to use
+   * @return map {tableName -&gt; state}
+   */
+  public static Map<TableName, TableState> getTableStates(Connection conn) throws IOException {
+    final Map<TableName, TableState> states = new LinkedHashMap<>();
+    ClientMetaTableAccessor.Visitor collector = r -> {
+      TableState state = CatalogFamilyFormat.getTableState(r);
+      if (state != null) {
+        states.put(state.getTableName(), state);
+      }
+      return true;
+    };
+    fullScanTables(conn, collector);
+    return states;
+  }
+
+  /**
+   * Updates state in META Do not use. For internal use only.
+   * @param conn connection to use
+   * @param tableName table to look for
+   */
+  public static void updateTableState(Connection conn, TableName tableName, TableState.State actual)
+    throws IOException {
+    updateTableState(conn, new TableState(tableName, actual));
+  }
+
+  ////////////////////////
+  // Editing operations //
+  ////////////////////////
+  /**
+   * Generates and returns a Put containing the region into for the catalog table
+   */
+  public static Put makePutFromRegionInfo(RegionInfo regionInfo, long ts) throws IOException {
+    return addRegionInfo(new Put(regionInfo.getRegionName(), ts), regionInfo);
+  }
+
+  /**
+   * Generates and returns a Delete containing the region info for the catalog table
+   */
+  public static Delete makeDeleteFromRegionInfo(RegionInfo regionInfo, long ts) {
+    if (regionInfo == null) {
+      throw new IllegalArgumentException("Can't make a delete for null region");
+    }
+    Delete delete = new Delete(regionInfo.getRegionName());
+    delete.addFamily(HConstants.CATALOG_FAMILY, ts);
+    return delete;
+  }
+
+  /**
+   * Adds split daughters to the Put
+   */
+  public static Put addDaughtersToPut(Put put, RegionInfo splitA, RegionInfo splitB)
+    throws IOException {
+    if (splitA != null) {
+      put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
+        .setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.SPLITA_QUALIFIER)
+        .setTimestamp(put.getTimestamp()).setType(Type.Put).setValue(RegionInfo.toByteArray(splitA))
+        .build());
+    }
+    if (splitB != null) {
+      put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
+        .setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.SPLITB_QUALIFIER)
+        .setTimestamp(put.getTimestamp()).setType(Type.Put).setValue(RegionInfo.toByteArray(splitB))
+        .build());
+    }
+    return put;
+  }
+
+  /**
+   * Put the passed <code>p</code> to the <code>hbase:meta</code> table.
+   * @param connection connection we're using
+   * @param p Put to add to hbase:meta
+   */
+  private static void putToMetaTable(Connection connection, Put p) throws IOException {
+    try (Table table = getCatalogHTable(connection, TableName.META_TABLE_NAME)) {
+      put(table, p);
+    }
+  }
+
+  /**
+   * @param t Table to use
+   * @param p put to make
+   */
+  private static void put(Table t, Put p) throws IOException {
+    debugLogMutation(t.getName(), p);
+    t.put(p);
+  }
+
+  /**
+   * Put the passed <code>ps</code> to the <code>hbase:meta</code> table.
+   * @param connection connection we're using
+   * @param ps Put to add to hbase:meta
+   */
+  public static void putsToCatalogTable(final Connection connection, TableName tableName,
+    final List<Put> ps)
+      throws IOException {
+    if (ps.isEmpty()) {
+      return;
+    }
+    try (Table t = getCatalogHTable(connection, tableName)) {
+      debugLogMutations(t.getName(), ps);
+      // the implementation for putting a single Put is much simpler so here we do a check first.
+      if (ps.size() == 1) {
+        t.put(ps.get(0));
+      } else {
+        t.put(ps);
+      }
+    }
+  }
+
+  /**
+   * Delete the passed <code>d</code> from the <code>hbase:meta</code> table.
+   * @param connection connection we're using
+   * @param d Delete to add to hbase:meta
+   */
+  private static void deleteFromCatalogTable(final Connection connection, TableName catalogTable,
+    final Delete d) throws IOException {
+    List<Delete> dels = new ArrayList<>(1);
+    dels.add(d);
+    deleteFromCatalogTable(connection, catalogTable, dels);
+  }
+
+  /**
+   * Delete the passed <code>deletes</code> from the <code>hbase:meta</code> table.
+   * @param connection connection we're using
+   * @param deletes Deletes to add to hbase:meta  This list should support #remove.
+   */
+  private static void deleteFromCatalogTable(final Connection connection,
+    TableName catalogTableName, final List<Delete> deletes) throws IOException {
+    try (Table t = getCatalogHTable(connection, catalogTableName)) {
+      debugLogMutations(catalogTableName, deletes);
+      t.delete(deletes);
+    }
+  }
+
+  /**
+   * Deletes some replica columns corresponding to replicas for the passed rows
+   * @param metaRows rows in hbase:meta
+   * @param replicaIndexToDeleteFrom the replica ID we would start deleting from
+   * @param numReplicasToRemove how many replicas to remove
+   * @param connection connection we're using to access meta table
+   */
+  public static void removeRegionReplicasFromCatalog(Set<byte[]> metaRows,
+    int replicaIndexToDeleteFrom, int numReplicasToRemove, Connection connection,
+    TableName catalogTableName) throws IOException {
+    int absoluteIndex = replicaIndexToDeleteFrom + numReplicasToRemove;
+    for (byte[] row : metaRows) {
+      long now = EnvironmentEdgeManager.currentTime();
+      Delete deleteReplicaLocations = new Delete(row);
+      for (int i = replicaIndexToDeleteFrom; i < absoluteIndex; i++) {
+        deleteReplicaLocations.addColumns(HConstants.CATALOG_FAMILY,
+          CatalogFamilyFormat.getServerColumn(i), now);
+        deleteReplicaLocations.addColumns(HConstants.CATALOG_FAMILY,
+          CatalogFamilyFormat.getSeqNumColumn(i), now);
+        deleteReplicaLocations.addColumns(HConstants.CATALOG_FAMILY,
+          CatalogFamilyFormat.getStartCodeColumn(i), now);
+        deleteReplicaLocations.addColumns(HConstants.CATALOG_FAMILY,
+          CatalogFamilyFormat.getServerNameColumn(i), now);
+        deleteReplicaLocations.addColumns(HConstants.CATALOG_FAMILY,
+          CatalogFamilyFormat.getRegionStateColumn(i), now);
+      }
+
+      deleteFromCatalogTable(connection, catalogTableName, deleteReplicaLocations);
+    }
+  }
+
+  public static Put addRegionStateToPut(Put put, RegionState.State state) throws IOException {
+    put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
+      .setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.STATE_QUALIFIER)
+      .setTimestamp(put.getTimestamp()).setType(Cell.Type.Put).setValue(Bytes.toBytes(state.name()))
+      .build());
+    return put;
+  }
+
+  /**
+   * Update state column in hbase:meta.
+   */
+  public static void updateRegionState(Connection connection, RegionInfo ri,
+    RegionState.State state) throws IOException {
+    Put put = new Put(RegionReplicaUtil.getRegionInfoForDefaultReplica(ri).getRegionName());
+    putsToCatalogTable(connection, getCatalogTableForTable(ri.getTable()),
+        Collections.singletonList(addRegionStateToPut(put, state)));
+  }
+
+  /**
+   * Adds daughter region infos to hbase:meta row for the specified region. Note that this does not
+   * add its daughter's as different rows, but adds information about the daughters in the same row
+   * as the parent. Use
+   * {@link #splitRegion(Connection, RegionInfo, long, RegionInfo, RegionInfo, ServerName, int)} if
+   * you want to do that.
+   * @param connection connection we're using
+   * @param regionInfo RegionInfo of parent region
+   * @param splitA first split daughter of the parent regionInfo
+   * @param splitB second split daughter of the parent regionInfo
+   * @throws IOException if problem connecting or updating meta
+   */
+  public static void addSplitsToParent(Connection connection, RegionInfo regionInfo,
+      RegionInfo splitA, RegionInfo splitB) throws IOException {
+    TableName catalogTable = getCatalogTableForTable(regionInfo.getTable());
+    try (Table catalog = getCatalogHTable(connection, catalogTable)) {
+      Put put = makePutFromRegionInfo(regionInfo, EnvironmentEdgeManager.currentTime());
+      addDaughtersToPut(put, splitA, splitB);
+      catalog.put(put);
+      debugLogMutation(catalogTable, put);
+      LOG.debug("Added region {}", regionInfo.getRegionNameAsString());
+    }
+  }
+
+  /**
+   * Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
+   * does not add its daughter's as different rows, but adds information about the daughters in the
+   * same row as the parent. Use
+   * {@link #splitRegion(Connection, RegionInfo, long, RegionInfo, RegionInfo, ServerName, int)} if
+   * you want to do that.
+   * @param connection connection we're using
+   * @param regionInfo region information
+   * @throws IOException if problem connecting or updating meta
+   */
+  @VisibleForTesting
+  public static void addRegionToMeta(Connection connection, RegionInfo regionInfo)
+    throws IOException {
+    addRegionsToMeta(connection, Collections.singletonList(regionInfo), 1);
+  }
+
+  /**
+   * Adds a hbase:meta row for each of the specified new regions. Initial state for new regions is
+   * CLOSED.
+   * @param connection connection we're using
+   * @param regionInfos region information list
+   * @throws IOException if problem connecting or updating meta
+   */
+  public static void addRegionsToMeta(Connection connection, List<RegionInfo> regionInfos,
+    int regionReplication) throws IOException {
+    addRegionsToMeta(connection, regionInfos, regionReplication,
+      EnvironmentEdgeManager.currentTime());
+  }
+
+  /**
+   * Adds a hbase:meta row for each of the specified new regions. Initial state for new regions is
+   * CLOSED.
+   * @param connection connection we're using
+   * @param regionInfos region information list
+   * @param ts desired timestamp
+   * @throws IOException if problem connecting or updating meta
+   */
+  public static void addRegionsToMeta(Connection connection, List<RegionInfo> regionInfos,
+    int regionReplication, long ts) throws IOException {
+    List<Put> puts = new ArrayList<>();
+    for (RegionInfo regionInfo : regionInfos) {
+      if (RegionReplicaUtil.isDefaultReplica(regionInfo)) {
+        Put put = makePutFromRegionInfo(regionInfo, ts);
+        // New regions are added with initial state of CLOSED.
+        addRegionStateToPut(put, RegionState.State.CLOSED);
+        // Add empty locations for region replicas so that number of replicas can be cached
+        // whenever the primary region is looked up from meta
+        for (int i = 1; i < regionReplication; i++) {
+          addEmptyLocation(put, i);
+        }
+        puts.add(put);
+      }
+    }
+    putsToCatalogTable(connection, TableName.META_TABLE_NAME, puts);
+    LOG.info("Added {} regions to {}.",puts.size(), TableName.META_TABLE_NAME);
+  }
+
+  /**
+   * Update state of the table in meta.
+   * @param connection what we use for update
+   * @param state new state
+   */
+  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);
+  }
+
+  /**
+   * Construct PUT for given state
+   * @param state new state
+   */
+  public static Put makePutFromTableState(TableState state, long ts) {
+    Put put = new Put(state.getTableName().getName(), ts);
+    put.addColumn(HConstants.TABLE_FAMILY, HConstants.TABLE_STATE_QUALIFIER,
+      state.convert().toByteArray());
+    return put;
+  }
+
+  /**
+   * Remove state for table from meta
+   * @param connection to use for deletion
+   * @param table to delete state for
+   */
+  public static void deleteTableState(Connection connection, TableName table) throws IOException {
+    long time = EnvironmentEdgeManager.currentTime();
+    Delete delete = new Delete(table.getName());
+    delete.addColumns(HConstants.TABLE_FAMILY, HConstants.TABLE_STATE_QUALIFIER, time);
+    deleteFromCatalogTable(connection, TableName.META_TABLE_NAME, delete);
+    LOG.info("Deleted table " + table + " state from Catalog");
+  }
+
+  /**
+   * Updates the location of the specified region in hbase:meta to be the specified server hostname
+   * and startcode.
+   * <p>
+   * Uses passed catalog tracker to get a connection to the server hosting hbase:meta and makes
+   * edits to that region.
+   * @param connection connection we're using
+   * @param regionInfo region to update location of
+   * @param openSeqNum the latest sequence number obtained when the region was open
+   * @param sn Server name
+   * @param masterSystemTime wall clock time from master if passed in the open region RPC
+   */
+  @VisibleForTesting
+  public static void updateRegionLocation(Connection connection, RegionInfo regionInfo,
+    ServerName sn, long openSeqNum, long masterSystemTime) throws IOException {
+    updateLocation(connection, regionInfo, sn, openSeqNum, masterSystemTime);
+  }
+
+  /**
+   * Updates the location of the specified region to be the specified server.
+   * <p>
+   * Connects to the specified server which should be hosting the specified catalog region name to
+   * perform the edit.
+   * @param connection connection we're using
+   * @param regionInfo region to update location of
+   * @param sn Server name
+   * @param openSeqNum the latest sequence number obtained when the region was open
+   * @param masterSystemTime wall clock time from master if passed in the open region RPC
+   * @throws IOException In particular could throw {@link java.net.ConnectException} if the server
+   *           is down on other end.
+   */
+  private static void updateLocation(Connection connection, RegionInfo regionInfo, ServerName sn,
+    long openSeqNum, long masterSystemTime) throws IOException {
+    // region replicas are kept in the primary region's row
+    Put put = new Put(CatalogFamilyFormat.getMetaKeyForRegion(regionInfo), masterSystemTime);
+    addRegionInfo(put, regionInfo);
+    addLocation(put, sn, openSeqNum, regionInfo.getReplicaId());
+    putToMetaTable(connection, put);
+    LOG.info("Updated row {} with server=", regionInfo.getRegionNameAsString(), sn);
+  }
+
+  public static Put addRegionInfo(final Put p, final RegionInfo hri) throws IOException {
+    p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(p.getRow())
+      .setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.REGIONINFO_QUALIFIER)
+      .setTimestamp(p.getTimestamp()).setType(Type.Put)
+      // Serialize the Default Replica HRI otherwise scan of hbase:meta
+      // shows an info:regioninfo value with encoded name and region
+      // name that differs from that of the hbase;meta row.
+      .setValue(RegionInfo.toByteArray(RegionReplicaUtil.getRegionInfoForDefaultReplica(hri)))
+      .build());
+    return p;
+  }
+
+  public static Put addLocation(Put p, ServerName sn, long openSeqNum, int replicaId)
+    throws IOException {
+    CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
+    return p
+      .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)
+        .setQualifier(CatalogFamilyFormat.getServerColumn(replicaId)).setTimestamp(p.getTimestamp())
+        .setType(Cell.Type.Put).setValue(Bytes.toBytes(sn.getAddress().toString())).build())
+      .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)
+        .setQualifier(CatalogFamilyFormat.getStartCodeColumn(replicaId))
+        .setTimestamp(p.getTimestamp()).setType(Cell.Type.Put)
+        .setValue(Bytes.toBytes(sn.getStartcode())).build())
+      .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)
+        .setQualifier(CatalogFamilyFormat.getSeqNumColumn(replicaId)).setTimestamp(p.getTimestamp())
+        .setType(Type.Put).setValue(Bytes.toBytes(openSeqNum)).build());
+  }
+
+  public static Put addEmptyLocation(Put p, int replicaId) throws IOException {
+    CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
+    return p
+      .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)
+        .setQualifier(CatalogFamilyFormat.getServerColumn(replicaId)).setTimestamp(p.getTimestamp())
+        .setType(Type.Put).build())
+      .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)
+        .setQualifier(CatalogFamilyFormat.getStartCodeColumn(replicaId))
+        .setTimestamp(p.getTimestamp()).setType(Cell.Type.Put).build())
+      .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)
+        .setQualifier(CatalogFamilyFormat.getSeqNumColumn(replicaId)).setTimestamp(p.getTimestamp())
+        .setType(Cell.Type.Put).build());
+  }
+
+  private static void debugLogMutations(TableName tableName, List<? extends Mutation> mutations)
+    throws IOException {
+    if (!CATALOGLOG.isDebugEnabled()) {
+      return;
+    }
+    // Logging each mutation in separate line makes it easier to see diff between them visually
+    // because of common starting indentation.
+    for (Mutation mutation : mutations) {
+      debugLogMutation(tableName, mutation);
+    }
+  }
+
+  private static void debugLogMutation(TableName t, Mutation p) throws IOException {
+    CATALOGLOG.debug("{} {} {}", t, p.getClass().getSimpleName(), p.toJSON());
+  }
+}

Review comment:
       I went halfway on this class. It seems like needs a bit of tidying around when hbase:meta and when both. Some method renames and javadoc cleanups needed I think.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#issuecomment-698279168


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 38s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  2s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  2s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 37s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 14s |  master passed  |
   | +1 :green_heart: |  spotbugs  |  17m 16s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 25s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 19s |  root: The patch generated 68 new + 916 unchanged - 5 fixed = 984 total (was 921)  |
   | -0 :warning: |  whitespace  |   0m  0s |  The patch has 1 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  hadoopcheck  |  13m 24s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   7m 24s |  the patch passed  |
   | -1 :x: |  spotbugs  |   1m 14s |  hbase-client generated 3 new + 0 unchanged - 0 fixed = 3 total (was 0)  |
   | -1 :x: |  spotbugs  |   2m 14s |  hbase-server generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -1 :x: |  spotbugs  |   9m 41s |  root generated 4 new + 0 unchanged - 0 fixed = 4 total (was 0)  |
   ||| _ Other Tests _ |
   | -1 :x: |  asflicense  |   1m 36s |  The patch generated 1 ASF License warnings.  |
   |  |   |  83m 27s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-client |
   |  |  Nullcheck of other at line 829 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.isOverlap(RegionInfo)  At RegionInfo.java:829 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.isOverlap(RegionInfo)  At RegionInfo.java:[line 828] |
   |  |  Nullcheck of rhs at line 109 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.lambda$static$0(RegionInfo, RegionInfo)  At RegionInfo.java:109 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.lambda$static$0(RegionInfo, RegionInfo)  At RegionInfo.java:[line 108] |
   |  |  Nullcheck of regionA at line 444 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.areAdjacent(RegionInfo, RegionInfo)  At RegionInfo.java:444 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.areAdjacent(RegionInfo, RegionInfo)  At RegionInfo.java:[line 443] |
   | FindBugs | module:hbase-server |
   |  |  Possible null pointer dereference of region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:[line 149] |
   | FindBugs | module:root |
   |  |  Nullcheck of other at line 829 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.isOverlap(RegionInfo)  At RegionInfo.java:829 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.isOverlap(RegionInfo)  At RegionInfo.java:[line 828] |
   |  |  Nullcheck of rhs at line 109 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.lambda$static$0(RegionInfo, RegionInfo)  At RegionInfo.java:109 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.lambda$static$0(RegionInfo, RegionInfo)  At RegionInfo.java:[line 108] |
   |  |  Nullcheck of regionA at line 444 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.areAdjacent(RegionInfo, RegionInfo)  At RegionInfo.java:444 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.areAdjacent(RegionInfo, RegionInfo)  At RegionInfo.java:[line 443] |
   |  |  Possible null pointer dereference of region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:[line 149] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2445 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle cc hbaseprotoc prototool |
   | uname | Linux dc508e71d578 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 687e53b7e4 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/1/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | whitespace | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/1/artifact/yetus-general-check/output/whitespace-eol.txt |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/1/artifact/yetus-general-check/output/new-spotbugs-hbase-client.html |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/1/artifact/yetus-general-check/output/new-spotbugs-hbase-server.html |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/1/artifact/yetus-general-check/output/new-spotbugs-root.html |
   | asflicense | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/1/artifact/yetus-general-check/output/patch-asflicense-problems.txt |
   | Max. process+thread count | 137 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-zookeeper hbase-balancer hbase-server hbase-backup . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506772798



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java
##########
@@ -206,16 +209,20 @@ public TableName getTable() {
    */
   @Override
   public boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey) {
-    if (Bytes.compareTo(rangeStartKey, rangeEndKey) > 0) {
+    CellComparator comparator = CellComparator.getComparator(tableName);

Review comment:
       Yep will do. 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506777664



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -152,9 +159,19 @@
   private static final int DEFAULT_RIT_STUCK_WARNING_THRESHOLD = 60 * 1000;
   public static final String UNEXPECTED_STATE_REGION = "Unexpected state for ";
 
-  private final ProcedureEvent<?> metaAssignEvent = new ProcedureEvent<>("meta assign");
+  private final ProcedureEvent<?> rootAssignEvent = new ProcedureEvent<>("root assign");
+  private final ProcedureEvent<?> rootLoadEvent = new ProcedureEvent<>("root load");
+
   private final ProcedureEvent<?> metaLoadEvent = new ProcedureEvent<>("meta load");
 
+  private final ConcurrentSkipListMap<byte[], ProcedureEvent<?>> metaAssignEventMap =

Review comment:
       This is here so procedures that depend on particular regions of meta to be available (eg SCP, RTSP, etc) are only waiting on the necessary regions.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506776199



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
##########
@@ -257,46 +327,62 @@ private void cleanupSplitDir(MasterProcedureEnv env) {
         // old hbase:meta tenancy on this server; clean these up if any before trying to remove the
         // WAL directory of this server or we will fail. See archiveMetaLog comment for more details
         // on this condition.
-        env.getMasterServices().getMasterWalManager().archiveMetaLog(this.serverName);
+        env.getMasterServices().getMasterWalManager().archiveCatalogLog(this.serverName, false);
       }
       splitWALManager.deleteWALDir(serverName);
     } catch (IOException e) {
       LOG.warn("Remove WAL directory for {} failed, ignore...{}", serverName, e.getMessage());
     }
   }
 
-  private boolean isSplittingDone(MasterProcedureEnv env, boolean splitMeta) {
+  private boolean isSplittingDone(MasterProcedureEnv env, SplitWALManager.SplitType splitType) {
     SplitWALManager splitWALManager = env.getMasterServices().getSplitWALManager();
     try {
-      int wals = splitWALManager.getWALsToSplit(serverName, splitMeta).size();
-      LOG.debug("Check if {} WAL splitting is done? wals={}, meta={}", serverName, wals, splitMeta);
+      int wals = splitWALManager.getWALsToSplit(serverName, splitType).size();
+      LOG.debug("Check if {} WAL splitting is done? wals={}, SplitType={}",
+        serverName, wals, splitType);
       return wals == 0;
     } catch (IOException e) {
       LOG.warn("Get WALs of {} failed, retry...", serverName, e);
       return false;
     }
   }
 
-  private Procedure[] createSplittingWalProcedures(MasterProcedureEnv env, boolean splitMeta)
+  private Procedure[] createSplittingWalProcedures(MasterProcedureEnv env,
+    SplitWALManager.SplitType splitType)
       throws IOException {
-    LOG.info("Splitting WALs {}, isMeta: {}", this, splitMeta);
+    LOG.info("Splitting WALs {}, SplitType: {}", this, splitType);
     SplitWALManager splitWALManager = env.getMasterServices().getSplitWALManager();
-    List<Procedure> procedures = splitWALManager.splitWALs(serverName, splitMeta);
+    List<Procedure> procedures = splitWALManager.splitWALs(serverName, splitType);
     return procedures.toArray(new Procedure[procedures.size()]);
   }
 
   private boolean filterDefaultMetaRegions() {
     if (regionsOnCrashedServer == null) {
       return false;
     }
-    regionsOnCrashedServer.removeIf(this::isDefaultMetaRegion);
+    regionsOnCrashedServer.removeIf((x) -> isDefaultMetaRegion(x) || isDefaultRootRegion(x));
     return !regionsOnCrashedServer.isEmpty();
   }
 
+  private boolean isDefaultRootRegion(RegionInfo hri) {
+    return hri.isRootRegion() && RegionReplicaUtil.isDefaultReplica(hri);
+  }
+
   private boolean isDefaultMetaRegion(RegionInfo hri) {
     return hri.isMetaRegion() && RegionReplicaUtil.isDefaultReplica(hri);
   }
 
+  private void zkCoordinatedSplitRootLogs(MasterProcedureEnv env) throws IOException {
+    LOG.debug("Splitting root WALs {}", this);
+    MasterWalManager mwm = env.getMasterServices().getMasterWalManager();
+    AssignmentManager am = env.getMasterServices().getAssignmentManager();
+    am.getRegionStates().rootLogSplitting(serverName);
+    mwm.splitRootLog(serverName);
+    am.getRegionStates().rootLogSplit(serverName);
+    LOG.debug("Done splitting root WALs {}", this);
+  }

Review comment:
       will make sure to update things then. 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506774952



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
##########
@@ -33,6 +33,25 @@
   public static final RegionInfo UNDEFINED =
     RegionInfoBuilder.newBuilder(TableName.valueOf("__UNDEFINED__")).build();
 
+  /**
+   * RegionInfo for first root region
+   * You cannot use this builder to make an instance of the {@link #ROOT_REGIONINFO}.
+   * Just refer to this instance. Also, while the instance is actually a MutableRI, its type is
+   * just RI so the mutable methods are not available (unless you go casting); it appears
+   * as immutable (I tried adding Immutable type but it just makes a mess).
+   *
+   * We are using the non-legacy encoding format to reduce the boilerplace code
+   */
+  public static final RegionInfo ROOT_REGIONINFO =
+    new MutableRegionInfo(TableName.ROOT_TABLE_NAME,
+      HConstants.EMPTY_START_ROW,
+      HConstants.EMPTY_END_ROW,
+      false,
+      0,
+      RegionInfo.DEFAULT_REPLICA_ID,
+      false);

Review comment:
       That would be in the table schema. Should still be changeable. I didn't not remove support.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
##########
@@ -33,6 +33,25 @@
   public static final RegionInfo UNDEFINED =
     RegionInfoBuilder.newBuilder(TableName.valueOf("__UNDEFINED__")).build();
 
+  /**
+   * RegionInfo for first root region
+   * You cannot use this builder to make an instance of the {@link #ROOT_REGIONINFO}.
+   * Just refer to this instance. Also, while the instance is actually a MutableRI, its type is
+   * just RI so the mutable methods are not available (unless you go casting); it appears
+   * as immutable (I tried adding Immutable type but it just makes a mess).
+   *
+   * We are using the non-legacy encoding format to reduce the boilerplace code
+   */
+  public static final RegionInfo ROOT_REGIONINFO =
+    new MutableRegionInfo(TableName.ROOT_TABLE_NAME,
+      HConstants.EMPTY_START_ROW,
+      HConstants.EMPTY_END_ROW,
+      false,
+      0,
+      RegionInfo.DEFAULT_REPLICA_ID,
+      false);

Review comment:
       That would be in the table schema?  Should still be changeable. I didn't remove support.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r503642848



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
##########
@@ -33,6 +33,25 @@
   public static final RegionInfo UNDEFINED =
     RegionInfoBuilder.newBuilder(TableName.valueOf("__UNDEFINED__")).build();
 
+  /**
+   * RegionInfo for first root region
+   * You cannot use this builder to make an instance of the {@link #ROOT_REGIONINFO}.
+   * Just refer to this instance. Also, while the instance is actually a MutableRI, its type is
+   * just RI so the mutable methods are not available (unless you go casting); it appears
+   * as immutable (I tried adding Immutable type but it just makes a mess).

Review comment:
       Good

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
##########
@@ -33,6 +33,25 @@
   public static final RegionInfo UNDEFINED =
     RegionInfoBuilder.newBuilder(TableName.valueOf("__UNDEFINED__")).build();
 
+  /**
+   * RegionInfo for first root region

Review comment:
       first? Only?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
##########
@@ -300,8 +300,12 @@ public static CellComparator getCellComparator(TableName tableName) {
    * @return CellComparator to use going off the {@code tableName} passed.
    */
   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())?
-      MetaCellComparator.META_COMPARATOR: CellComparatorImpl.COMPARATOR;
+    if (Bytes.equals(tableName, TableName.ROOT_TABLE_NAME.toBytes())) {
+      return RootCellComparator.ROOT_COMPARATOR;
+    }
+    if (Bytes.equals(tableName, TableName.META_TABLE_NAME.toBytes())) {
+      return MetaCellComparator.META_COMPARATOR;
+    }
+    return CellComparatorImpl.COMPARATOR;

Review comment:
       Can't this call the Interface version?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
##########
@@ -2531,6 +2531,7 @@ public GetTableStateResponse setTableStateInMeta(RpcController controller,
    *
    * @return previous states of the regions
    */
+  //TODO francis support root here

Review comment:
       Whats this?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -152,9 +159,19 @@
   private static final int DEFAULT_RIT_STUCK_WARNING_THRESHOLD = 60 * 1000;
   public static final String UNEXPECTED_STATE_REGION = "Unexpected state for ";
 
-  private final ProcedureEvent<?> metaAssignEvent = new ProcedureEvent<>("meta assign");
+  private final ProcedureEvent<?> rootAssignEvent = new ProcedureEvent<>("root assign");
+  private final ProcedureEvent<?> rootLoadEvent = new ProcedureEvent<>("root load");
+
   private final ProcedureEvent<?> metaLoadEvent = new ProcedureEvent<>("meta load");
 
+  private final ConcurrentSkipListMap<byte[], ProcedureEvent<?>> metaAssignEventMap =

Review comment:
       Whats this about?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
##########
@@ -250,6 +260,7 @@
     DEFAULT_VALUES.put(PRIORITY, String.valueOf(DEFAULT_PRIORITY));
     DEFAULT_VALUES.keySet().stream()
             .map(s -> new Bytes(Bytes.toBytes(s))).forEach(RESERVED_KEYWORDS::add);
+    RESERVED_KEYWORDS.add(IS_ROOT_KEY);
     RESERVED_KEYWORDS.add(IS_META_KEY);

Review comment:
       ouch. who did this.
   
   Ok. I suppose ROOT needs to follow form.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
##########
@@ -33,6 +33,25 @@
   public static final RegionInfo UNDEFINED =
     RegionInfoBuilder.newBuilder(TableName.valueOf("__UNDEFINED__")).build();
 
+  /**
+   * RegionInfo for first root region
+   * You cannot use this builder to make an instance of the {@link #ROOT_REGIONINFO}.
+   * Just refer to this instance. Also, while the instance is actually a MutableRI, its type is
+   * just RI so the mutable methods are not available (unless you go casting); it appears
+   * as immutable (I tried adding Immutable type but it just makes a mess).
+   *
+   * We are using the non-legacy encoding format to reduce the boilerplace code

Review comment:
       Whats this?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
##########
@@ -124,6 +124,16 @@
   @InterfaceAudience.Private
   public static final String FLUSH_POLICY = "FLUSH_POLICY";
   private static final Bytes FLUSH_POLICY_KEY = new Bytes(Bytes.toBytes(FLUSH_POLICY));
+
+  /**
+   * Used by rest interface to access this metadata attribute
+   * which denotes if it is a catalog table, either <code> hbase:meta </code>.
+   */
+  @InterfaceAudience.Private
+  public static final String IS_ROOT = "IS_ROOT";

Review comment:
       s/IS_ROOT/ROOT/?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
##########
@@ -124,6 +124,16 @@
   @InterfaceAudience.Private
   public static final String FLUSH_POLICY = "FLUSH_POLICY";
   private static final Bytes FLUSH_POLICY_KEY = new Bytes(Bytes.toBytes(FLUSH_POLICY));
+
+  /**
+   * Used by rest interface to access this metadata attribute
+   * which denotes if it is a catalog table, either <code> hbase:meta </code>.
+   */
+  @InterfaceAudience.Private
+  public static final String IS_ROOT = "IS_ROOT";
+  private static final Bytes IS_ROOT_KEY

Review comment:
       s/IS_ROOT_KEY/ROOT_KEY/

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
##########
@@ -33,6 +33,25 @@
   public static final RegionInfo UNDEFINED =
     RegionInfoBuilder.newBuilder(TableName.valueOf("__UNDEFINED__")).build();
 
+  /**
+   * RegionInfo for first root region
+   * You cannot use this builder to make an instance of the {@link #ROOT_REGIONINFO}.
+   * Just refer to this instance. Also, while the instance is actually a MutableRI, its type is
+   * just RI so the mutable methods are not available (unless you go casting); it appears
+   * as immutable (I tried adding Immutable type but it just makes a mess).
+   *
+   * We are using the non-legacy encoding format to reduce the boilerplace code
+   */
+  public static final RegionInfo ROOT_REGIONINFO =
+    new MutableRegionInfo(TableName.ROOT_TABLE_NAME,
+      HConstants.EMPTY_START_ROW,
+      HConstants.EMPTY_END_ROW,
+      false,
+      0,
+      RegionInfo.DEFAULT_REPLICA_ID,
+      false);

Review comment:
       Is this schema immutable? What if I want to change it? To up the read repicas or to add a bloom or an encoding?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
##########
@@ -158,4 +174,14 @@ default int compareRows(ByteBuffer row, Cell cell) {
    *   Do not pollute with types other than BBKV if can be helped; the Comparator will slow.
    */
   Comparator getSimpleComparator();
+
+  static CellComparator getComparator(TableName tableName) {
+    if (tableName.equals(TableName.ROOT_TABLE_NAME)) {
+      return RootCellComparator.ROOT_COMPARATOR;
+    }
+    if (tableName.equals(TableName.META_TABLE_NAME)) {
+      return MetaCellComparator.META_COMPARATOR;
+    }
+    return CellComparatorImpl.COMPARATOR;

Review comment:
       This makes sense here. You can't use it from async client above? You have to repro it?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -511,6 +654,39 @@ public boolean isMetaLoaded() {
     return metaLoadEvent.isReady();
   }
 
+
+  /**
+   * @return True if region is online and scannable else false if an error or shutdown (Otherwise
+   *   we just block in here holding up all forward-progess).

Review comment:
       Radical

##########
File path: hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
##########
@@ -205,6 +243,37 @@ public void testMetaComparisons2() {
             Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",,1"), now))) > 0);
   }
 
+  @Test
+  public void testRootComparisons2() {

Review comment:
       There used to be a million tests for the Root comparator... there were all kinds of interesting corner cases. Need to steal them and bring them back?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
##########
@@ -124,7 +124,7 @@
 import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
 import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
-import org.apache.hadoop.hbase.master.procedure.InitMetaProcedure;
+import org.apache.hadoop.hbase.master.procedure.InitRootProcedure;

Review comment:
       FYI, the InitMetaProcedure just removed from master and branch-2.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
##########
@@ -110,6 +111,15 @@
   @Deprecated
   public static final KVComparator META_COMPARATOR = new MetaComparator();
 
+  /**
+   * A {@link KVComparator} for <code>hbase:root</code> catalog table
+   * {@link KeyValue}s.
+   * @deprecated Use {@link RootCellComparator#ROOT_COMPARATOR} instead.
+   * Deprecated for hbase 2.0, remove for hbase 3.0.
+   */
+  @Deprecated
+  public static final KVComparator ROOT_COMPARATOR = new RootComparator();

Review comment:
       Can it be a CellComparator? Does it have to be KVComparator?

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
##########
@@ -1604,6 +1614,79 @@ public static int getDelimiterInReverse(final byte [] b, final int offset,
     return result;
   }
 
+  /**
+   * A {@link KVComparator} for <code>-ROOT-</code> catalog table
+   * {@link KeyValue}s.
+   * @deprecated : {@link RootCellComparator#ROOT_COMPARATOR} to be used.
+   * Deprecated for hbase 2.0, remove for hbase 3.0.

Review comment:
       Why this add and deprecate?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -1440,10 +1625,42 @@ private void handleRegionOverStuckWarningThreshold(final RegionInfo regionInfo)
   // ============================================================================================
   //  TODO: Master load/bootstrap
   // ============================================================================================
-  public void joinCluster() throws IOException {
+  public void joinCluster(boolean loadRoot) throws IOException {
+    joinCluster(loadRoot, true, true);
+  }
+
+  @VisibleForTesting
+  public void joinCluster(boolean loadRoot, boolean shouldWaitForRootOnline,
+    boolean shouldWaitForMetaOnline)
+    throws IOException {
     long startTime = System.nanoTime();
     LOG.debug("Joining cluster...");
 
+    // FIRST Catalog tables READ!!!!
+    // The below cannot make progress w/o hbase:meta being online.

Review comment:
       This talks of hbase:meta but below is hbase;root

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCatalogBootstrap.java
##########
@@ -61,23 +68,56 @@ void assignMetaReplicas()
       throw new IllegalStateException("hbase:meta must be initialized first before we can " +
           "assign out its replicas");
     }
-    ServerName metaServername = MetaTableLocator.getMetaRegionLocation(this.master.getZooKeeper());
+    ServerName rootServername = RootTableLocator.getRootRegionLocation(this.master.getZooKeeper());
     for (int i = 1; i < numReplicas; i++) {
-      // Get current meta state for replica from zk.
-      RegionState metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper(), i);
+      // Get current hbase:root state for replica from zk.
+      RegionState rootState = RootTableLocator.getRootRegionState(master.getZooKeeper(), i);
       RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
-          RegionInfoBuilder.FIRST_META_REGIONINFO, i);
-      LOG.debug(hri.getRegionNameAsString() + " replica region state from zookeeper=" + metaState);
-      if (metaServername.equals(metaState.getServerName())) {
-        metaState = null;
+          RegionInfoBuilder.ROOT_REGIONINFO, i);

Review comment:
       Or rather I think this just removed in master.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -430,11 +553,19 @@ private boolean isCarryingRegion(final ServerName serverName, final RegionInfo r
     return(node != null && serverName.equals(node.getRegionLocation()));
   }
 
-  private RegionInfo getMetaForRegion(final RegionInfo regionInfo) {
-    //if (regionInfo.isMetaRegion()) return regionInfo;
-    // TODO: handle multiple meta. if the region provided is not meta lookup
-    // which meta the region belongs to.
-    return RegionInfoBuilder.FIRST_META_REGIONINFO;
+  /**
+   * Check hbase:meta is up and ready for reading. For use during Master startup only.
+   * @return True if meta is UP and online and startup can progress. Otherwise, meta is not online
+   *   and we will hold here until operator intervention.
+   */
+  @VisibleForTesting
+  public boolean waitForMetaOnline() {

Review comment:
       So only if all meta regions are up? What if 10k meta regions? We wait till all up? Can fix in follow-on?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -400,28 +419,132 @@ public boolean isTableDisabled(final TableName tableName) {
   }
 
   // ============================================================================================
-  //  META Helpers
+  //  ROOT Helpers
   // ============================================================================================
-  private boolean isMetaRegion(final RegionInfo regionInfo) {
-    return regionInfo.isMetaRegion();
+  private boolean isRootRegion(final RegionInfo regionInfo) {
+    return regionInfo.isRootRegion();
   }
 
-  public boolean isMetaRegion(final byte[] regionName) {
-    return getMetaRegionFromName(regionName) != null;
+  public boolean isCarryingRoot(final ServerName serverName) {
+    // TODO: handle multiple root
+    return isCarryingRegion(serverName, RegionInfoBuilder.ROOT_REGIONINFO);
   }
 
-  public RegionInfo getMetaRegionFromName(final byte[] regionName) {
-    for (RegionInfo hri: getMetaRegionSet()) {
-      if (Bytes.equals(hri.getRegionName(), regionName)) {
-        return hri;
-      }
+  private RegionInfo getRootForRegion(final RegionInfo regionInfo) {
+    //if (regionInfo.isRootRegion()) return regionInfo;
+    // TODO: handle multiple root. if the region provided is not root lookup
+    // which root the region belongs to.
+    return RegionInfoBuilder.ROOT_REGIONINFO;
+  }
+
+  /**
+   * Check hbase:root is up and ready for reading. For use during Master startup only.
+   * @return True if root is UP and online and startup can progress. Otherwise, root is not online
+   *   and we will hold here until operator intervention.
+   */
+  public boolean waitForRootOnline() {
+    return isRegionOnline(RegionInfoBuilder.ROOT_REGIONINFO);
+  }
+
+  private static final Set<RegionInfo> ROOT_REGION_SET =
+    Collections.singleton(RegionInfoBuilder.ROOT_REGIONINFO);
+  public Set<RegionInfo> getRootRegionSet() {
+    return ROOT_REGION_SET;
+  }
+
+  // ============================================================================================
+  //  ROOT Event(s) helpers
+  // ============================================================================================
+  /**
+   * Notice that, this only means the root region is available on a RS, but the AM may still be
+   * loading the region states from root, so usually you need to check {@link #isRootLoaded()} first
+   * before checking this method, unless you can make sure that your piece of code can only be
+   * executed after AM builds the region states.
+   * @see #isRootLoaded()
+   */
+  public boolean isRootAssigned() {
+    return rootAssignEvent.isReady();
+  }
+
+  public boolean isRootRegionInTransition() {
+    return !isRootAssigned();
+  }
+
+  /**
+   * Notice that this event does not mean the AM has already finished region state rebuilding. See
+   * the comment of {@link #isRootAssigned()} for more details.
+   * @see #isRootAssigned()
+   */
+  public boolean waitRootAssigned(Procedure<?> proc, RegionInfo regionInfo) {
+    return getRootAssignEvent(getRootForRegion(regionInfo)).suspendIfNotReady(proc);
+  }
+
+  private void setRootAssigned(RegionInfo rootRegionInfo, boolean assigned) {
+    assert isRootRegion(rootRegionInfo) : "unexpected non-root region " + rootRegionInfo;
+    if (!RegionReplicaUtil.isDefaultReplica(rootRegionInfo)) {
+      return;
     }
-    return null;
+    ProcedureEvent<?> rootAssignEvent = getRootAssignEvent(rootRegionInfo);
+    if (assigned) {
+      LOG.debug("Setting hbase:root region assigned: "+rootRegionInfo);
+      rootAssignEvent.wake(getProcedureScheduler());
+    } else {
+      LOG.debug("Setting hbase:root region unassigned: "+rootRegionInfo);
+      rootAssignEvent.suspend();
+    }
+  }
+
+  private ProcedureEvent<?> getRootAssignEvent(RegionInfo rootRegionInfo) {
+    assert isRootRegion(rootRegionInfo) : "unexpected non-catalog region " + rootRegionInfo;
+    // TODO: handle multiple root.

Review comment:
       Only one root or do you mean replicas? The old stuff did async assign of replicas waiting on primary only FYI

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
##########
@@ -2531,6 +2531,7 @@ public GetTableStateResponse setTableStateInMeta(RpcController controller,
    *
    * @return previous states of the regions
    */
+  //TODO francis support root here

Review comment:
       Needs finishing?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -1926,6 +2225,27 @@ public void markRegionAsMerged(final RegionInfo child, final ServerName serverNa
     }
     TableDescriptor td = master.getTableDescriptors().get(child.getTable());
     regionStateStore.mergeRegions(child, mergeParents, serverName, td);
+
+    //Split meta assignment event
+    if (td.isMetaTable()) {
+      ProcedureEvent<?> parentEvent[] = new ProcedureEvent[mergeParents.length];
+      for (int i=0; i<parentEvent.length; i++) {
+        parentEvent[i] = metaAssignEventMap.get(mergeParents[i].getStartKey());
+      }
+
+      metaAssignEventMap.put(child.getStartKey(),
+        new ProcedureEvent<>("meta assign: " + child.getRegionNameAsString()));
+
+      //wake the procedures waiting on parent event, the procedures will awaken
+      //and wait on newly created child event
+      for (int i= mergeParents.length-1; i>=1; i--) {
+        metaAssignEventMap.remove(mergeParents[i].getStartKey());
+        parentEvent[i].wake(getProcedureScheduler());
+      }
+      //for the first key we don't remove since we already replaced it
+      parentEvent[0].wake(getProcedureScheduler());
+    }
+
     if (shouldAssignFavoredNodes(child)) {
       getFavoredNodePromoter().generateFavoredNodesForMergedRegion(child, mergeParents);
     }

Review comment:
       Concerned about blocking on Region transitions.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
##########
@@ -2986,6 +2988,7 @@ public GetMastersResponse getMasters(RpcController rpcController, GetMastersRequ
     return resp.build();
   }
 
+  //TODO francis this needs to be get RootLocation now?
   @Override
   public GetMetaRegionLocationsResponse getMetaRegionLocations(RpcController rpcController,

Review comment:
       You've seen the ConnectionRegistry stuff?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -768,6 +944,15 @@ public void move(RegionInfo regionInfo) throws IOException {
 
   @VisibleForTesting
   static int compare(TransitRegionStateProcedure left, TransitRegionStateProcedure right) {
+    //TODO francis this is broken once we have more meta entries

Review comment:
       Does this patch include support for split meta?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
##########
@@ -257,46 +327,62 @@ private void cleanupSplitDir(MasterProcedureEnv env) {
         // old hbase:meta tenancy on this server; clean these up if any before trying to remove the
         // WAL directory of this server or we will fail. See archiveMetaLog comment for more details
         // on this condition.
-        env.getMasterServices().getMasterWalManager().archiveMetaLog(this.serverName);
+        env.getMasterServices().getMasterWalManager().archiveCatalogLog(this.serverName, false);
       }
       splitWALManager.deleteWALDir(serverName);
     } catch (IOException e) {
       LOG.warn("Remove WAL directory for {} failed, ignore...{}", serverName, e.getMessage());
     }
   }
 
-  private boolean isSplittingDone(MasterProcedureEnv env, boolean splitMeta) {
+  private boolean isSplittingDone(MasterProcedureEnv env, SplitWALManager.SplitType splitType) {
     SplitWALManager splitWALManager = env.getMasterServices().getSplitWALManager();
     try {
-      int wals = splitWALManager.getWALsToSplit(serverName, splitMeta).size();
-      LOG.debug("Check if {} WAL splitting is done? wals={}, meta={}", serverName, wals, splitMeta);
+      int wals = splitWALManager.getWALsToSplit(serverName, splitType).size();
+      LOG.debug("Check if {} WAL splitting is done? wals={}, SplitType={}",
+        serverName, wals, splitType);
       return wals == 0;
     } catch (IOException e) {
       LOG.warn("Get WALs of {} failed, retry...", serverName, e);
       return false;
     }
   }
 
-  private Procedure[] createSplittingWalProcedures(MasterProcedureEnv env, boolean splitMeta)
+  private Procedure[] createSplittingWalProcedures(MasterProcedureEnv env,
+    SplitWALManager.SplitType splitType)
       throws IOException {
-    LOG.info("Splitting WALs {}, isMeta: {}", this, splitMeta);
+    LOG.info("Splitting WALs {}, SplitType: {}", this, splitType);
     SplitWALManager splitWALManager = env.getMasterServices().getSplitWALManager();
-    List<Procedure> procedures = splitWALManager.splitWALs(serverName, splitMeta);
+    List<Procedure> procedures = splitWALManager.splitWALs(serverName, splitType);
     return procedures.toArray(new Procedure[procedures.size()]);
   }
 
   private boolean filterDefaultMetaRegions() {
     if (regionsOnCrashedServer == null) {
       return false;
     }
-    regionsOnCrashedServer.removeIf(this::isDefaultMetaRegion);
+    regionsOnCrashedServer.removeIf((x) -> isDefaultMetaRegion(x) || isDefaultRootRegion(x));
     return !regionsOnCrashedServer.isEmpty();
   }
 
+  private boolean isDefaultRootRegion(RegionInfo hri) {
+    return hri.isRootRegion() && RegionReplicaUtil.isDefaultReplica(hri);
+  }
+
   private boolean isDefaultMetaRegion(RegionInfo hri) {
     return hri.isMetaRegion() && RegionReplicaUtil.isDefaultReplica(hri);
   }
 
+  private void zkCoordinatedSplitRootLogs(MasterProcedureEnv env) throws IOException {
+    LOG.debug("Splitting root WALs {}", this);
+    MasterWalManager mwm = env.getMasterServices().getMasterWalManager();
+    AssignmentManager am = env.getMasterServices().getAssignmentManager();
+    am.getRegionStates().rootLogSplitting(serverName);
+    mwm.splitRootLog(serverName);
+    am.getRegionStates().rootLogSplit(serverName);
+    LOG.debug("Done splitting root WALs {}", this);
+  }

Review comment:
       The zk coordinated splitter is deprecated in favor of a procedure-based one (was enabled about a month or more ago)




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#issuecomment-716450300


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 12s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  1s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 35s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 11s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 19s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 45s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 37s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 37s |  the patch passed  |
   | -1 :x: |  shadedjars  |   5m 38s |  patch has 10 errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-common generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  javadoc  |   2m 11s |  root generated 1 new + 32 unchanged - 0 fixed = 33 total (was 32)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  21m 23s |  root in the patch failed.  |
   |  |   |  60m 12s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2445 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 538973ea2b1b 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 164cc5a3dc |
   | Default Java | 1.8.0_232 |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-jdk8-hadoop3-check/output/patch-shadedjars.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-root.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/testReport/ |
   | Max. process+thread count | 908 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-zookeeper hbase-balancer hbase-server hbase-backup . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#issuecomment-698279168


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 38s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  2s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  2s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 37s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 14s |  master passed  |
   | +1 :green_heart: |  spotbugs  |  17m 16s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 25s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 19s |  root: The patch generated 68 new + 916 unchanged - 5 fixed = 984 total (was 921)  |
   | -0 :warning: |  whitespace  |   0m  0s |  The patch has 1 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  hadoopcheck  |  13m 24s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  hbaseprotoc  |   7m 24s |  the patch passed  |
   | -1 :x: |  spotbugs  |   1m 14s |  hbase-client generated 3 new + 0 unchanged - 0 fixed = 3 total (was 0)  |
   | -1 :x: |  spotbugs  |   2m 14s |  hbase-server generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -1 :x: |  spotbugs  |   9m 41s |  root generated 4 new + 0 unchanged - 0 fixed = 4 total (was 0)  |
   ||| _ Other Tests _ |
   | -1 :x: |  asflicense  |   1m 36s |  The patch generated 1 ASF License warnings.  |
   |  |   |  83m 27s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-client |
   |  |  Nullcheck of other at line 829 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.isOverlap(RegionInfo)  At RegionInfo.java:829 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.isOverlap(RegionInfo)  At RegionInfo.java:[line 828] |
   |  |  Nullcheck of rhs at line 109 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.lambda$static$0(RegionInfo, RegionInfo)  At RegionInfo.java:109 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.lambda$static$0(RegionInfo, RegionInfo)  At RegionInfo.java:[line 108] |
   |  |  Nullcheck of regionA at line 444 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.areAdjacent(RegionInfo, RegionInfo)  At RegionInfo.java:444 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.areAdjacent(RegionInfo, RegionInfo)  At RegionInfo.java:[line 443] |
   | FindBugs | module:hbase-server |
   |  |  Possible null pointer dereference of region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:[line 149] |
   | FindBugs | module:root |
   |  |  Nullcheck of other at line 829 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.isOverlap(RegionInfo)  At RegionInfo.java:829 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.isOverlap(RegionInfo)  At RegionInfo.java:[line 828] |
   |  |  Nullcheck of rhs at line 109 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.lambda$static$0(RegionInfo, RegionInfo)  At RegionInfo.java:109 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.lambda$static$0(RegionInfo, RegionInfo)  At RegionInfo.java:[line 108] |
   |  |  Nullcheck of regionA at line 444 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.areAdjacent(RegionInfo, RegionInfo)  At RegionInfo.java:444 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.areAdjacent(RegionInfo, RegionInfo)  At RegionInfo.java:[line 443] |
   |  |  Possible null pointer dereference of region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:[line 149] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2445 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle cc hbaseprotoc prototool |
   | uname | Linux dc508e71d578 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 687e53b7e4 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/1/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | whitespace | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/1/artifact/yetus-general-check/output/whitespace-eol.txt |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/1/artifact/yetus-general-check/output/new-spotbugs-hbase-client.html |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/1/artifact/yetus-general-check/output/new-spotbugs-hbase-server.html |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/1/artifact/yetus-general-check/output/new-spotbugs-root.html |
   | asflicense | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/1/artifact/yetus-general-check/output/patch-asflicense-problems.txt |
   | Max. process+thread count | 137 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-zookeeper hbase-balancer hbase-server hbase-backup . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506772765



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonRootRegionLocator.java
##########
@@ -174,10 +184,14 @@ private boolean tryComplete(LocateRequest req, CompletableFuture<RegionLocations
         // startKey < req.row and endKey >= req.row. Here we split it to endKey == req.row ||
         // (endKey > req.row && startKey < req.row). The two conditions are equal since startKey <
         // endKey.
+        KeyValue.KVComparator comparator = getComparator(loc.getRegion().getTable());
         byte[] endKey = loc.getRegion().getEndKey();
-        int c = Bytes.compareTo(endKey, req.row);
+        int c = comparator.compareRows(endKey, 0, endKey.length,
+          req.row,0, req.row.length);

Review comment:
       There are interfaces of cell and bytebuffer in CellComparator so I can use them if the keys are in the appropriate although in this case they are just byte arrays. Did this answer your question?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506774887



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
##########
@@ -33,6 +33,25 @@
   public static final RegionInfo UNDEFINED =
     RegionInfoBuilder.newBuilder(TableName.valueOf("__UNDEFINED__")).build();
 
+  /**
+   * RegionInfo for first root region
+   * You cannot use this builder to make an instance of the {@link #ROOT_REGIONINFO}.
+   * Just refer to this instance. Also, while the instance is actually a MutableRI, its type is
+   * just RI so the mutable methods are not available (unless you go casting); it appears
+   * as immutable (I tried adding Immutable type but it just makes a mess).
+   *
+   * We are using the non-legacy encoding format to reduce the boilerplace code

Review comment:
       It's basically saying we're not using the legacy regionId == 0 form or else we'll have to do the same ugliness we did for meta discussed [here](https://github.com/apache/hbase/pull/2445/files/483d02c8785be9f3884afa05116e0127f38fdd0d#r503639854).




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506766409



##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {

Review comment:
       Yes it does wanted to avoid the rename until I got almost everything working. Will do it.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#issuecomment-710736283


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  5s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +0 :ok: |  prototool  |   0m  1s |  prototool was not available.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 20s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  4s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 34s |  master passed  |
   | +1 :green_heart: |  spotbugs  |  19m 17s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 50s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   2m 34s |  root: The patch generated 76 new + 962 unchanged - 5 fixed = 1038 total (was 967)  |
   | -0 :warning: |  whitespace  |   0m  0s |  The patch has 1 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  hadoopcheck  |  18m 54s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1 3.3.0.  |
   | +1 :green_heart: |  hbaseprotoc  |   7m 28s |  the patch passed  |
   | -1 :x: |  spotbugs  |   1m 12s |  hbase-client generated 3 new + 0 unchanged - 0 fixed = 3 total (was 0)  |
   | -1 :x: |  spotbugs  |   2m 18s |  hbase-server generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -1 :x: |  spotbugs  |  10m 13s |  root generated 4 new + 0 unchanged - 0 fixed = 4 total (was 0)  |
   ||| _ Other Tests _ |
   | -1 :x: |  asflicense  |   1m 26s |  The patch generated 2 ASF License warnings.  |
   |  |   |  92m 28s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-client |
   |  |  Nullcheck of other at line 833 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.isOverlap(RegionInfo)  At RegionInfo.java:833 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.isOverlap(RegionInfo)  At RegionInfo.java:[line 832] |
   |  |  Nullcheck of rhs at line 113 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.lambda$static$0(RegionInfo, RegionInfo)  At RegionInfo.java:113 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.lambda$static$0(RegionInfo, RegionInfo)  At RegionInfo.java:[line 112] |
   |  |  Nullcheck of regionA at line 448 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.areAdjacent(RegionInfo, RegionInfo)  At RegionInfo.java:448 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.areAdjacent(RegionInfo, RegionInfo)  At RegionInfo.java:[line 447] |
   | FindBugs | module:hbase-server |
   |  |  Possible null pointer dereference of region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:[line 149] |
   | FindBugs | module:root |
   |  |  Nullcheck of other at line 833 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.isOverlap(RegionInfo)  At RegionInfo.java:833 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.isOverlap(RegionInfo)  At RegionInfo.java:[line 832] |
   |  |  Nullcheck of rhs at line 113 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.lambda$static$0(RegionInfo, RegionInfo)  At RegionInfo.java:113 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.lambda$static$0(RegionInfo, RegionInfo)  At RegionInfo.java:[line 112] |
   |  |  Nullcheck of regionA at line 448 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.areAdjacent(RegionInfo, RegionInfo)  At RegionInfo.java:448 of value previously dereferenced in org.apache.hadoop.hbase.client.RegionInfo.areAdjacent(RegionInfo, RegionInfo)  At RegionInfo.java:[line 447] |
   |  |  Possible null pointer dereference of region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:region in org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler.create(HRegionServer, String, long, boolean, ServerName)  Dereferenced at UnassignRegionHandler.java:[line 149] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2445 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle cc hbaseprotoc prototool |
   | uname | Linux bfcb56fb39ed 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e6c78f4037 |
   | checkstyle | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-general-check/output/diff-checkstyle-root.txt |
   | whitespace | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-general-check/output/whitespace-eol.txt |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-general-check/output/new-spotbugs-hbase-client.html |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-general-check/output/new-spotbugs-hbase-server.html |
   | spotbugs | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-general-check/output/new-spotbugs-root.html |
   | asflicense | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-general-check/output/patch-asflicense-problems.txt |
   | Max. process+thread count | 122 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-zookeeper hbase-balancer hbase-server hbase-backup . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack closed pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
saintstack closed pull request #2445:
URL: https://github.com/apache/hbase/pull/2445


   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506767036



##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.

Review comment:
       same as previous comment.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r503638276



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonRootRegionLocator.java
##########
@@ -174,10 +184,14 @@ private boolean tryComplete(LocateRequest req, CompletableFuture<RegionLocations
         // startKey < req.row and endKey >= req.row. Here we split it to endKey == req.row ||
         // (endKey > req.row && startKey < req.row). The two conditions are equal since startKey <
         // endKey.
+        KeyValue.KVComparator comparator = getComparator(loc.getRegion().getTable());
         byte[] endKey = loc.getRegion().getEndKey();
-        int c = Bytes.compareTo(endKey, req.row);
+        int c = comparator.compareRows(endKey, 0, endKey.length,
+          req.row,0, req.row.length);

Review comment:
       This works even if the Cell is ByteBuffer backed?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonRootRegionLocator.java
##########
@@ -68,9 +70,9 @@
  * The asynchronous locator for regions other than meta.
  */
 @InterfaceAudience.Private
-class AsyncNonMetaRegionLocator {
+class AsyncNonRootRegionLocator {

Review comment:
       This locates user-space and hbase:meta Regions?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
##########
@@ -2391,42 +2406,91 @@ public void run(Timeout timeout) throws Exception {
       return failedFuture(new IllegalArgumentException("Passed region name can't be null"));
     }
     try {
+      TableName parentTable;
       CompletableFuture<Optional<HRegionLocation>> future;
       if (RegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) {
         String encodedName = Bytes.toString(regionNameOrEncodedRegionName);
-        if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) {
-          // old format encodedName, should be meta region
+
+        //TODO francis do we really need to support encoded name for root?

Review comment:
       Dunno. I think the region id used to be 0. Could it be the encoded name too?
   
   The encoded name is usually md5 hash but old style Regions like hbase:meta used the original simple hash encoded number.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java
##########
@@ -206,16 +209,20 @@ public TableName getTable() {
    */
   @Override
   public boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey) {
-    if (Bytes.compareTo(rangeStartKey, rangeEndKey) > 0) {
+    CellComparator comparator = CellComparator.getComparator(tableName);

Review comment:
       Yeah, this getComparator is a dupe method? Can we have just one?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
##########
@@ -2391,42 +2406,91 @@ public void run(Timeout timeout) throws Exception {
       return failedFuture(new IllegalArgumentException("Passed region name can't be null"));
     }
     try {
+      TableName parentTable;
       CompletableFuture<Optional<HRegionLocation>> future;
       if (RegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) {
         String encodedName = Bytes.toString(regionNameOrEncodedRegionName);
-        if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) {
-          // old format encodedName, should be meta region
+
+        //TODO francis do we really need to support encoded name for root?
+        boolean isRoot = false;

Review comment:
       isRoot is name of the method that checks a boolean, not the name of variable.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonRootRegionLocator.java
##########
@@ -667,4 +689,14 @@ int getNumberOfCachedRegionLocations(TableName tableName) {
     }
     return tableCache.cache.values().stream().mapToInt(RegionLocations::numNonNullElements).sum();
   }
+
+  private static KeyValue.KVComparator getComparator(TableName tableName) {

Review comment:
       Is this the right place for this method?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java
##########
@@ -126,7 +127,9 @@ 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,
+      //1 is region id of FIRST_META_REGION_INFO
+      !(this.tableName.equals(TableName.META_TABLE_NAME) && regionId == 1));

Review comment:
       What is happening here? '1' is the id of the hardcoded single region hbase:meta table but is it going to be the region id when we split meta?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
##########
@@ -1130,22 +1143,24 @@ public void run(PRESP resp) {
    * List all region locations for the specific table.
    */
   private CompletableFuture<List<HRegionLocation>> getTableHRegionLocations(TableName tableName) {
-    if (TableName.META_TABLE_NAME.equals(tableName)) {
+    if (ROOT_TABLE_NAME.equals(tableName)) {
       CompletableFuture<List<HRegionLocation>> future = new CompletableFuture<>();
-      addListener(connection.registry.getMetaRegionLocations(), (metaRegions, err) -> {
+      addListener(connection.registry.getMetaRegionLocations(), (rootRegions, err) -> {
         if (err != null) {
           future.completeExceptionally(err);
-        } else if (metaRegions == null || metaRegions.isEmpty() ||
-          metaRegions.getDefaultRegionLocation() == null) {
+        } else if (rootRegions == null || rootRegions.isEmpty() ||
+          rootRegions.getDefaultRegionLocation() == null) {
           future.completeExceptionally(new IOException("meta region does not found"));
         } else {
-          future.complete(Collections.singletonList(metaRegions.getDefaultRegionLocation()));
+          future.complete(Collections.singletonList(rootRegions.getDefaultRegionLocation()));
         }
       });
       return future;
     } else {
-      // For non-meta table, we fetch all locations by scanning hbase:meta table
-      return ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName);
+      // For non-meta table, we fetch all locations by scanning catalog table

Review comment:
       Is comment right? Seems like we can get users-space tables and regions in meta too?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
##########
@@ -2391,42 +2406,91 @@ public void run(Timeout timeout) throws Exception {
       return failedFuture(new IllegalArgumentException("Passed region name can't be null"));
     }
     try {
+      TableName parentTable;
       CompletableFuture<Optional<HRegionLocation>> future;
       if (RegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) {
         String encodedName = Bytes.toString(regionNameOrEncodedRegionName);
-        if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) {
-          // old format encodedName, should be meta region
+
+        //TODO francis do we really need to support encoded name for root?
+        boolean isRoot = false;
+        for (int i = 0; i< numRootReplicas; i++) {
+          RegionInfo info =
+            RegionReplicaUtil.getRegionInfoForReplica(RegionInfoBuilder.ROOT_REGIONINFO, i);
+          if (Bytes.equals(info.getRegionName(), regionNameOrEncodedRegionName)) {
+            isRoot = true;
+            break;
+          }
+        }
+        if (isRoot) {
           future = connection.registry.getMetaRegionLocations()
             .thenApply(locs -> Stream.of(locs.getRegionLocations())
               .filter(loc -> loc.getRegion().getEncodedName().equals(encodedName)).findFirst());
+          parentTable = null;
+        } else if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) {
+          future = ClientMetaTableAccessor.getRegionLocationWithEncodedName(rootTable,
+            regionNameOrEncodedRegionName);
+          parentTable = ROOT_TABLE_NAME;
         } else {
           future = ClientMetaTableAccessor.getRegionLocationWithEncodedName(metaTable,
             regionNameOrEncodedRegionName);
+          parentTable = META_TABLE_NAME;
         }
       } else {
         RegionInfo regionInfo =
           CatalogFamilyFormat.parseRegionInfoFromRegionName(regionNameOrEncodedRegionName);
-        if (regionInfo.isMetaRegion()) {
+        if (regionInfo.isRootRegion()) {
           future = connection.registry.getMetaRegionLocations()
             .thenApply(locs -> Stream.of(locs.getRegionLocations())
               .filter(loc -> loc.getRegion().getReplicaId() == regionInfo.getReplicaId())
               .findFirst());
+          parentTable = null;
+          //TODO francis it won't reach here once meta is split
+        } else if (regionInfo.isMetaRegion())   {
+          parentTable = ROOT_TABLE_NAME;
+          future =
+            ClientMetaTableAccessor.getRegionLocation(rootTable, regionNameOrEncodedRegionName);
         } else {
+          parentTable = META_TABLE_NAME;
           future =
             ClientMetaTableAccessor.getRegionLocation(metaTable, regionNameOrEncodedRegionName);
         }
       }
 
+      final TableName finalParentTable = parentTable;
       CompletableFuture<HRegionLocation> returnedFuture = new CompletableFuture<>();
       addListener(future, (location, err) -> {
         if (err != null) {
           returnedFuture.completeExceptionally(err);
           return;
         }
         if (!location.isPresent() || location.get().getRegion() == null) {
-          returnedFuture.completeExceptionally(
-            new UnknownRegionException("Invalid region name or encoded region name: " +
-              Bytes.toStringBinary(regionNameOrEncodedRegionName)));
+          if (META_TABLE_NAME.equals(finalParentTable)) {
+            if (LOG.isDebugEnabled()) {

Review comment:
       No need of if LOG.isDebugEnabled... do paramaterized logging.... 
   "... root for region : {}"

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonRootRegionLocator.java
##########
@@ -174,10 +184,14 @@ private boolean tryComplete(LocateRequest req, CompletableFuture<RegionLocations
         // startKey < req.row and endKey >= req.row. Here we split it to endKey == req.row ||
         // (endKey > req.row && startKey < req.row). The two conditions are equal since startKey <
         // endKey.
+        KeyValue.KVComparator comparator = getComparator(loc.getRegion().getTable());
         byte[] endKey = loc.getRegion().getEndKey();
-        int c = Bytes.compareTo(endKey, req.row);
+        int c = comparator.compareRows(endKey, 0, endKey.length,
+          req.row,0, req.row.length);

Review comment:
       Will it never be ByteBuffer backed because it is endkey from RegionInfo?

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
##########
@@ -2391,42 +2406,91 @@ public void run(Timeout timeout) throws Exception {
       return failedFuture(new IllegalArgumentException("Passed region name can't be null"));
     }
     try {
+      TableName parentTable;
       CompletableFuture<Optional<HRegionLocation>> future;
       if (RegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) {
         String encodedName = Bytes.toString(regionNameOrEncodedRegionName);
-        if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) {
-          // old format encodedName, should be meta region
+
+        //TODO francis do we really need to support encoded name for root?
+        boolean isRoot = false;
+        for (int i = 0; i< numRootReplicas; i++) {
+          RegionInfo info =
+            RegionReplicaUtil.getRegionInfoForReplica(RegionInfoBuilder.ROOT_REGIONINFO, i);

Review comment:
       So, you support replicas in root?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506773341



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
##########
@@ -1130,22 +1143,24 @@ public void run(PRESP resp) {
    * List all region locations for the specific table.
    */
   private CompletableFuture<List<HRegionLocation>> getTableHRegionLocations(TableName tableName) {
-    if (TableName.META_TABLE_NAME.equals(tableName)) {
+    if (ROOT_TABLE_NAME.equals(tableName)) {
       CompletableFuture<List<HRegionLocation>> future = new CompletableFuture<>();
-      addListener(connection.registry.getMetaRegionLocations(), (metaRegions, err) -> {
+      addListener(connection.registry.getMetaRegionLocations(), (rootRegions, err) -> {
         if (err != null) {
           future.completeExceptionally(err);
-        } else if (metaRegions == null || metaRegions.isEmpty() ||
-          metaRegions.getDefaultRegionLocation() == null) {
+        } else if (rootRegions == null || rootRegions.isEmpty() ||
+          rootRegions.getDefaultRegionLocation() == null) {
           future.completeExceptionally(new IOException("meta region does not found"));
         } else {
-          future.complete(Collections.singletonList(metaRegions.getDefaultRegionLocation()));
+          future.complete(Collections.singletonList(rootRegions.getDefaultRegionLocation()));
         }
       });
       return future;
     } else {
-      // For non-meta table, we fetch all locations by scanning hbase:meta table
-      return ClientMetaTableAccessor.getTableHRegionLocations(metaTable, tableName);
+      // For non-meta table, we fetch all locations by scanning catalog table

Review comment:
       Yes this needs updating. 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506766297



##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.

Review comment:
       It should be for the catalog table. Apologies I haven't cleaned up the comments yet let me start doing that.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506766854



##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);

Review comment:
       It is scanMeta. I kept the old fullScanRegions() api to avoid having a lot of just renames all over the code. I'm hoping we can do the consolidation as a follow-up? To make this patch a bit more digestible. The same goes for scanMeta().
   
   




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506767100



##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>

Review comment:
       Will fix.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506769908



##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
+    throws IOException {
+    byte[] row = regionName;
+    RegionInfo parsedInfo = null;
+    try {
+      parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
+      row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);
+    } catch (Exception parseEx) {
+      return null;
+      // Ignore. This is used with tableName passed as regionName.
+    }
+    Get get = new Get(row);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    Result r;
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(parsedInfo.getTable()))) {
+      r = t.get(get);
+    }
+    RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+    return locations == null ? null :
+      locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionInfo region information
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
+    throws IOException {
+    return CatalogFamilyFormat.getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
+      regionInfo, regionInfo.getReplicaId());
+  }
+
+  /**
+   * @return Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta.
+   */
+  public static Result getCatalogFamilyRow(Connection connection, RegionInfo ri)
+    throws IOException {
+    Get get = new Get(CatalogFamilyFormat.getMetaKeyForRegion(ri));
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(ri.getTable()))) {
+      return t.get(get);
+    }
+  }
+
+  public static TableName getCatalogTableForTable(TableName tableName) {
+    if (TableName.ROOT_TABLE_NAME.equals(tableName)) {
+      throw new IllegalStateException("Can't get catalog table for hbase:root table");
+    }
+    if (TableName.META_TABLE_NAME.equals(tableName)) {
+      return TableName.ROOT_TABLE_NAME;
+    }
+    return TableName.META_TABLE_NAME;
+  }
+
+  public static boolean isCatalogTable(TableName tableName) {
+    return tableName.equals(TableName.ROOT_TABLE_NAME) ||
+      tableName.equals(TableName.META_TABLE_NAME);
+  }
+
+  /**
+   * Gets the result in hbase:meta for the specified region.
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return result of the specified region
+   */
+  public static Result getRegionResult(Connection connection,
+      byte[] regionName) throws IOException {
+    Table catalogTable = null;
+    if (Bytes.equals(RegionInfoBuilder.ROOT_REGIONINFO.getRegionName(), regionName)) {
+      throw new IllegalStateException("This method cannot be used for hbase:root region");
+    }
+    catalogTable = getCatalogHTable(connection,
+      getCatalogTableForTable(
+        CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName).getTable()));
+    Get get = new Get(regionName);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = catalogTable) {
+      return t.get(get);
+    }
+  }
+
+  /**
+   * Scans META table for a row whose key contains the specified <B>regionEncodedName</B>, returning
+   * a single related <code>Result</code> instance if any row is found, null otherwise.
+   * @param connection the connection to query META table.
+   * @param regionEncodedName the region encoded name to look for at META.
+   * @return <code>Result</code> instance with the row related info in META, null otherwise.
+   * @throws IOException if any errors occur while querying META.
+   */
+  public static Result scanByRegionEncodedName(Connection connection, String regionEncodedName)
+    throws IOException {
+    RowFilter rowFilter =
+      new RowFilter(CompareOperator.EQUAL, new SubstringComparator(regionEncodedName));
+    Scan scan = getCatalogScan(connection, 1);
+    scan.setFilter(rowFilter);
+    Result res = null;
+    try (ResultScanner resultScanner =
+      getCatalogHTable(connection, TableName.META_TABLE_NAME).getScanner(scan)) {
+      res = resultScanner.next();
+    }
+    if (res == null) {
+      scan = getCatalogScan(connection, 1);
+      scan.setFilter(rowFilter);
+      try (ResultScanner resultScanner =
+        getCatalogHTable(connection, TableName.ROOT_TABLE_NAME).getScanner(scan)) {
+        res = resultScanner.next();
+      }
+    }
+    return res;
+  }
+
+  /**
+   * Lists all of the regions currently in META.
+   * @param connection to connect with
+   * @param excludeOfflinedSplitParents False if we are to include offlined/splitparents regions,
+   *          true and we'll leave out offlined regions from returned list
+   * @return List of all user-space regions.
+   */
+  @VisibleForTesting
+  public static List<RegionInfo> getAllRegions(Connection connection,
+    boolean excludeOfflinedSplitParents) throws IOException {
+    List<Pair<RegionInfo, ServerName>> result;
+
+    result = getTableRegionsAndLocations(connection, null, excludeOfflinedSplitParents);
+
+    return getListOfRegionInfos(result);
+
+  }
+
+  /**
+   * Gets all of the regions of the specified table. Do not use this method to get meta table
+   * regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @return Ordered list of {@link RegionInfo}.
+   */
+  public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName)
+  throws IOException {
+    return getTableRegions(connection, tableName, false);
+  }
+
+  /**
+   * Gets all of the regions of the specified table. Do not use this method to get meta table
+   * regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @param excludeOfflinedSplitParents If true, do not include offlined split parents in the
+   *          return.
+   * @return Ordered list of {@link RegionInfo}.
+   */
+  public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName,
+    final boolean excludeOfflinedSplitParents) throws IOException {
+    List<Pair<RegionInfo, ServerName>> result =
+      getTableRegionsAndLocations(connection, tableName, excludeOfflinedSplitParents);
+    return getListOfRegionInfos(result);
+  }
+
+  private static List<RegionInfo>
+    getListOfRegionInfos(final List<Pair<RegionInfo, ServerName>> pairs) {
+    if (pairs == null || pairs.isEmpty()) {
+      return Collections.emptyList();
+    }
+    List<RegionInfo> result = new ArrayList<>(pairs.size());
+    for (Pair<RegionInfo, ServerName> pair : pairs) {
+      result.add(pair.getFirst());
+    }
+    return result;
+  }
+
+  /**
+   * This method creates a Scan object that will only scan catalog rows that belong to the specified
+   * table. It doesn't specify any columns. This is a better alternative to just using a start row
+   * and scan until it hits a new table since that requires parsing the HRI to get the table name.
+   * @param tableName bytes of table's name
+   * @return configured Scan object
+   * @deprecated This is internal so please remove it when we get a chance.
+   */
+  @Deprecated
+  public static Scan getScanForTableName(Connection connection, TableName tableName) {
+    // Start key is just the table name with delimiters
+    byte[] startKey = ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION);
+    // Stop key appends the smallest possible char to the table name
+    byte[] stopKey = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);
+
+    Scan scan = getCatalogScan(connection, -1);
+    scan.withStartRow(startKey);
+    scan.withStopRow(stopKey);
+    return scan;
+  }
+
+  private static Scan getCatalogScan(Connection connection, int rowUpperLimit) {
+    Scan scan = new Scan();
+    int scannerCaching = connection.getConfiguration().getInt(HConstants.HBASE_META_SCANNER_CACHING,
+      HConstants.DEFAULT_HBASE_META_SCANNER_CACHING);
+    if (connection.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS,
+      HConstants.DEFAULT_USE_META_REPLICAS)) {
+      scan.setConsistency(Consistency.TIMELINE);
+    }
+    if (rowUpperLimit > 0) {
+      scan.setLimit(rowUpperLimit);
+      scan.setReadType(Scan.ReadType.PREAD);
+    }
+    scan.setCaching(scannerCaching);
+    return scan;
+  }
+
+  /**
+   * Do not use this method to get meta table regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @return Return list of regioninfos and server.
+   */
+  public static List<Pair<RegionInfo, ServerName>>
+    getTableRegionsAndLocations(Connection connection, TableName tableName) throws IOException {
+    return getTableRegionsAndLocations(connection, tableName, true);
+  }
+
+  /**
+   * Do not use this method to get meta table regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table to work with, can be null for getting all regions
+   * @param excludeOfflinedSplitParents don't return split parents
+   * @return Return list of regioninfos and server addresses.
+   */
+  // What happens here when 1M regions in hbase:meta? This won't scale?
+  public static List<Pair<RegionInfo, ServerName>> getTableRegionsAndLocations(
+    Connection connection, @Nullable final TableName tableName,
+    final boolean excludeOfflinedSplitParents) throws IOException {
+    if (tableName != null && tableName.equals(TableName.ROOT_TABLE_NAME)) {
+      throw new IOException(
+        "This method can't be used to locate root regions; use RootTableLocator instead");
+    }
+    // Make a version of CollectingVisitor that collects RegionInfo and ServerAddress
+    ClientMetaTableAccessor.CollectRegionLocationsVisitor visitor =
+      new ClientMetaTableAccessor.CollectRegionLocationsVisitor(excludeOfflinedSplitParents);
+    byte[] startRow = ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION);
+    byte[] stopRow = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);
+    TableName parentTable = TableName.META_TABLE_NAME;
+    if (TableName.META_TABLE_NAME.equals(tableName)) {
+      parentTable = TableName.ROOT_TABLE_NAME;
+      startRow = null;
+      stopRow = null;
+    }
+
+    scanCatalog(connection,
+      parentTable,
+      startRow,
+      stopRow,
+      QueryType.REGION,
+      Integer.MAX_VALUE,
+      visitor);
+    return visitor.getResults();
+  }
+
+  public static void fullScanMetaAndPrint(Connection connection) throws IOException {
+    ClientMetaTableAccessor.Visitor v = r -> {
+      if (r == null || r.isEmpty()) {
+        return true;
+      }
+      LOG.info("fullScanMetaAndPrint.Current Meta Row: " + r);
+      TableState state = CatalogFamilyFormat.getTableState(r);
+      if (state != null) {
+        LOG.info("fullScanMetaAndPrint.Table State={}" + state);
+      } else {
+        RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+        if (locations == null) {
+          return true;
+        }
+        for (HRegionLocation loc : locations.getRegionLocations()) {
+          if (loc != null) {
+            LOG.info("fullScanMetaAndPrint.HRI Print={}", loc.getRegion());
+          }
+        }
+      }
+      return true;
+    };
+    scanMeta(connection, null, null, QueryType.ALL, v);
+  }
+
+  public static void scanMetaForTableRegions(Connection connection,
+    ClientMetaTableAccessor.Visitor visitor, TableName tableName) throws IOException {
+    scanCatalogForTableRegions(connection, QueryType.REGION, Integer.MAX_VALUE, visitor, tableName);
+  }
+
+  public static void scanCatalogForTableRegions(Connection connection, QueryType type, int maxRows,
+    final ClientMetaTableAccessor.Visitor visitor, TableName table) throws IOException {
+    scanCatalog(connection, getCatalogTableForTable(table),
+      ClientMetaTableAccessor.getTableStartRowForMeta(table, type),
+      ClientMetaTableAccessor.getTableStopRowForMeta(table, type),
+      type, maxRows, visitor);
+  }
+
+  public static void scanMeta(Connection connection, @Nullable final byte[] startRow,
+    @Nullable final byte[] stopRow, QueryType type, final ClientMetaTableAccessor.Visitor visitor)
+    throws IOException {
+    scanCatalog(connection, TableName.META_TABLE_NAME, startRow, stopRow, type, Integer.MAX_VALUE,
+        visitor);
+  }
+
+  /**
+   * Performs a scan of META table for given table starting from given row.
+   * @param connection connection we're using
+   * @param visitor visitor to call
+   * @param tableName table withing we scan
+   * @param row start scan from this row
+   * @param rowLimit max number of rows to return
+   */
+  public static void scanMeta(Connection connection, final ClientMetaTableAccessor.Visitor visitor,
+    final TableName tableName, final byte[] row, final int rowLimit) throws IOException {
+    byte[] startRow = null;
+    byte[] stopRow = null;
+    if (tableName != null) {
+      startRow = ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION);
+      if (row != null) {
+        RegionInfo closestRi = getClosestRegionInfo(connection, tableName, row);
+        startRow =
+          RegionInfo.createRegionName(tableName, closestRi.getStartKey(), HConstants.ZEROES, false);
+      }
+      stopRow = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);
+    }
+    scanCatalog(connection, TableName.META_TABLE_NAME, startRow, stopRow, QueryType.REGION,
+        rowLimit, visitor);
+  }
+
+  /**
+   * Performs a scan of META table.
+   * @param connection connection we're using
+   * @param startRow Where to start the scan. Pass null if want to begin scan at first row.
+   * @param stopRow Where to stop the scan. Pass null if want to scan all rows from the start one
+   * @param type scanned part of meta
+   * @param maxRows maximum rows to return
+   * @param visitor Visitor invoked against each row.
+   */
+  public static void scanCatalog(Connection connection, TableName catalogTable,
+      @Nullable final byte[] startRow, @Nullable final byte[] stopRow, QueryType type, int maxRows,
+      final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTable, startRow, stopRow, type, null, maxRows, visitor);
+  }
+
+  public static void scanCatalog(Connection connection, TableName catalogTableName,
+    @Nullable final byte[] startRow, @Nullable final byte[] stopRow,
+    QueryType type, @Nullable Filter filter, int maxRows,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE;
+    Scan scan = getCatalogScan(connection, rowUpperLimit);
+
+    for (byte[] family : type.getFamilies()) {
+      scan.addFamily(family);
+    }
+    if (startRow != null) {
+      scan.withStartRow(startRow);
+    }
+    if (stopRow != null) {
+      scan.withStopRow(stopRow);
+    }
+    if (filter != null) {
+      scan.setFilter(filter);
+    }
+
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Scanning META" + " starting at row=" + Bytes.toStringBinary(startRow) +
+        " stopping at row=" + Bytes.toStringBinary(stopRow) + " for max=" + rowUpperLimit +
+        " with caching=" + scan.getCaching());
+    }
+
+    int currentRow = 0;
+    try (Table catalogTable = getCatalogHTable(connection, catalogTableName)) {
+      try (ResultScanner scanner = catalogTable.getScanner(scan)) {
+        Result data;
+        while ((data = scanner.next()) != null) {
+          if (data.isEmpty()) {
+            continue;
+          }
+          // Break if visit returns false.
+          if (!visitor.visit(data)) {
+            break;
+          }
+          if (++currentRow >= rowUpperLimit) {
+            break;
+          }
+        }
+      }
+    }
+    if (visitor instanceof Closeable) {
+      try {
+        ((Closeable) visitor).close();
+      } catch (Throwable t) {
+        ExceptionUtil.rethrowIfInterrupt(t);
+        LOG.debug("Got exception in closing the meta scanner visitor", t);
+      }
+    }
+  }
+
+  /**
+   * @return Get closest metatable region row to passed <code>row</code>
+   */
+  @NonNull
+  private static RegionInfo getClosestRegionInfo(Connection connection,
+    @NonNull final TableName tableName, @NonNull final byte[] row) throws IOException {
+    byte[] searchRow = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
+    Scan scan = getCatalogScan(connection, 1);
+    scan.setReversed(true);
+    scan.withStartRow(searchRow);
+    try (ResultScanner resultScanner =
+        getCatalogHTable(connection, getCatalogTableForTable(tableName)).getScanner(scan)) {
+      Result result = resultScanner.next();
+      if (result == null) {
+        throw new TableNotFoundException("Cannot find row in META " + " for table: " + tableName +
+          ", row=" + Bytes.toStringBinary(row));
+      }
+      RegionInfo regionInfo = CatalogFamilyFormat.getRegionInfo(result);
+      if (regionInfo == null) {
+        throw new IOException("RegionInfo was null or empty in Meta for " + tableName + ", row=" +
+          Bytes.toStringBinary(row));
+      }
+      return regionInfo;
+    }
+  }
+
+  /**
+   * Returns the {@link ServerName} from catalog table {@link Result} where the region is
+   * transitioning on. It should be the same as
+   * {@link CatalogFamilyFormat#getServerName(Result,int)} if the server is at OPEN state.
+   * @param r Result to pull the transitioning server name from
+   * @return A ServerName instance or {@link CatalogFamilyFormat#getServerName(Result,int)} if
+   *         necessary fields not found or empty.
+   */
+  @Nullable
+  public static ServerName getTargetServerName(final Result r, final int replicaId) {
+    final Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY,
+      CatalogFamilyFormat.getServerNameColumn(replicaId));
+    if (cell == null || cell.getValueLength() == 0) {
+      RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+      if (locations != null) {
+        HRegionLocation location = locations.getRegionLocation(replicaId);
+        if (location != null) {
+          return location.getServerName();
+        }
+      }
+      return null;
+    }
+    return ServerName.parseServerName(
+      Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
+  }
+
+  /**
+   * Returns the daughter regions by reading the corresponding columns of the catalog table Result.
+   * @param data a Result object from the catalog table scan
+   * @return pair of RegionInfo or PairOfSameType(null, null) if region is not a split parent
+   */
+  public static PairOfSameType<RegionInfo> getDaughterRegions(Result data) {
+    RegionInfo splitA = CatalogFamilyFormat.getRegionInfo(data, HConstants.SPLITA_QUALIFIER);
+    RegionInfo splitB = CatalogFamilyFormat.getRegionInfo(data, HConstants.SPLITB_QUALIFIER);
+    return new PairOfSameType<>(splitA, splitB);
+  }
+
+  /**
+   * Fetch table state for given table from META table
+   * @param conn connection to use
+   * @param tableName table to fetch state for
+   */
+  @Nullable
+  public static TableState getTableState(Connection conn, TableName tableName)
+    throws IOException {
+    if (tableName.equals(TableName.ROOT_TABLE_NAME) ||
+      tableName.equals(TableName.META_TABLE_NAME)) {
+      return new TableState(tableName, TableState.State.ENABLED);
+    }
+    Table metaHTable = getCatalogHTable(conn, TableName.META_TABLE_NAME);
+    Get get = new Get(tableName.getName()).addColumn(HConstants.TABLE_FAMILY,
+      HConstants.TABLE_STATE_QUALIFIER);
+    Result result = metaHTable.get(get);
+    return CatalogFamilyFormat.getTableState(result);
+  }
+
+  /**
+   * Fetch table states from META table
+   * @param conn connection to use
+   * @return map {tableName -&gt; state}
+   */
+  public static Map<TableName, TableState> getTableStates(Connection conn) throws IOException {
+    final Map<TableName, TableState> states = new LinkedHashMap<>();
+    ClientMetaTableAccessor.Visitor collector = r -> {
+      TableState state = CatalogFamilyFormat.getTableState(r);
+      if (state != null) {
+        states.put(state.getTableName(), state);
+      }
+      return true;
+    };
+    fullScanTables(conn, collector);
+    return states;
+  }
+
+  /**
+   * Updates state in META Do not use. For internal use only.
+   * @param conn connection to use
+   * @param tableName table to look for
+   */
+  public static void updateTableState(Connection conn, TableName tableName, TableState.State actual)
+    throws IOException {
+    updateTableState(conn, new TableState(tableName, actual));
+  }
+
+  ////////////////////////
+  // Editing operations //
+  ////////////////////////
+  /**
+   * Generates and returns a Put containing the region into for the catalog table
+   */
+  public static Put makePutFromRegionInfo(RegionInfo regionInfo, long ts) throws IOException {
+    return addRegionInfo(new Put(regionInfo.getRegionName(), ts), regionInfo);
+  }
+
+  /**
+   * Generates and returns a Delete containing the region info for the catalog table
+   */
+  public static Delete makeDeleteFromRegionInfo(RegionInfo regionInfo, long ts) {
+    if (regionInfo == null) {
+      throw new IllegalArgumentException("Can't make a delete for null region");
+    }
+    Delete delete = new Delete(regionInfo.getRegionName());
+    delete.addFamily(HConstants.CATALOG_FAMILY, ts);
+    return delete;
+  }
+
+  /**
+   * Adds split daughters to the Put
+   */
+  public static Put addDaughtersToPut(Put put, RegionInfo splitA, RegionInfo splitB)
+    throws IOException {
+    if (splitA != null) {
+      put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
+        .setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.SPLITA_QUALIFIER)
+        .setTimestamp(put.getTimestamp()).setType(Type.Put).setValue(RegionInfo.toByteArray(splitA))
+        .build());
+    }
+    if (splitB != null) {
+      put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
+        .setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.SPLITB_QUALIFIER)
+        .setTimestamp(put.getTimestamp()).setType(Type.Put).setValue(RegionInfo.toByteArray(splitB))
+        .build());
+    }
+    return put;
+  }
+
+  /**
+   * Put the passed <code>p</code> to the <code>hbase:meta</code> table.
+   * @param connection connection we're using
+   * @param p Put to add to hbase:meta
+   */
+  private static void putToMetaTable(Connection connection, Put p) throws IOException {
+    try (Table table = getCatalogHTable(connection, TableName.META_TABLE_NAME)) {
+      put(table, p);
+    }
+  }
+
+  /**
+   * @param t Table to use
+   * @param p put to make
+   */
+  private static void put(Table t, Put p) throws IOException {
+    debugLogMutation(t.getName(), p);
+    t.put(p);
+  }
+
+  /**
+   * Put the passed <code>ps</code> to the <code>hbase:meta</code> table.
+   * @param connection connection we're using
+   * @param ps Put to add to hbase:meta
+   */
+  public static void putsToCatalogTable(final Connection connection, TableName tableName,
+    final List<Put> ps)
+      throws IOException {
+    if (ps.isEmpty()) {
+      return;
+    }
+    try (Table t = getCatalogHTable(connection, tableName)) {
+      debugLogMutations(t.getName(), ps);
+      // the implementation for putting a single Put is much simpler so here we do a check first.
+      if (ps.size() == 1) {
+        t.put(ps.get(0));
+      } else {
+        t.put(ps);
+      }
+    }
+  }
+
+  /**
+   * Delete the passed <code>d</code> from the <code>hbase:meta</code> table.
+   * @param connection connection we're using
+   * @param d Delete to add to hbase:meta
+   */
+  private static void deleteFromCatalogTable(final Connection connection, TableName catalogTable,
+    final Delete d) throws IOException {
+    List<Delete> dels = new ArrayList<>(1);
+    dels.add(d);
+    deleteFromCatalogTable(connection, catalogTable, dels);
+  }
+
+  /**
+   * Delete the passed <code>deletes</code> from the <code>hbase:meta</code> table.
+   * @param connection connection we're using
+   * @param deletes Deletes to add to hbase:meta  This list should support #remove.
+   */
+  private static void deleteFromCatalogTable(final Connection connection,
+    TableName catalogTableName, final List<Delete> deletes) throws IOException {
+    try (Table t = getCatalogHTable(connection, catalogTableName)) {
+      debugLogMutations(catalogTableName, deletes);
+      t.delete(deletes);
+    }
+  }
+
+  /**
+   * Deletes some replica columns corresponding to replicas for the passed rows
+   * @param metaRows rows in hbase:meta
+   * @param replicaIndexToDeleteFrom the replica ID we would start deleting from
+   * @param numReplicasToRemove how many replicas to remove
+   * @param connection connection we're using to access meta table
+   */
+  public static void removeRegionReplicasFromCatalog(Set<byte[]> metaRows,
+    int replicaIndexToDeleteFrom, int numReplicasToRemove, Connection connection,
+    TableName catalogTableName) throws IOException {
+    int absoluteIndex = replicaIndexToDeleteFrom + numReplicasToRemove;
+    for (byte[] row : metaRows) {
+      long now = EnvironmentEdgeManager.currentTime();
+      Delete deleteReplicaLocations = new Delete(row);
+      for (int i = replicaIndexToDeleteFrom; i < absoluteIndex; i++) {
+        deleteReplicaLocations.addColumns(HConstants.CATALOG_FAMILY,
+          CatalogFamilyFormat.getServerColumn(i), now);
+        deleteReplicaLocations.addColumns(HConstants.CATALOG_FAMILY,
+          CatalogFamilyFormat.getSeqNumColumn(i), now);
+        deleteReplicaLocations.addColumns(HConstants.CATALOG_FAMILY,
+          CatalogFamilyFormat.getStartCodeColumn(i), now);
+        deleteReplicaLocations.addColumns(HConstants.CATALOG_FAMILY,
+          CatalogFamilyFormat.getServerNameColumn(i), now);
+        deleteReplicaLocations.addColumns(HConstants.CATALOG_FAMILY,
+          CatalogFamilyFormat.getRegionStateColumn(i), now);
+      }
+
+      deleteFromCatalogTable(connection, catalogTableName, deleteReplicaLocations);
+    }
+  }
+
+  public static Put addRegionStateToPut(Put put, RegionState.State state) throws IOException {
+    put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
+      .setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.STATE_QUALIFIER)
+      .setTimestamp(put.getTimestamp()).setType(Cell.Type.Put).setValue(Bytes.toBytes(state.name()))
+      .build());
+    return put;
+  }
+
+  /**
+   * Update state column in hbase:meta.
+   */
+  public static void updateRegionState(Connection connection, RegionInfo ri,
+    RegionState.State state) throws IOException {
+    Put put = new Put(RegionReplicaUtil.getRegionInfoForDefaultReplica(ri).getRegionName());
+    putsToCatalogTable(connection, getCatalogTableForTable(ri.getTable()),
+        Collections.singletonList(addRegionStateToPut(put, state)));
+  }
+
+  /**
+   * Adds daughter region infos to hbase:meta row for the specified region. Note that this does not
+   * add its daughter's as different rows, but adds information about the daughters in the same row
+   * as the parent. Use
+   * {@link #splitRegion(Connection, RegionInfo, long, RegionInfo, RegionInfo, ServerName, int)} if
+   * you want to do that.
+   * @param connection connection we're using
+   * @param regionInfo RegionInfo of parent region
+   * @param splitA first split daughter of the parent regionInfo
+   * @param splitB second split daughter of the parent regionInfo
+   * @throws IOException if problem connecting or updating meta
+   */
+  public static void addSplitsToParent(Connection connection, RegionInfo regionInfo,
+      RegionInfo splitA, RegionInfo splitB) throws IOException {
+    TableName catalogTable = getCatalogTableForTable(regionInfo.getTable());
+    try (Table catalog = getCatalogHTable(connection, catalogTable)) {
+      Put put = makePutFromRegionInfo(regionInfo, EnvironmentEdgeManager.currentTime());
+      addDaughtersToPut(put, splitA, splitB);
+      catalog.put(put);
+      debugLogMutation(catalogTable, put);
+      LOG.debug("Added region {}", regionInfo.getRegionNameAsString());
+    }
+  }
+
+  /**
+   * Adds a (single) hbase:meta row for the specified new region and its daughters. Note that this
+   * does not add its daughter's as different rows, but adds information about the daughters in the
+   * same row as the parent. Use
+   * {@link #splitRegion(Connection, RegionInfo, long, RegionInfo, RegionInfo, ServerName, int)} if
+   * you want to do that.
+   * @param connection connection we're using
+   * @param regionInfo region information
+   * @throws IOException if problem connecting or updating meta
+   */
+  @VisibleForTesting
+  public static void addRegionToMeta(Connection connection, RegionInfo regionInfo)
+    throws IOException {
+    addRegionsToMeta(connection, Collections.singletonList(regionInfo), 1);
+  }
+
+  /**
+   * Adds a hbase:meta row for each of the specified new regions. Initial state for new regions is
+   * CLOSED.
+   * @param connection connection we're using
+   * @param regionInfos region information list
+   * @throws IOException if problem connecting or updating meta
+   */
+  public static void addRegionsToMeta(Connection connection, List<RegionInfo> regionInfos,
+    int regionReplication) throws IOException {
+    addRegionsToMeta(connection, regionInfos, regionReplication,
+      EnvironmentEdgeManager.currentTime());
+  }
+
+  /**
+   * Adds a hbase:meta row for each of the specified new regions. Initial state for new regions is
+   * CLOSED.
+   * @param connection connection we're using
+   * @param regionInfos region information list
+   * @param ts desired timestamp
+   * @throws IOException if problem connecting or updating meta
+   */
+  public static void addRegionsToMeta(Connection connection, List<RegionInfo> regionInfos,
+    int regionReplication, long ts) throws IOException {
+    List<Put> puts = new ArrayList<>();
+    for (RegionInfo regionInfo : regionInfos) {
+      if (RegionReplicaUtil.isDefaultReplica(regionInfo)) {
+        Put put = makePutFromRegionInfo(regionInfo, ts);
+        // New regions are added with initial state of CLOSED.
+        addRegionStateToPut(put, RegionState.State.CLOSED);
+        // Add empty locations for region replicas so that number of replicas can be cached
+        // whenever the primary region is looked up from meta
+        for (int i = 1; i < regionReplication; i++) {
+          addEmptyLocation(put, i);
+        }
+        puts.add(put);
+      }
+    }
+    putsToCatalogTable(connection, TableName.META_TABLE_NAME, puts);
+    LOG.info("Added {} regions to {}.",puts.size(), TableName.META_TABLE_NAME);
+  }
+
+  /**
+   * Update state of the table in meta.
+   * @param connection what we use for update
+   * @param state new state
+   */
+  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);
+  }
+
+  /**
+   * Construct PUT for given state
+   * @param state new state
+   */
+  public static Put makePutFromTableState(TableState state, long ts) {
+    Put put = new Put(state.getTableName().getName(), ts);
+    put.addColumn(HConstants.TABLE_FAMILY, HConstants.TABLE_STATE_QUALIFIER,
+      state.convert().toByteArray());
+    return put;
+  }
+
+  /**
+   * Remove state for table from meta
+   * @param connection to use for deletion
+   * @param table to delete state for
+   */
+  public static void deleteTableState(Connection connection, TableName table) throws IOException {
+    long time = EnvironmentEdgeManager.currentTime();
+    Delete delete = new Delete(table.getName());
+    delete.addColumns(HConstants.TABLE_FAMILY, HConstants.TABLE_STATE_QUALIFIER, time);
+    deleteFromCatalogTable(connection, TableName.META_TABLE_NAME, delete);
+    LOG.info("Deleted table " + table + " state from Catalog");
+  }
+
+  /**
+   * Updates the location of the specified region in hbase:meta to be the specified server hostname
+   * and startcode.
+   * <p>
+   * Uses passed catalog tracker to get a connection to the server hosting hbase:meta and makes
+   * edits to that region.
+   * @param connection connection we're using
+   * @param regionInfo region to update location of
+   * @param openSeqNum the latest sequence number obtained when the region was open
+   * @param sn Server name
+   * @param masterSystemTime wall clock time from master if passed in the open region RPC
+   */
+  @VisibleForTesting
+  public static void updateRegionLocation(Connection connection, RegionInfo regionInfo,
+    ServerName sn, long openSeqNum, long masterSystemTime) throws IOException {
+    updateLocation(connection, regionInfo, sn, openSeqNum, masterSystemTime);
+  }
+
+  /**
+   * Updates the location of the specified region to be the specified server.
+   * <p>
+   * Connects to the specified server which should be hosting the specified catalog region name to
+   * perform the edit.
+   * @param connection connection we're using
+   * @param regionInfo region to update location of
+   * @param sn Server name
+   * @param openSeqNum the latest sequence number obtained when the region was open
+   * @param masterSystemTime wall clock time from master if passed in the open region RPC
+   * @throws IOException In particular could throw {@link java.net.ConnectException} if the server
+   *           is down on other end.
+   */
+  private static void updateLocation(Connection connection, RegionInfo regionInfo, ServerName sn,
+    long openSeqNum, long masterSystemTime) throws IOException {
+    // region replicas are kept in the primary region's row
+    Put put = new Put(CatalogFamilyFormat.getMetaKeyForRegion(regionInfo), masterSystemTime);
+    addRegionInfo(put, regionInfo);
+    addLocation(put, sn, openSeqNum, regionInfo.getReplicaId());
+    putToMetaTable(connection, put);
+    LOG.info("Updated row {} with server=", regionInfo.getRegionNameAsString(), sn);
+  }
+
+  public static Put addRegionInfo(final Put p, final RegionInfo hri) throws IOException {
+    p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(p.getRow())
+      .setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.REGIONINFO_QUALIFIER)
+      .setTimestamp(p.getTimestamp()).setType(Type.Put)
+      // Serialize the Default Replica HRI otherwise scan of hbase:meta
+      // shows an info:regioninfo value with encoded name and region
+      // name that differs from that of the hbase;meta row.
+      .setValue(RegionInfo.toByteArray(RegionReplicaUtil.getRegionInfoForDefaultReplica(hri)))
+      .build());
+    return p;
+  }
+
+  public static Put addLocation(Put p, ServerName sn, long openSeqNum, int replicaId)
+    throws IOException {
+    CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
+    return p
+      .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)
+        .setQualifier(CatalogFamilyFormat.getServerColumn(replicaId)).setTimestamp(p.getTimestamp())
+        .setType(Cell.Type.Put).setValue(Bytes.toBytes(sn.getAddress().toString())).build())
+      .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)
+        .setQualifier(CatalogFamilyFormat.getStartCodeColumn(replicaId))
+        .setTimestamp(p.getTimestamp()).setType(Cell.Type.Put)
+        .setValue(Bytes.toBytes(sn.getStartcode())).build())
+      .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)
+        .setQualifier(CatalogFamilyFormat.getSeqNumColumn(replicaId)).setTimestamp(p.getTimestamp())
+        .setType(Type.Put).setValue(Bytes.toBytes(openSeqNum)).build());
+  }
+
+  public static Put addEmptyLocation(Put p, int replicaId) throws IOException {
+    CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
+    return p
+      .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)
+        .setQualifier(CatalogFamilyFormat.getServerColumn(replicaId)).setTimestamp(p.getTimestamp())
+        .setType(Type.Put).build())
+      .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)
+        .setQualifier(CatalogFamilyFormat.getStartCodeColumn(replicaId))
+        .setTimestamp(p.getTimestamp()).setType(Cell.Type.Put).build())
+      .add(builder.clear().setRow(p.getRow()).setFamily(HConstants.CATALOG_FAMILY)
+        .setQualifier(CatalogFamilyFormat.getSeqNumColumn(replicaId)).setTimestamp(p.getTimestamp())
+        .setType(Cell.Type.Put).build());
+  }
+
+  private static void debugLogMutations(TableName tableName, List<? extends Mutation> mutations)
+    throws IOException {
+    if (!CATALOGLOG.isDebugEnabled()) {
+      return;
+    }
+    // Logging each mutation in separate line makes it easier to see diff between them visually
+    // because of common starting indentation.
+    for (Mutation mutation : mutations) {
+      debugLogMutation(tableName, mutation);
+    }
+  }
+
+  private static void debugLogMutation(TableName t, Mutation p) throws IOException {
+    CATALOGLOG.debug("{} {} {}", t, p.getClass().getSimpleName(), p.toJSON());
+  }
+}

Review comment:
       Will do thanks. I think might just move all these apis into MetaTableAccessor and make CatalogAccessor a facade. So the diffs on existing methods can be differentiated.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506768145



##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
+    throws IOException {
+    byte[] row = regionName;
+    RegionInfo parsedInfo = null;
+    try {
+      parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
+      row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);
+    } catch (Exception parseEx) {
+      return null;
+      // Ignore. This is used with tableName passed as regionName.
+    }
+    Get get = new Get(row);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    Result r;
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(parsedInfo.getTable()))) {
+      r = t.get(get);
+    }
+    RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+    return locations == null ? null :
+      locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionInfo region information
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
+    throws IOException {
+    return CatalogFamilyFormat.getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
+      regionInfo, regionInfo.getReplicaId());
+  }
+
+  /**
+   * @return Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta.
+   */
+  public static Result getCatalogFamilyRow(Connection connection, RegionInfo ri)
+    throws IOException {
+    Get get = new Get(CatalogFamilyFormat.getMetaKeyForRegion(ri));
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(ri.getTable()))) {
+      return t.get(get);
+    }
+  }
+
+  public static TableName getCatalogTableForTable(TableName tableName) {
+    if (TableName.ROOT_TABLE_NAME.equals(tableName)) {
+      throw new IllegalStateException("Can't get catalog table for hbase:root table");
+    }
+    if (TableName.META_TABLE_NAME.equals(tableName)) {
+      return TableName.ROOT_TABLE_NAME;
+    }
+    return TableName.META_TABLE_NAME;
+  }
+
+  public static boolean isCatalogTable(TableName tableName) {
+    return tableName.equals(TableName.ROOT_TABLE_NAME) ||
+      tableName.equals(TableName.META_TABLE_NAME);
+  }
+
+  /**
+   * Gets the result in hbase:meta for the specified region.
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return result of the specified region
+   */
+  public static Result getRegionResult(Connection connection,
+      byte[] regionName) throws IOException {
+    Table catalogTable = null;
+    if (Bytes.equals(RegionInfoBuilder.ROOT_REGIONINFO.getRegionName(), regionName)) {
+      throw new IllegalStateException("This method cannot be used for hbase:root region");
+    }
+    catalogTable = getCatalogHTable(connection,

Review comment:
       Will do. Looks like I just need to move this assignment right before the succeed try-with-resources block.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506774233



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
##########
@@ -33,6 +33,25 @@
   public static final RegionInfo UNDEFINED =
     RegionInfoBuilder.newBuilder(TableName.valueOf("__UNDEFINED__")).build();
 
+  /**
+   * RegionInfo for first root region

Review comment:
       Ah right I can remove. I was thinking first as there are replicas but that's not really it. 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506777664



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -152,9 +159,19 @@
   private static final int DEFAULT_RIT_STUCK_WARNING_THRESHOLD = 60 * 1000;
   public static final String UNEXPECTED_STATE_REGION = "Unexpected state for ";
 
-  private final ProcedureEvent<?> metaAssignEvent = new ProcedureEvent<>("meta assign");
+  private final ProcedureEvent<?> rootAssignEvent = new ProcedureEvent<>("root assign");
+  private final ProcedureEvent<?> rootLoadEvent = new ProcedureEvent<>("root load");
+
   private final ProcedureEvent<?> metaLoadEvent = new ProcedureEvent<>("meta load");
 
+  private final ConcurrentSkipListMap<byte[], ProcedureEvent<?>> metaAssignEventMap =

Review comment:
       This is here so procedures that depend on particular regions of meta to be available (eg RTSP, ORP, CRP, etc) are only waiting on the necessary region(s).




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506771434



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonRootRegionLocator.java
##########
@@ -667,4 +689,14 @@ int getNumberOfCachedRegionLocations(TableName tableName) {
     }
     return tableCache.cache.values().stream().mapToInt(RegionLocations::numNonNullElements).sum();
   }
+
+  private static KeyValue.KVComparator getComparator(TableName tableName) {

Review comment:
       Ah no this is an artifact of PoC just to get things working. I will consolidate this into CellComparator as well as remove usage of KVComparator.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
saintstack commented on pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#issuecomment-799847038


   Closing out draft.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506770199



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllRootRegionsOnlineException.java
##########
@@ -25,19 +25,19 @@
  * Thrown when an operation requires the root and all meta regions to be online
  */
 @InterfaceAudience.Public
-public class NotAllMetaRegionsOnlineException extends DoNotRetryIOException {
+public class NotAllRootRegionsOnlineException extends DoNotRetryIOException {

Review comment:
       Yes good point. 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506778454



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -768,6 +944,15 @@ public void move(RegionInfo regionInfo) throws IOException {
 
   @VisibleForTesting
   static int compare(TransitRegionStateProcedure left, TransitRegionStateProcedure right) {
+    //TODO francis this is broken once we have more meta entries

Review comment:
       Yes.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506773828



##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
##########
@@ -2391,42 +2406,91 @@ public void run(Timeout timeout) throws Exception {
       return failedFuture(new IllegalArgumentException("Passed region name can't be null"));
     }
     try {
+      TableName parentTable;
       CompletableFuture<Optional<HRegionLocation>> future;
       if (RegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) {
         String encodedName = Bytes.toString(regionNameOrEncodedRegionName);
-        if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) {
-          // old format encodedName, should be meta region
+
+        //TODO francis do we really need to support encoded name for root?

Review comment:
       Oh this question was whether we wanted to support the encoded name or root as argument for "regionNameOrEncodedRegionName"? Becuase then I have to do the succeeding not so nice thing of enumerating all the root replicas and check if there is a match.

##########
File path: hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
##########
@@ -2391,42 +2406,91 @@ public void run(Timeout timeout) throws Exception {
       return failedFuture(new IllegalArgumentException("Passed region name can't be null"));
     }
     try {
+      TableName parentTable;
       CompletableFuture<Optional<HRegionLocation>> future;
       if (RegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) {
         String encodedName = Bytes.toString(regionNameOrEncodedRegionName);
-        if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) {
-          // old format encodedName, should be meta region
+
+        //TODO francis do we really need to support encoded name for root?
+        boolean isRoot = false;

Review comment:
       Will rename.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506775087



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparator.java
##########
@@ -158,4 +174,14 @@ default int compareRows(ByteBuffer row, Cell cell) {
    *   Do not pollute with types other than BBKV if can be helped; the Comparator will slow.
    */
   Comparator getSimpleComparator();
+
+  static CellComparator getComparator(TableName tableName) {
+    if (tableName.equals(TableName.ROOT_TABLE_NAME)) {
+      return RootCellComparator.ROOT_COMPARATOR;
+    }
+    if (tableName.equals(TableName.META_TABLE_NAME)) {
+      return MetaCellComparator.META_COMPARATOR;
+    }
+    return CellComparatorImpl.COMPARATOR;

Review comment:
       No that's just artifact from the PoC iteration will consolidate the other one to use this.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#issuecomment-716446598


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 14s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 47s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 38s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   5m 13s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 56s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 48s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 48s |  the patch passed  |
   | -1 :x: |  shadedjars  |   5m 26s |  patch has 10 errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-balancer generated 3 new + 1 unchanged - 1 fixed = 4 total (was 2)  |
   | -0 :warning: |  javadoc  |   2m 42s |  root generated 3 new + 173 unchanged - 1 fixed = 176 total (was 174)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  12m 20s |  root in the patch failed.  |
   |  |   |  52m 55s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2445 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 62696b6b0efa 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 164cc5a3dc |
   | Default Java | 2020-01-14 |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-jdk11-hadoop3-check/output/patch-shadedjars.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-jdk11-hadoop3-check/output/diff-javadoc-javadoc-hbase-balancer.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-jdk11-hadoop3-check/output/diff-javadoc-javadoc-root.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/testReport/ |
   | Max. process+thread count | 1228 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-zookeeper hbase-balancer hbase-server hbase-backup . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/6/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506778311



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -430,11 +553,19 @@ private boolean isCarryingRegion(final ServerName serverName, final RegionInfo r
     return(node != null && serverName.equals(node.getRegionLocation()));
   }
 
-  private RegionInfo getMetaForRegion(final RegionInfo regionInfo) {
-    //if (regionInfo.isMetaRegion()) return regionInfo;
-    // TODO: handle multiple meta. if the region provided is not meta lookup
-    // which meta the region belongs to.
-    return RegionInfoBuilder.FIRST_META_REGIONINFO;
+  /**
+   * Check hbase:meta is up and ready for reading. For use during Master startup only.
+   * @return True if meta is UP and online and startup can progress. Otherwise, meta is not online
+   *   and we will hold here until operator intervention.
+   */
+  @VisibleForTesting
+  public boolean waitForMetaOnline() {

Review comment:
       This is only during master startup/initialization. We can do as a follow-up as that'd be a big change. As currently RegionStates is mainly used after it has read all the contents of meta. That would be a change in expectation if it would now be possible that RegionStates may not know about certain regions. Eg it would be possible that SCP will only recover regions that RegionStates knows about. And we would need a mechanism to cover for when other regions of meta are loaded down the road. 
   
   Alternatively we can just speedup/parallelize meta assignment until that is no longer good enough. Note also that as part of startup/initialization. We would need to scan meta to bootstrap RegionStates. Internally we parallelized the scanning of meta during startup. I'll remember to add that here too.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506766960



##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.

Review comment:
       This just does meta as well. Part of the original apis.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506777755



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -400,28 +419,132 @@ public boolean isTableDisabled(final TableName tableName) {
   }
 
   // ============================================================================================
-  //  META Helpers
+  //  ROOT Helpers
   // ============================================================================================
-  private boolean isMetaRegion(final RegionInfo regionInfo) {
-    return regionInfo.isMetaRegion();
+  private boolean isRootRegion(final RegionInfo regionInfo) {
+    return regionInfo.isRootRegion();
   }
 
-  public boolean isMetaRegion(final byte[] regionName) {
-    return getMetaRegionFromName(regionName) != null;
+  public boolean isCarryingRoot(final ServerName serverName) {
+    // TODO: handle multiple root
+    return isCarryingRegion(serverName, RegionInfoBuilder.ROOT_REGIONINFO);
   }
 
-  public RegionInfo getMetaRegionFromName(final byte[] regionName) {
-    for (RegionInfo hri: getMetaRegionSet()) {
-      if (Bytes.equals(hri.getRegionName(), regionName)) {
-        return hri;
-      }
+  private RegionInfo getRootForRegion(final RegionInfo regionInfo) {
+    //if (regionInfo.isRootRegion()) return regionInfo;
+    // TODO: handle multiple root. if the region provided is not root lookup
+    // which root the region belongs to.
+    return RegionInfoBuilder.ROOT_REGIONINFO;
+  }
+
+  /**
+   * Check hbase:root is up and ready for reading. For use during Master startup only.
+   * @return True if root is UP and online and startup can progress. Otherwise, root is not online
+   *   and we will hold here until operator intervention.
+   */
+  public boolean waitForRootOnline() {
+    return isRegionOnline(RegionInfoBuilder.ROOT_REGIONINFO);
+  }
+
+  private static final Set<RegionInfo> ROOT_REGION_SET =
+    Collections.singleton(RegionInfoBuilder.ROOT_REGIONINFO);
+  public Set<RegionInfo> getRootRegionSet() {
+    return ROOT_REGION_SET;
+  }
+
+  // ============================================================================================
+  //  ROOT Event(s) helpers
+  // ============================================================================================
+  /**
+   * Notice that, this only means the root region is available on a RS, but the AM may still be
+   * loading the region states from root, so usually you need to check {@link #isRootLoaded()} first
+   * before checking this method, unless you can make sure that your piece of code can only be
+   * executed after AM builds the region states.
+   * @see #isRootLoaded()
+   */
+  public boolean isRootAssigned() {
+    return rootAssignEvent.isReady();
+  }
+
+  public boolean isRootRegionInTransition() {
+    return !isRootAssigned();
+  }
+
+  /**
+   * Notice that this event does not mean the AM has already finished region state rebuilding. See
+   * the comment of {@link #isRootAssigned()} for more details.
+   * @see #isRootAssigned()
+   */
+  public boolean waitRootAssigned(Procedure<?> proc, RegionInfo regionInfo) {
+    return getRootAssignEvent(getRootForRegion(regionInfo)).suspendIfNotReady(proc);
+  }
+
+  private void setRootAssigned(RegionInfo rootRegionInfo, boolean assigned) {
+    assert isRootRegion(rootRegionInfo) : "unexpected non-root region " + rootRegionInfo;
+    if (!RegionReplicaUtil.isDefaultReplica(rootRegionInfo)) {
+      return;
     }
-    return null;
+    ProcedureEvent<?> rootAssignEvent = getRootAssignEvent(rootRegionInfo);
+    if (assigned) {
+      LOG.debug("Setting hbase:root region assigned: "+rootRegionInfo);
+      rootAssignEvent.wake(getProcedureScheduler());
+    } else {
+      LOG.debug("Setting hbase:root region unassigned: "+rootRegionInfo);
+      rootAssignEvent.suspend();
+    }
+  }
+
+  private ProcedureEvent<?> getRootAssignEvent(RegionInfo rootRegionInfo) {
+    assert isRootRegion(rootRegionInfo) : "unexpected non-catalog region " + rootRegionInfo;
+    // TODO: handle multiple root.

Review comment:
       Yeah only one root. Thanks for confirming that. 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506769105



##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
+    throws IOException {
+    byte[] row = regionName;
+    RegionInfo parsedInfo = null;
+    try {
+      parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
+      row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);
+    } catch (Exception parseEx) {
+      return null;
+      // Ignore. This is used with tableName passed as regionName.
+    }
+    Get get = new Get(row);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    Result r;
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(parsedInfo.getTable()))) {
+      r = t.get(get);
+    }
+    RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+    return locations == null ? null :
+      locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionInfo region information
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
+    throws IOException {
+    return CatalogFamilyFormat.getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
+      regionInfo, regionInfo.getReplicaId());
+  }
+
+  /**
+   * @return Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta.
+   */
+  public static Result getCatalogFamilyRow(Connection connection, RegionInfo ri)
+    throws IOException {
+    Get get = new Get(CatalogFamilyFormat.getMetaKeyForRegion(ri));
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(ri.getTable()))) {
+      return t.get(get);
+    }
+  }
+
+  public static TableName getCatalogTableForTable(TableName tableName) {
+    if (TableName.ROOT_TABLE_NAME.equals(tableName)) {
+      throw new IllegalStateException("Can't get catalog table for hbase:root table");
+    }
+    if (TableName.META_TABLE_NAME.equals(tableName)) {
+      return TableName.ROOT_TABLE_NAME;
+    }
+    return TableName.META_TABLE_NAME;
+  }
+
+  public static boolean isCatalogTable(TableName tableName) {
+    return tableName.equals(TableName.ROOT_TABLE_NAME) ||
+      tableName.equals(TableName.META_TABLE_NAME);
+  }
+
+  /**
+   * Gets the result in hbase:meta for the specified region.
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return result of the specified region
+   */
+  public static Result getRegionResult(Connection connection,
+      byte[] regionName) throws IOException {
+    Table catalogTable = null;
+    if (Bytes.equals(RegionInfoBuilder.ROOT_REGIONINFO.getRegionName(), regionName)) {
+      throw new IllegalStateException("This method cannot be used for hbase:root region");
+    }
+    catalogTable = getCatalogHTable(connection,
+      getCatalogTableForTable(
+        CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName).getTable()));
+    Get get = new Get(regionName);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    try (Table t = catalogTable) {
+      return t.get(get);
+    }
+  }
+
+  /**
+   * Scans META table for a row whose key contains the specified <B>regionEncodedName</B>, returning
+   * a single related <code>Result</code> instance if any row is found, null otherwise.
+   * @param connection the connection to query META table.
+   * @param regionEncodedName the region encoded name to look for at META.
+   * @return <code>Result</code> instance with the row related info in META, null otherwise.
+   * @throws IOException if any errors occur while querying META.
+   */
+  public static Result scanByRegionEncodedName(Connection connection, String regionEncodedName)
+    throws IOException {
+    RowFilter rowFilter =
+      new RowFilter(CompareOperator.EQUAL, new SubstringComparator(regionEncodedName));
+    Scan scan = getCatalogScan(connection, 1);
+    scan.setFilter(rowFilter);
+    Result res = null;
+    try (ResultScanner resultScanner =
+      getCatalogHTable(connection, TableName.META_TABLE_NAME).getScanner(scan)) {
+      res = resultScanner.next();
+    }
+    if (res == null) {
+      scan = getCatalogScan(connection, 1);
+      scan.setFilter(rowFilter);
+      try (ResultScanner resultScanner =
+        getCatalogHTable(connection, TableName.ROOT_TABLE_NAME).getScanner(scan)) {
+        res = resultScanner.next();
+      }
+    }
+    return res;
+  }
+
+  /**
+   * Lists all of the regions currently in META.
+   * @param connection to connect with
+   * @param excludeOfflinedSplitParents False if we are to include offlined/splitparents regions,
+   *          true and we'll leave out offlined regions from returned list
+   * @return List of all user-space regions.
+   */
+  @VisibleForTesting
+  public static List<RegionInfo> getAllRegions(Connection connection,
+    boolean excludeOfflinedSplitParents) throws IOException {
+    List<Pair<RegionInfo, ServerName>> result;
+
+    result = getTableRegionsAndLocations(connection, null, excludeOfflinedSplitParents);
+
+    return getListOfRegionInfos(result);
+
+  }
+
+  /**
+   * Gets all of the regions of the specified table. Do not use this method to get meta table
+   * regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @return Ordered list of {@link RegionInfo}.
+   */
+  public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName)
+  throws IOException {
+    return getTableRegions(connection, tableName, false);
+  }
+
+  /**
+   * Gets all of the regions of the specified table. Do not use this method to get meta table
+   * regions, use methods in RootTableLocator instead.
+   * @param connection connection we're using
+   * @param tableName table we're looking for
+   * @param excludeOfflinedSplitParents If true, do not include offlined split parents in the
+   *          return.
+   * @return Ordered list of {@link RegionInfo}.
+   */
+  public static List<RegionInfo> getTableRegions(Connection connection, TableName tableName,
+    final boolean excludeOfflinedSplitParents) throws IOException {
+    List<Pair<RegionInfo, ServerName>> result =
+      getTableRegionsAndLocations(connection, tableName, excludeOfflinedSplitParents);
+    return getListOfRegionInfos(result);
+  }
+
+  private static List<RegionInfo>
+    getListOfRegionInfos(final List<Pair<RegionInfo, ServerName>> pairs) {
+    if (pairs == null || pairs.isEmpty()) {
+      return Collections.emptyList();
+    }
+    List<RegionInfo> result = new ArrayList<>(pairs.size());
+    for (Pair<RegionInfo, ServerName> pair : pairs) {
+      result.add(pair.getFirst());
+    }
+    return result;
+  }
+
+  /**
+   * This method creates a Scan object that will only scan catalog rows that belong to the specified
+   * table. It doesn't specify any columns. This is a better alternative to just using a start row

Review comment:
       I think it means it will retrieve all the columns (there are not columns selected explicitly).
   
   I only changed one line in this code actually. It's pretty much a copy form https://github.com/apache/hbase/blob/master/hbase-balancer/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java#L323. 
   
   It seems if I keep both classes CatalogAccessor and MetaTableAccessor (one as a facade). The methods that got copied as part of the migration would look newly added. I wonder if I should just move these methods back to MetaTableAccessor and make CatalogAccessor the facade, it would make the diffs less confusing?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506767895



##########
File path: hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
##########
@@ -0,0 +1,1071 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import edu.umd.cs.findbugs.annotations.NonNull;
+import edu.umd.cs.findbugs.annotations.Nullable;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import org.apache.hadoop.hbase.Cell.Type;
+import org.apache.hadoop.hbase.ClientMetaTableAccessor.QueryType;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableState;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.ExceptionUtil;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.PairOfSameType;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
+ * to <code>hbase:meta</code>.
+ * <p/>
+ * Some of the methods of this class take ZooKeeperWatcher as a param. The only reason for this is
+ * when this class is used on client-side (e.g. HBaseAdmin), we want to use short-lived connection
+ * (opened before each operation, closed right after), while when used on HM or HRS (like in
+ * AssignmentManager) we want permanent connection.
+ * <p/>
+ * HBASE-10070 adds a replicaId to HRI, meaning more than one HRI can be defined for the same table
+ * range (table, startKey, endKey). For every range, there will be at least one HRI defined which is
+ * called default replica.
+ * <p/>
+ * <h2>Meta layout</h2> For each table there is single row named for the table with a 'table' column
+ * family. The column family currently has one column in it, the 'state' column:
+ *
+ * <pre>
+ * table:state => contains table state
+ * </pre>
+ *
+ * For the catalog family, see the comments of {@link CatalogFamilyFormat} for more details.
+ * <p/>
+ * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
+ * <p/>
+ * The actual layout of meta should be encapsulated inside CatalogAccessor methods, and should not
+ * leak out of it (through Result objects, etc)
+ * @see CatalogFamilyFormat
+ * @see ClientMetaTableAccessor
+ */
+@InterfaceAudience.Private
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
+
+
+
+  ////////////////////////
+  // Reading operations //
+  ////////////////////////
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection, TableName catalogTableName,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection, catalogTableName, null, null, QueryType.REGION,
+      null, Integer.MAX_VALUE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in regions family.
+   */
+  public static void fullScanRegions(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.REGION, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for regions.
+   * @param connection connection we're using
+   */
+  public static List<Result> fullScanRegions(Connection connection) throws IOException {
+    return fullScan(connection, QueryType.REGION);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code> for tables.
+   * @param connection connection we're using
+   * @param visitor Visitor invoked against each row in tables family.
+   */
+  public static void fullScanTables(Connection connection,
+    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanMeta(connection, null, null, QueryType.TABLE, visitor);
+  }
+
+  /**
+   * Performs a full scan of <code>hbase:meta</code>.
+   * @param connection connection we're using
+   * @param type scanned part of meta
+   * @return List of {@link Result}
+   */
+  private static List<Result> fullScan(Connection connection, QueryType type) throws IOException {
+    ClientMetaTableAccessor.CollectAllVisitor v = new ClientMetaTableAccessor.CollectAllVisitor();
+    scanMeta(connection, null, null, type, v);
+    return v.getResults();
+  }
+
+  /**
+   * Callers should call close on the returned {@link Table} instance.
+   * @param connection connection we're using to access Meta
+   * @return An {@link Table} for <code>hbase:meta</code>
+   * @throws NullPointerException if {@code connection} is {@code null}
+   */
+  public static Table getCatalogHTable(final Connection connection, TableName catalogTableName)
+    throws IOException {
+    if (!isCatalogTable(catalogTableName)) {
+      throw new IllegalStateException("Table supplied is not a catalog table: " + catalogTableName);
+    }
+    // We used to pass whole CatalogTracker in here, now we just pass in Connection
+    Objects.requireNonNull(connection, "Connection cannot be null");
+    if (connection.isClosed()) {
+      throw new IOException("connection is closed");
+    }
+    return connection.getTable(catalogTableName);
+  }
+
+  /**
+   * Gets the region info and assignment for the specified region.
+   * @param connection connection we're using
+   * @param regionName Region to lookup.
+   * @return Location and RegionInfo for <code>regionName</code>
+   * @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
+   */
+  @Deprecated
+  public static Pair<RegionInfo, ServerName> getRegion(Connection connection, byte[] regionName)
+    throws IOException {
+    HRegionLocation location = getRegionLocation(connection, regionName);
+    return location == null ? null : new Pair<>(location.getRegion(), location.getServerName());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionName region we're looking for
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
+    throws IOException {
+    byte[] row = regionName;
+    RegionInfo parsedInfo = null;
+    try {
+      parsedInfo = CatalogFamilyFormat.parseRegionInfoFromRegionName(regionName);
+      row = CatalogFamilyFormat.getMetaKeyForRegion(parsedInfo);
+    } catch (Exception parseEx) {
+      return null;
+      // Ignore. This is used with tableName passed as regionName.
+    }
+    Get get = new Get(row);
+    get.addFamily(HConstants.CATALOG_FAMILY);
+    Result r;
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(parsedInfo.getTable()))) {
+      r = t.get(get);
+    }
+    RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
+    return locations == null ? null :
+      locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
+  }
+
+  /**
+   * Returns the HRegionLocation from meta for the given region
+   * @param connection connection we're using
+   * @param regionInfo region information
+   * @return HRegionLocation for the given region
+   */
+  public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
+    throws IOException {
+    return CatalogFamilyFormat.getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
+      regionInfo, regionInfo.getReplicaId());
+  }
+
+  /**
+   * @return Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta.
+   */
+  public static Result getCatalogFamilyRow(Connection connection, RegionInfo ri)

Review comment:
       For both will rename. 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#issuecomment-710730879


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 43s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 25s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 35s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m 23s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 30s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 21s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 21s |  the patch passed  |
   | -1 :x: |  shadedjars  |   4m 16s |  patch has 10 errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 20s |  hbase-common generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  javadoc  |   0m 13s |  hbase-balancer generated 1 new + 0 unchanged - 1 fixed = 1 total (was 1)  |
   | -0 :warning: |  javadoc  |   2m  5s |  root generated 2 new + 31 unchanged - 1 fixed = 33 total (was 32)  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  19m 35s |  root in the patch failed.  |
   |  |   |  55m  6s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.13 Server=19.03.13 base: https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/2445 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 9ff64dd85fb3 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e6c78f4037 |
   | Default Java | 1.8.0_232 |
   | shadedjars | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-jdk8-hadoop3-check/output/patch-shadedjars.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-common.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-balancer.txt |
   | javadoc | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-root.txt |
   | unit | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-root.txt |
   |  Test Results | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/testReport/ |
   | Max. process+thread count | 1093 (vs. ulimit of 30000) |
   | modules | C: hbase-protocol-shaded hbase-common hbase-client hbase-zookeeper hbase-balancer hbase-server hbase-backup . U: . |
   | Console output | https://ci-hadoop.apache.org/job/HBase/job/HBase-PreCommit-GitHub-PR/job/PR-2445/5/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506778937



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -1440,10 +1625,42 @@ private void handleRegionOverStuckWarningThreshold(final RegionInfo regionInfo)
   // ============================================================================================
   //  TODO: Master load/bootstrap
   // ============================================================================================
-  public void joinCluster() throws IOException {
+  public void joinCluster(boolean loadRoot) throws IOException {
+    joinCluster(loadRoot, true, true);
+  }
+
+  @VisibleForTesting
+  public void joinCluster(boolean loadRoot, boolean shouldWaitForRootOnline,
+    boolean shouldWaitForMetaOnline)
+    throws IOException {
     long startTime = System.nanoTime();
     LOG.debug("Joining cluster...");
 
+    // FIRST Catalog tables READ!!!!
+    // The below cannot make progress w/o hbase:meta being online.

Review comment:
       this should talk about loadRoot() and loadMeta(). let me change this to be more generalized to address both code sections?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
saintstack commented on pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#issuecomment-707473657


   First pass notes above.
   
   Point me at the overview on what this patch delivers. Looks like it doesn't include splitting meta support yet? It inserts root.
   
   The blocking in AM waiting on all of hbase:meta to be up before progress is coarse; the whole table must be up before progress can be made. To be addressed in a follow-on.
   
   Seems like it has gotten a bit stale. Needs rebase (conflicts above and some stuff has changed since).
   
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506775285



##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
##########
@@ -110,6 +111,15 @@
   @Deprecated
   public static final KVComparator META_COMPARATOR = new MetaComparator();
 
+  /**
+   * A {@link KVComparator} for <code>hbase:root</code> catalog table
+   * {@link KeyValue}s.
+   * @deprecated Use {@link RootCellComparator#ROOT_COMPARATOR} instead.
+   * Deprecated for hbase 2.0, remove for hbase 3.0.
+   */
+  @Deprecated
+  public static final KVComparator ROOT_COMPARATOR = new RootComparator();

Review comment:
       This will be removed.

##########
File path: hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
##########
@@ -1604,6 +1614,79 @@ public static int getDelimiterInReverse(final byte [] b, final int offset,
     return result;
   }
 
+  /**
+   * A {@link KVComparator} for <code>-ROOT-</code> catalog table
+   * {@link KeyValue}s.
+   * @deprecated : {@link RootCellComparator#ROOT_COMPARATOR} to be used.
+   * Deprecated for hbase 2.0, remove for hbase 3.0.

Review comment:
       This will be removed.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r commented on a change in pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r commented on a change in pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#discussion_r506777664



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
##########
@@ -152,9 +159,19 @@
   private static final int DEFAULT_RIT_STUCK_WARNING_THRESHOLD = 60 * 1000;
   public static final String UNEXPECTED_STATE_REGION = "Unexpected state for ";
 
-  private final ProcedureEvent<?> metaAssignEvent = new ProcedureEvent<>("meta assign");
+  private final ProcedureEvent<?> rootAssignEvent = new ProcedureEvent<>("root assign");
+  private final ProcedureEvent<?> rootLoadEvent = new ProcedureEvent<>("root load");
+
   private final ProcedureEvent<?> metaLoadEvent = new ProcedureEvent<>("meta load");
 
+  private final ConcurrentSkipListMap<byte[], ProcedureEvent<?>> metaAssignEventMap =

Review comment:
       This is here so procedures that depend on particular regions of meta to be available (eg RTSP, ORP, CRP, etc) are only waiting on the necessary regions.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hbase] z3n3r edited a comment on pull request #2445: HBASE-11288 Splittable Meta (WIP) Draft

Posted by GitBox <gi...@apache.org>.
z3n3r edited a comment on pull request #2445:
URL: https://github.com/apache/hbase/pull/2445#issuecomment-710722619


   @saintstack , I did a rebase and force pushed the updated branch. I'll address your comments. 


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org