You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by to...@apache.org on 2020/09/17 09:33:00 UTC

[hbase] 01/01: rebased and flattened split meta PoC

This is an automated email from the ASF dual-hosted git repository.

toffer pushed a commit to branch HBASE-11288
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 6c28aaec5c0fee87275ec612d60b7597bdd33372
Author: Francis Liu <to...@apache.org>
AuthorDate: Thu Sep 17 02:17:32 2020 -0700

    rebased and flattened split meta PoC
---
 conf/log4j.properties                              |    2 +-
 .../backup/impl/IncrementalBackupManager.java      |   10 +-
 .../hadoop/hbase/backup/util/BackupUtils.java      |    2 +-
 .../hbase/backup/master/TestBackupLogCleaner.java  |    2 +-
 ...MetaTableAccessor.java => CatalogAccessor.java} |  249 +++--
 .../org/apache/hadoop/hbase/MetaTableAccessor.java | 1054 +-------------------
 .../hbase/master/assignment/ServerState.java       |   15 +-
 .../hadoop/hbase/ClientMetaTableAccessor.java      |    6 +-
 ....java => NotAllRootRegionsOnlineException.java} |    6 +-
 ...Locator.java => AsyncNonRootRegionLocator.java} |   58 +-
 .../hadoop/hbase/client/AsyncRegionLocator.java    |   29 +-
 ...ionLocator.java => AsyncRootRegionLocator.java} |    4 +-
 .../hbase/client/AsyncTableRegionLocatorImpl.java  |    6 +-
 .../hadoop/hbase/client/MutableRegionInfo.java     |   16 +-
 .../hadoop/hbase/client/RawAsyncHBaseAdmin.java    |   61 +-
 .../org/apache/hadoop/hbase/client/RegionInfo.java |   47 +-
 .../hadoop/hbase/client/RegionInfoBuilder.java     |   19 +
 .../hadoop/hbase/client/TableDescriptor.java       |   14 +
 .../hbase/client/TableDescriptorBuilder.java       |   33 +
 .../hadoop/hbase/client/ZKConnectionRegistry.java  |    6 +-
 .../hadoop/hbase/shaded/protobuf/ProtobufUtil.java |    9 +-
 ...ava => TestAsyncRootRegionLocatorFailFast.java} |    8 +-
 .../apache/hadoop/hbase/CellComparatorImpl.java    |   10 +-
 .../java/org/apache/hadoop/hbase/KeyValue.java     |   83 ++
 .../apache/hadoop/hbase/MetaCellComparator.java    |    2 +-
 .../apache/hadoop/hbase/RootCellComparator.java    |   81 ++
 .../java/org/apache/hadoop/hbase/TableName.java    |   11 +
 .../apache/hadoop/hbase/TestCellComparator.java    |   69 ++
 .../protobuf/server/master/MasterProcedure.proto   |   20 +-
 .../coordination/ZkSplitLogWorkerCoordination.java |   14 +-
 .../hadoop/hbase/io/hfile/FixedFileTrailer.java    |   12 +
 .../hadoop/hbase/io/hfile/HFileWriterImpl.java     |    4 +-
 .../org/apache/hadoop/hbase/master/HMaster.java    |   65 +-
 ...aBootstrap.java => MasterCatalogBootstrap.java} |   93 +-
 .../hadoop/hbase/master/MasterRpcServices.java     |   26 +-
 .../hadoop/hbase/master/MasterStatusServlet.java   |    4 +-
 .../hadoop/hbase/master/MasterWalManager.java      |   50 +-
 .../hbase/master/MetaRegionLocationCache.java      |    4 +-
 .../hadoop/hbase/master/SplitWALManager.java       |   48 +-
 .../hbase/master/assignment/AssignmentManager.java |  338 ++++++-
 .../assignment/MergeTableRegionsProcedure.java     |   11 +-
 .../assignment/RegionRemoteProcedureBase.java      |   17 +-
 .../hbase/master/assignment/RegionStateStore.java  |   77 +-
 .../hbase/master/assignment/RegionStates.java      |   16 +
 .../assignment/TransitRegionStateProcedure.java    |   17 +-
 .../master/procedure/HBCKServerCrashProcedure.java |   10 +-
 .../hbase/master/procedure/InitMetaProcedure.java  |   29 +-
 ...itMetaProcedure.java => InitRootProcedure.java} |  161 ++-
 .../master/procedure/MasterProcedureUtil.java      |    8 +-
 .../hbase/master/procedure/ProcedureSyncWait.java  |   10 +-
 .../master/procedure/ServerCrashProcedure.java     |  116 ++-
 .../master/procedure/ServerProcedureInterface.java |    5 +
 .../hbase/master/procedure/SplitWALProcedure.java  |    4 +
 .../master/procedure/SplitWALRemoteProcedure.java  |    5 +
 .../procedure/SwitchRpcThrottleProcedure.java      |    5 +
 .../SwitchRpcThrottleRemoteProcedure.java          |    5 +
 .../master/snapshot/MasterSnapshotVerifier.java    |   10 +-
 .../hbase/master/snapshot/TakeSnapshotHandler.java |    8 +-
 .../hadoop/hbase/master/webapp/MetaBrowser.java    |    2 +-
 .../flush/MasterFlushTableProcedureManager.java    |    8 +-
 .../apache/hadoop/hbase/regionserver/HRegion.java  |   27 +-
 .../hadoop/hbase/regionserver/HRegionServer.java   |   11 +-
 .../apache/hadoop/hbase/regionserver/HStore.java   |    2 +-
 .../hadoop/hbase/regionserver/RSRpcServices.java   |    6 +-
 .../regionserver/handler/AssignRegionHandler.java  |    4 +-
 .../regionserver/handler/OpenRootHandler.java      |   33 +-
 .../hadoop/hbase/util/FSTableDescriptors.java      |   90 +-
 .../hadoop/hbase/wal/AbstractFSWALProvider.java    |   24 +
 .../hadoop/hbase/wal/AsyncFSWALProvider.java       |    8 +-
 .../wal/BoundedRecoveredHFilesOutputSink.java      |   16 +-
 .../apache/hadoop/hbase/wal/FSHLogProvider.java    |    8 +-
 .../hadoop/hbase/wal/RegionGroupingProvider.java   |   20 +-
 .../org/apache/hadoop/hbase/wal/WALFactory.java    |   61 +-
 .../main/resources/hbase-webapps/master/table.jsp  |    8 +-
 .../java/org/apache/hadoop/hbase/HBaseCluster.java |    7 +
 ...TableAccessor.java => TestCatalogAccessor.java} |   89 +-
 .../hadoop/hbase/TestClientClusterMetrics.java     |   26 +-
 .../hbase/TestMetaUpdatesGoToPriorityQueue.java    |   14 +-
 ...TableLocator.java => TestRootTableLocator.java} |   48 +-
 .../hbase/client/MetaWithReplicasTestBase.java     |    4 +-
 .../client/TestAsyncAdminWithRegionReplicas.java   |   10 +-
 ...tor.java => TestAsyncNonRootRegionLocator.java} |    8 +-
 ...tAsyncNonRootRegionLocatorConcurrenyLimit.java} |   10 +-
 ...ocator.java => TestAsyncRootRegionLocator.java} |   10 +-
 .../hbase/client/TestAsyncTableLocatePrefetch.java |    6 +-
 .../hbase/client/TestAsyncTableRSCrashPublish.java |    2 +-
 ...ster.java => TestCatalogAccessorNoCluster.java} |    6 +-
 .../hbase/client/TestMetaRegionLocationCache.java  |   10 +-
 .../hbase/client/TestMetaWithReplicasBasic.java    |    4 +-
 .../hbase/client/TestSplitMetaBasicOperations.java |  412 ++++++++
 .../hbase/client/TestZKConnectionRegistry.java     |    2 +-
 .../hadoop/hbase/http/TestInfoServersACL.java      |    2 +-
 .../apache/hadoop/hbase/master/TestDeadServer.java |    2 +-
 .../hadoop/hbase/master/TestMasterFailover.java    |    6 +-
 .../hbase/master/TestMetaShutdownHandler.java      |    6 +-
 .../hbase/master/TestRegionsRecoveryChore.java     |   10 +
 .../hadoop/hbase/master/TestSplitWALManager.java   |    8 +-
 .../master/assignment/TestAssignmentManager.java   |   14 +-
 .../assignment/TestAssignmentManagerBase.java      |    2 +
 .../assignment/TestCloseRegionWhileRSCrash.java    |    5 +
 .../master/assignment/TestRegionStateStore.java    |   32 +-
 .../hadoop/hbase/master/procedure/TestSCP.java     |    2 +-
 .../procedure/TestServerRemoteProcedure.java       |    4 +
 .../hbase/regionserver/TestDefaultMemStore.java    |    2 +-
 .../regionserver/TestGetClosestAtOrBefore.java     |    6 +-
 .../TestReadAndWriteRegionInfoFile.java            |    2 +-
 .../hadoop/hbase/regionserver/TestRegionInfo.java  |    2 +-
 .../hbase/regionserver/TestRegionReplicas.java     |    4 +-
 .../TestRegionReplicasWithRestartScenarios.java    |    2 +-
 .../regionserver/TestRegionServerNoMaster.java     |    6 +-
 .../regionserver/wal/TestLogRollingNoCluster.java  |    4 +-
 .../hadoop/hbase/rsgroup/TestRSGroupsBasics.java   |    4 +-
 ...MetaTableLocator.java => RootTableLocator.java} |  134 +--
 .../org/apache/hadoop/hbase/zookeeper/ZKUtil.java  |    4 +-
 .../apache/hadoop/hbase/zookeeper/ZKWatcher.java   |   18 +-
 115 files changed, 2624 insertions(+), 1757 deletions(-)

diff --git a/conf/log4j.properties b/conf/log4j.properties
index 2282fa5..f50f7ff 100644
--- a/conf/log4j.properties
+++ b/conf/log4j.properties
@@ -96,7 +96,7 @@ log4j.appender.asyncconsole.target=System.err
 log4j.logger.org.apache.zookeeper=${hbase.log.level}
 #log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG
 log4j.logger.org.apache.hadoop.hbase=${hbase.log.level}
-log4j.logger.org.apache.hadoop.hbase.META=${hbase.log.level}
+log4j.logger.org.apache.hadoop.hbase.CATALOG=${hbase.log.level}
 # Make these two classes INFO-level. Make them DEBUG to see more zk debug.
 log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil=${hbase.log.level}
 log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKWatcher=${hbase.log.level}
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
index 93d264a..c643aff 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/IncrementalBackupManager.java
@@ -288,9 +288,9 @@ public class IncrementalBackupManager extends BackupManager {
       }
       for (FileStatus log : logs) {
         LOG.debug("currentLogFile: " + log.getPath().toString());
-        if (AbstractFSWALProvider.isMetaFile(log.getPath())) {
+        if (AbstractFSWALProvider.isCatalogFile(log.getPath())) {
           if (LOG.isDebugEnabled()) {
-            LOG.debug("Skip hbase:meta log file: " + log.getPath().getName());
+            LOG.debug("Skip hbase:root/hbase:meta log file: " + log.getPath().getName());
           }
           continue;
         }
@@ -321,7 +321,7 @@ public class IncrementalBackupManager extends BackupManager {
     for (FileStatus oldlog : oldlogs) {
       p = oldlog.getPath();
       currentLogFile = p.toString();
-      if (AbstractFSWALProvider.isMetaFile(p)) {
+      if (AbstractFSWALProvider.isCatalogFile(p)) {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Skip .meta log file: " + currentLogFile);
         }
@@ -377,9 +377,9 @@ public class IncrementalBackupManager extends BackupManager {
     @Override
     public boolean accept(Path path) {
       // skip meta table log -- ts.meta file
-      if (AbstractFSWALProvider.isMetaFile(path)) {
+      if (AbstractFSWALProvider.isCatalogFile(path)) {
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Skip .meta log file: " + path.getName());
+          LOG.debug("Skip .root/.meta log file: " + path.getName());
         }
         return false;
       }
diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
index 2f81bfc..65a9d9e 100644
--- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
+++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/BackupUtils.java
@@ -258,7 +258,7 @@ public final class BackupUtils {
 
     PathFilter filter = p -> {
       try {
-        if (AbstractFSWALProvider.isMetaFile(p)) {
+        if (AbstractFSWALProvider.isCatalogFile(p)) {
           return false;
         }
         String host = parseHostNameFromLogFile(p);
diff --git a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java
index 6b8011e..ed6233a 100644
--- a/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java
+++ b/hbase-backup/src/test/java/org/apache/hadoop/hbase/backup/master/TestBackupLogCleaner.java
@@ -157,7 +157,7 @@ public class TestBackupLogCleaner extends TestBackupBase {
     List<FileStatus> logFiles = new ArrayList<FileStatus>();
     while (it.hasNext()) {
       LocatedFileStatus lfs = it.next();
-      if (lfs.isFile() && !AbstractFSWALProvider.isMetaFile(lfs.getPath())) {
+      if (lfs.isFile() && !AbstractFSWALProvider.isCatalogFile(lfs.getPath())) {
         logFiles.add(lfs);
         LOG.info(Objects.toString(lfs));
       }
diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
similarity index 82%
copy from hbase-balancer/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
copy to hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
index c90d662..6dc0a32 100644
--- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/CatalogAccessor.java
@@ -37,6 +37,7 @@ 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;
@@ -82,19 +83,19 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
  * <p/>
  * TODO: Add rep_barrier for serial replication explanation. See SerialReplicationChecker.
  * <p/>
- * The actual layout of meta should be encapsulated inside MetaTableAccessor methods, and should not
+ * 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 final class MetaTableAccessor {
+public class CatalogAccessor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(CatalogAccessor.class);
+  private static final Logger CATALOGLOG =
+    LoggerFactory.getLogger("org.apache.hadoop.hbase.CATALOG");
 
-  private static final Logger LOG = LoggerFactory.getLogger(MetaTableAccessor.class);
-  private static final Logger METALOG = LoggerFactory.getLogger("org.apache.hadoop.hbase.META");
 
-  private MetaTableAccessor() {
-  }
 
   ////////////////////////
   // Reading operations //
@@ -105,6 +106,17 @@ public final class MetaTableAccessor {
    * @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);
@@ -146,13 +158,17 @@ public final class MetaTableAccessor {
    * @return An {@link Table} for <code>hbase:meta</code>
    * @throws NullPointerException if {@code connection} is {@code null}
    */
-  public static Table getMetaHTable(final Connection connection) throws IOException {
+  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(TableName.META_TABLE_NAME);
+    return connection.getTable(catalogTableName);
   }
 
   /**
@@ -183,12 +199,13 @@ public final class MetaTableAccessor {
       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 = getMetaHTable(connection)) {
+    try (Table t = getCatalogHTable(connection, getCatalogTableForTable(parsedInfo.getTable()))) {
       r = t.get(get);
     }
     RegionLocations locations = CatalogFamilyFormat.getRegionLocations(r);
@@ -215,22 +232,44 @@ public final class MetaTableAccessor {
     throws IOException {
     Get get = new Get(CatalogFamilyFormat.getMetaKeyForRegion(ri));
     get.addFamily(HConstants.CATALOG_FAMILY);
-    try (Table t = getMetaHTable(connection)) {
+    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 {
+  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 = getMetaHTable(connection)) {
+    try (Table t = catalogTable) {
       return t.get(get);
     }
   }
@@ -247,10 +286,18 @@ public final class MetaTableAccessor {
     throws IOException {
     RowFilter rowFilter =
       new RowFilter(CompareOperator.EQUAL, new SubstringComparator(regionEncodedName));
-    Scan scan = getMetaScan(connection, 1);
+    Scan scan = getCatalogScan(connection, 1);
     scan.setFilter(rowFilter);
-    ResultScanner resultScanner = getMetaHTable(connection).getScanner(scan);
-    return resultScanner.next();
+    ResultScanner resultScanner =
+      getCatalogHTable(connection, TableName.META_TABLE_NAME).getScanner(scan);
+    Result res = resultScanner.next();
+    if (res == null) {
+      scan = getCatalogScan(connection, 1);
+      scan.setFilter(rowFilter);
+      resultScanner = getCatalogHTable(connection, TableName.ROOT_TABLE_NAME).getScanner(scan);
+      res = resultScanner.next();
+    }
+    return res;
   }
 
   /**
@@ -273,19 +320,19 @@ public final class MetaTableAccessor {
 
   /**
    * Gets all of the regions of the specified table. Do not use this method to get meta table
-   * regions, use methods in MetaTableLocator instead.
+   * 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 {
+  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 MetaTableLocator instead.
+   * 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
@@ -326,13 +373,13 @@ public final class MetaTableAccessor {
     // Stop key appends the smallest possible char to the table name
     byte[] stopKey = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);
 
-    Scan scan = getMetaScan(connection, -1);
+    Scan scan = getCatalogScan(connection, -1);
     scan.withStartRow(startKey);
     scan.withStopRow(stopKey);
     return scan;
   }
 
-  private static Scan getMetaScan(Connection connection, int rowUpperLimit) {
+  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);
@@ -349,7 +396,7 @@ public final class MetaTableAccessor {
   }
 
   /**
-   * Do not use this method to get meta table regions, use methods in MetaTableLocator instead.
+   * 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.
@@ -360,7 +407,7 @@ public final class MetaTableAccessor {
   }
 
   /**
-   * Do not use this method to get meta table regions, use methods in MetaTableLocator instead.
+   * 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
@@ -370,16 +417,28 @@ public final class MetaTableAccessor {
   public static List<Pair<RegionInfo, ServerName>> getTableRegionsAndLocations(
     Connection connection, @Nullable final TableName tableName,
     final boolean excludeOfflinedSplitParents) throws IOException {
-    if (tableName != null && tableName.equals(TableName.META_TABLE_NAME)) {
+    if (tableName != null && tableName.equals(TableName.ROOT_TABLE_NAME)) {
       throw new IOException(
-        "This method can't be used to locate meta regions;" + " use MetaTableLocator instead");
+        "This method can't be used to locate meta regions; use RootTableLocator instead");
     }
     // Make a version of CollectingVisitor that collects RegionInfo and ServerAddress
     ClientMetaTableAccessor.CollectRegionLocationsVisitor visitor =
       new ClientMetaTableAccessor.CollectRegionLocationsVisitor(excludeOfflinedSplitParents);
-    scanMeta(connection,
-      ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION),
-      ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION), QueryType.REGION,
+    byte[] startRow = ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION);
+    byte[] stopRow = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);
+    TableName parentTable = TableName.META_TABLE_NAME;
+    if (tableName.equals(TableName.META_TABLE_NAME)) {
+      parentTable = TableName.ROOT_TABLE_NAME;
+      startRow = null;
+      stopRow = null;
+    }
+
+    scanCatalog(connection,
+      parentTable,
+      startRow,
+      stopRow,
+      QueryType.REGION,
+      Integer.MAX_VALUE,
       visitor);
     return visitor.getResults();
   }
@@ -411,19 +470,22 @@ public final class MetaTableAccessor {
 
   public static void scanMetaForTableRegions(Connection connection,
     ClientMetaTableAccessor.Visitor visitor, TableName tableName) throws IOException {
-    scanMeta(connection, tableName, QueryType.REGION, Integer.MAX_VALUE, visitor);
+    scanCatalog(connection, tableName, QueryType.REGION, Integer.MAX_VALUE, visitor);
   }
 
-  private static void scanMeta(Connection connection, TableName table, QueryType type, int maxRows,
-    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
-    scanMeta(connection, ClientMetaTableAccessor.getTableStartRowForMeta(table, type),
-      ClientMetaTableAccessor.getTableStopRowForMeta(table, type), type, maxRows, visitor);
+  public static void scanCatalog(Connection connection, TableName table, QueryType type,
+    int maxRows, final ClientMetaTableAccessor.Visitor visitor) 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 {
-    scanMeta(connection, startRow, stopRow, type, Integer.MAX_VALUE, visitor);
+    scanCatalog(connection, TableName.META_TABLE_NAME, startRow, stopRow, type, Integer.MAX_VALUE,
+        visitor);
   }
 
   /**
@@ -447,7 +509,8 @@ public final class MetaTableAccessor {
       }
       stopRow = ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION);
     }
-    scanMeta(connection, startRow, stopRow, QueryType.REGION, rowLimit, visitor);
+    scanCatalog(connection, TableName.META_TABLE_NAME, startRow, stopRow, QueryType.REGION,
+        rowLimit, visitor);
   }
 
   /**
@@ -459,17 +522,18 @@ public final class MetaTableAccessor {
    * @param maxRows maximum rows to return
    * @param visitor Visitor invoked against each row.
    */
-  static void scanMeta(Connection connection, @Nullable final byte[] startRow,
-    @Nullable final byte[] stopRow, QueryType type, int maxRows,
-    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
-    scanMeta(connection, startRow, stopRow, type, null, maxRows, visitor);
+  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 scanMeta(Connection connection, @Nullable final byte[] startRow,
-    @Nullable final byte[] stopRow, QueryType type, @Nullable Filter filter, int maxRows,
+  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 = getMetaScan(connection, rowUpperLimit);
+    Scan scan = getCatalogScan(connection, rowUpperLimit);
 
     for (byte[] family : type.getFamilies()) {
       scan.addFamily(family);
@@ -491,8 +555,8 @@ public final class MetaTableAccessor {
     }
 
     int currentRow = 0;
-    try (Table metaTable = getMetaHTable(connection)) {
-      try (ResultScanner scanner = metaTable.getScanner(scan)) {
+    try (Table catalogTable = getCatalogHTable(connection, catalogTableName)) {
+      try (ResultScanner scanner = catalogTable.getScanner(scan)) {
         Result data;
         while ((data = scanner.next()) != null) {
           if (data.isEmpty()) {
@@ -525,10 +589,11 @@ public final class MetaTableAccessor {
   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 = getMetaScan(connection, 1);
+    Scan scan = getCatalogScan(connection, 1);
     scan.setReversed(true);
     scan.withStartRow(searchRow);
-    try (ResultScanner resultScanner = getMetaHTable(connection).getScanner(scan)) {
+    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 +
@@ -586,11 +651,13 @@ public final class MetaTableAccessor {
    * @param tableName table to fetch state for
    */
   @Nullable
-  public static TableState getTableState(Connection conn, TableName tableName) throws IOException {
-    if (tableName.equals(TableName.META_TABLE_NAME)) {
+  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 = getMetaHTable(conn);
+    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);
@@ -673,7 +740,7 @@ public final class MetaTableAccessor {
    * @param p Put to add to hbase:meta
    */
   private static void putToMetaTable(Connection connection, Put p) throws IOException {
-    try (Table table = getMetaHTable(connection)) {
+    try (Table table = getCatalogHTable(connection, TableName.META_TABLE_NAME)) {
       put(table, p);
     }
   }
@@ -683,7 +750,7 @@ public final class MetaTableAccessor {
    * @param p put to make
    */
   private static void put(Table t, Put p) throws IOException {
-    debugLogMutation(p);
+    debugLogMutation(t.getName(), p);
     t.put(p);
   }
 
@@ -692,13 +759,14 @@ public final class MetaTableAccessor {
    * @param connection connection we're using
    * @param ps Put to add to hbase:meta
    */
-  public static void putsToMetaTable(final Connection connection, final List<Put> ps)
-    throws IOException {
+  public static void putsToCatalogTable(final Connection connection, TableName tableName,
+    final List<Put> ps)
+      throws IOException {
     if (ps.isEmpty()) {
       return;
     }
-    try (Table t = getMetaHTable(connection)) {
-      debugLogMutations(ps);
+    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));
@@ -713,22 +781,22 @@ public final class MetaTableAccessor {
    * @param connection connection we're using
    * @param d Delete to add to hbase:meta
    */
-  private static void deleteFromMetaTable(final Connection connection, final Delete d)
-    throws IOException {
+  private static void deleteFromCatalogTable(final Connection connection, TableName catalogTable,
+    final Delete d) throws IOException {
     List<Delete> dels = new ArrayList<>(1);
     dels.add(d);
-    deleteFromMetaTable(connection, dels);
+    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.
+   * @param deletes Deletes to add to hbase:meta  This list should support #remove.
    */
-  private static void deleteFromMetaTable(final Connection connection, final List<Delete> deletes)
-    throws IOException {
-    try (Table t = getMetaHTable(connection)) {
-      debugLogMutations(deletes);
+  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);
     }
   }
@@ -740,9 +808,9 @@ public final class MetaTableAccessor {
    * @param numReplicasToRemove how many replicas to remove
    * @param connection connection we're using to access meta table
    */
-  public static void removeRegionReplicasFromMeta(Set<byte[]> metaRows,
-    int replicaIndexToDeleteFrom, int numReplicasToRemove, Connection connection)
-    throws IOException {
+  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();
@@ -760,7 +828,7 @@ public final class MetaTableAccessor {
           CatalogFamilyFormat.getRegionStateColumn(i), now);
       }
 
-      deleteFromMetaTable(connection, deleteReplicaLocations);
+      deleteFromCatalogTable(connection, catalogTableName, deleteReplicaLocations);
     }
   }
 
@@ -778,7 +846,8 @@ public final class MetaTableAccessor {
   public static void updateRegionState(Connection connection, RegionInfo ri,
     RegionState.State state) throws IOException {
     Put put = new Put(RegionReplicaUtil.getRegionInfoForDefaultReplica(ri).getRegionName());
-    putsToMetaTable(connection, Collections.singletonList(addRegionStateToPut(put, state)));
+    putsToCatalogTable(connection, getCatalogTableForTable(ri.getTable()),
+        Collections.singletonList(addRegionStateToPut(put, state)));
   }
 
   /**
@@ -794,12 +863,13 @@ public final class MetaTableAccessor {
    * @throws IOException if problem connecting or updating meta
    */
   public static void addSplitsToParent(Connection connection, RegionInfo regionInfo,
-    RegionInfo splitA, RegionInfo splitB) throws IOException {
-    try (Table meta = getMetaHTable(connection)) {
+      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);
-      meta.put(put);
-      debugLogMutation(put);
+      catalog.put(put);
+      debugLogMutation(catalogTable, put);
       LOG.debug("Added region {}", regionInfo.getRegionNameAsString());
     }
   }
@@ -857,8 +927,8 @@ public final class MetaTableAccessor {
         puts.add(put);
       }
     }
-    putsToMetaTable(connection, puts);
-    LOG.info("Added {} regions to meta.", puts.size());
+    putsToCatalogTable(connection, TableName.META_TABLE_NAME, puts);
+    LOG.info("Added {} regions to {}.",puts.size(), TableName.META_TABLE_NAME);
   }
 
   /**
@@ -892,8 +962,8 @@ public final class MetaTableAccessor {
     long time = EnvironmentEdgeManager.currentTime();
     Delete delete = new Delete(table.getName());
     delete.addColumns(HConstants.TABLE_FAMILY, HConstants.TABLE_STATE_QUALIFIER, time);
-    deleteFromMetaTable(connection, delete);
-    LOG.info("Deleted table " + table + " state from META");
+    deleteFromCatalogTable(connection, TableName.META_TABLE_NAME, delete);
+    LOG.info("Deleted table " + table + " state from Catalog");
   }
 
   /**
@@ -946,7 +1016,7 @@ public final class MetaTableAccessor {
     throws IOException {
     Delete delete = new Delete(regionInfo.getRegionName());
     delete.addFamily(HConstants.CATALOG_FAMILY, HConstants.LATEST_TIMESTAMP);
-    deleteFromMetaTable(connection, delete);
+    deleteFromCatalogTable(connection, getCatalogTableForTable(regionInfo.getTable()), delete);
     LOG.info("Deleted " + regionInfo.getRegionNameAsString());
   }
 
@@ -968,13 +1038,22 @@ public final class MetaTableAccessor {
   private static void deleteRegionInfos(Connection connection, List<RegionInfo> regionsInfo,
     long ts) throws IOException {
     List<Delete> deletes = new ArrayList<>(regionsInfo.size());
+    TableName prevCatalogTableName = null;
+    RegionInfo prevRegionInfo = null;
     for (RegionInfo hri : regionsInfo) {
       Delete e = new Delete(hri.getRegionName());
       e.addFamily(HConstants.CATALOG_FAMILY, ts);
       deletes.add(e);
+      TableName currCatalogTableName = getCatalogTableForTable(hri.getTable());
+      if (prevCatalogTableName != null && !prevCatalogTableName.equals(currCatalogTableName)) {
+        throw new IllegalStateException("Deleting from than one catalog table: " +
+          prevRegionInfo + ","+ hri);
+      }
+      prevCatalogTableName = currCatalogTableName;
+      prevRegionInfo = hri;
     }
-    deleteFromMetaTable(connection, deletes);
-    LOG.info("Deleted {} regions from META", regionsInfo.size());
+    deleteFromCatalogTable(connection, prevCatalogTableName,  deletes);
+    LOG.info("Deleted {} regions from {}", regionsInfo.size(), prevCatalogTableName);
     LOG.debug("Deleted regions: {}", regionsInfo);
   }
 
@@ -1043,19 +1122,19 @@ public final class MetaTableAccessor {
         .setType(Cell.Type.Put).build());
   }
 
-
-  private static void debugLogMutations(List<? extends Mutation> mutations) throws IOException {
-    if (!METALOG.isDebugEnabled()) {
+  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(mutation);
+      debugLogMutation(tableName, mutation);
     }
   }
 
-  private static void debugLogMutation(Mutation p) throws IOException {
-    METALOG.debug("{} {}", p.getClass().getSimpleName(), p.toJSON());
+  private static void debugLogMutation(TableName t, Mutation p) throws IOException {
+    CATALOGLOG.debug("{} {} {}", t, p.getClass().getSimpleName(), p.toJSON());
   }
 }
diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index c90d662..4c6ed89 100644
--- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -1,143 +1,36 @@
-/*
- * 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.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;
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
 
 /**
- * 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 MetaTableAccessor methods, and should not
- * leak out of it (through Result objects, etc)
- * @see CatalogFamilyFormat
- * @see ClientMetaTableAccessor
+ * This class has been replaced by CatalogAccessor.
+ * It is currently kept as a proxy for CatalogAccessor to ease
+ * the review of the HBASE-11288 (split meta) patch. This class will
+ * be remove as soon as this patch is committed. Which should be
+ * pretty much a simple search and replace operation.
  */
 @InterfaceAudience.Private
-public final class MetaTableAccessor {
-
-  private static final Logger LOG = LoggerFactory.getLogger(MetaTableAccessor.class);
-  private static final Logger METALOG = LoggerFactory.getLogger("org.apache.hadoop.hbase.META");
-
-  private MetaTableAccessor() {
-  }
-
-  ////////////////////////
-  // 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,
-    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
-    scanMeta(connection, null, null, QueryType.REGION, visitor);
-  }
+public class MetaTableAccessor extends CatalogAccessor {
 
   /**
-   * 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>.
+   * Put the passed <code>ps</code> to the <code>hbase:meta</code> table.
    * @param connection connection we're using
-   * @param type scanned part of meta
-   * @return List of {@link Result}
+   * @param ps Put to add to hbase:meta
    */
-  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();
+  public static void putsToMetaTable(final Connection connection, final List<Put> ps)
+      throws IOException {
+    putsToCatalogTable(connection, TableName.META_TABLE_NAME, ps);
   }
 
   /**
@@ -146,7 +39,8 @@ public final class MetaTableAccessor {
    * @return An {@link Table} for <code>hbase:meta</code>
    * @throws NullPointerException if {@code connection} is {@code null}
    */
-  public static Table getMetaHTable(final Connection connection) throws IOException {
+  public static Table getMetaHTable(final Connection connection)
+    throws IOException {
     // We used to pass whole CatalogTracker in here, now we just pass in Connection
     Objects.requireNonNull(connection, "Connection cannot be null");
     if (connection.isClosed()) {
@@ -155,582 +49,32 @@ public final class MetaTableAccessor {
     return connection.getTable(TableName.META_TABLE_NAME);
   }
 
-  /**
-   * 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) {
-      // Ignore. This is used with tableName passed as regionName.
-    }
-    Get get = new Get(row);
-    get.addFamily(HConstants.CATALOG_FAMILY);
-    Result r;
-    try (Table t = getMetaHTable(connection)) {
-      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 = getMetaHTable(connection)) {
-      return t.get(get);
-    }
-  }
-
-  /**
-   * 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 {
-    Get get = new Get(regionName);
-    get.addFamily(HConstants.CATALOG_FAMILY);
-    try (Table t = getMetaHTable(connection)) {
-      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 = getMetaScan(connection, 1);
-    scan.setFilter(rowFilter);
-    ResultScanner resultScanner = getMetaHTable(connection).getScanner(scan);
-    return resultScanner.next();
-  }
-
-  /**
-   * 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 MetaTableLocator 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 MetaTableLocator 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 = getMetaScan(connection, -1);
-    scan.withStartRow(startKey);
-    scan.withStopRow(stopKey);
-    return scan;
-  }
-
-  private static Scan getMetaScan(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 MetaTableLocator 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 MetaTableLocator 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.META_TABLE_NAME)) {
-      throw new IOException(
-        "This method can't be used to locate meta regions;" + " use MetaTableLocator instead");
-    }
-    // Make a version of CollectingVisitor that collects RegionInfo and ServerAddress
-    ClientMetaTableAccessor.CollectRegionLocationsVisitor visitor =
-      new ClientMetaTableAccessor.CollectRegionLocationsVisitor(excludeOfflinedSplitParents);
-    scanMeta(connection,
-      ClientMetaTableAccessor.getTableStartRowForMeta(tableName, QueryType.REGION),
-      ClientMetaTableAccessor.getTableStopRowForMeta(tableName, QueryType.REGION), QueryType.REGION,
+  public static void scanMeta(Connection connection,
+      @Nullable final byte[] startRow, @Nullable final byte[] stopRow,
+      ClientMetaTableAccessor.QueryType type, int maxRows,
+      final ClientMetaTableAccessor.Visitor visitor) throws IOException {
+    scanCatalog(connection,
+      TableName.META_TABLE_NAME,
+      startRow,
+      stopRow,
+      type,
+      null,
+      maxRows,
       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 {
-    scanMeta(connection, tableName, QueryType.REGION, Integer.MAX_VALUE, visitor);
   }
 
-  private static void scanMeta(Connection connection, TableName table, QueryType type, int maxRows,
+  public static void scanMeta(Connection connection,
+    @Nullable final byte[] startRow, @Nullable final byte[] stopRow,
+    ClientMetaTableAccessor.QueryType type, @Nullable Filter filter, int maxRows,
     final ClientMetaTableAccessor.Visitor visitor) throws IOException {
-    scanMeta(connection, 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 {
-    scanMeta(connection, 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);
-    }
-    scanMeta(connection, 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.
-   */
-  static void scanMeta(Connection connection, @Nullable final byte[] startRow,
-    @Nullable final byte[] stopRow, QueryType type, int maxRows,
-    final ClientMetaTableAccessor.Visitor visitor) throws IOException {
-    scanMeta(connection, startRow, stopRow, type, null, maxRows, visitor);
-  }
-
-  public static void scanMeta(Connection connection, @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 = getMetaScan(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 metaTable = getMetaHTable(connection)) {
-      try (ResultScanner scanner = metaTable.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 = getMetaScan(connection, 1);
-    scan.setReversed(true);
-    scan.withStartRow(searchRow);
-    try (ResultScanner resultScanner = getMetaHTable(connection).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.META_TABLE_NAME)) {
-      return new TableState(tableName, TableState.State.ENABLED);
-    }
-    Table metaHTable = getMetaHTable(conn);
-    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 = getMetaHTable(connection)) {
-      put(table, p);
-    }
-  }
-
-  /**
-   * @param t Table to use
-   * @param p put to make
-   */
-  private static void put(Table t, Put p) throws IOException {
-    debugLogMutation(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 putsToMetaTable(final Connection connection, final List<Put> ps)
-    throws IOException {
-    if (ps.isEmpty()) {
-      return;
-    }
-    try (Table t = getMetaHTable(connection)) {
-      debugLogMutations(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 deleteFromMetaTable(final Connection connection, final Delete d)
-    throws IOException {
-    List<Delete> dels = new ArrayList<>(1);
-    dels.add(d);
-    deleteFromMetaTable(connection, 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 deleteFromMetaTable(final Connection connection, final List<Delete> deletes)
-    throws IOException {
-    try (Table t = getMetaHTable(connection)) {
-      debugLogMutations(deletes);
-      t.delete(deletes);
-    }
+    scanCatalog(connection,
+      TableName.META_TABLE_NAME,
+      startRow,
+      stopRow,
+      type,
+      filter,
+      maxRows,
+      visitor);
   }
 
   /**
@@ -741,321 +85,9 @@ public final class MetaTableAccessor {
    * @param connection connection we're using to access meta table
    */
   public static void removeRegionReplicasFromMeta(Set<byte[]> metaRows,
-    int replicaIndexToDeleteFrom, int numReplicasToRemove, Connection connection)
-    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);
-      }
-
-      deleteFromMetaTable(connection, 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());
-    putsToMetaTable(connection, 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 {
-    try (Table meta = getMetaHTable(connection)) {
-      Put put = makePutFromRegionInfo(regionInfo, EnvironmentEdgeManager.currentTime());
-      addDaughtersToPut(put, splitA, splitB);
-      meta.put(put);
-      debugLogMutation(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
-   */
-  private 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);
-      }
-    }
-    putsToMetaTable(connection, puts);
-    LOG.info("Added {} regions to meta.", puts.size());
-  }
-
-  /**
-   * 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);
-    deleteFromMetaTable(connection, delete);
-    LOG.info("Deleted table " + table + " state from META");
-  }
-
-  /**
-   * 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);
-  }
-
-  /**
-   * Deletes the specified region from META.
-   * @param connection connection we're using
-   * @param regionInfo region to be deleted from META
-   */
-  public static void deleteRegionInfo(Connection connection, RegionInfo regionInfo)
-    throws IOException {
-    Delete delete = new Delete(regionInfo.getRegionName());
-    delete.addFamily(HConstants.CATALOG_FAMILY, HConstants.LATEST_TIMESTAMP);
-    deleteFromMetaTable(connection, delete);
-    LOG.info("Deleted " + regionInfo.getRegionNameAsString());
-  }
-
-  /**
-   * Deletes the specified regions from META.
-   * @param connection connection we're using
-   * @param regionsInfo list of regions to be deleted from META
-   */
-  public static void deleteRegionInfos(Connection connection, List<RegionInfo> regionsInfo)
-    throws IOException {
-    deleteRegionInfos(connection, regionsInfo, EnvironmentEdgeManager.currentTime());
-  }
-
-  /**
-   * Deletes the specified regions from META.
-   * @param connection connection we're using
-   * @param regionsInfo list of regions to be deleted from META
-   */
-  private static void deleteRegionInfos(Connection connection, List<RegionInfo> regionsInfo,
-    long ts) throws IOException {
-    List<Delete> deletes = new ArrayList<>(regionsInfo.size());
-    for (RegionInfo hri : regionsInfo) {
-      Delete e = new Delete(hri.getRegionName());
-      e.addFamily(HConstants.CATALOG_FAMILY, ts);
-      deletes.add(e);
-    }
-    deleteFromMetaTable(connection, deletes);
-    LOG.info("Deleted {} regions from META", regionsInfo.size());
-    LOG.debug("Deleted regions: {}", regionsInfo);
-  }
-
-  /**
-   * Overwrites the specified regions from hbase:meta. Deletes old rows for the given regions and
-   * adds new ones. Regions added back have state CLOSED.
-   * @param connection connection we're using
-   * @param regionInfos list of regions to be added to META
-   */
-  public static void overwriteRegions(Connection connection, List<RegionInfo> regionInfos,
-    int regionReplication) throws IOException {
-    // use master time for delete marker and the Put
-    long now = EnvironmentEdgeManager.currentTime();
-    deleteRegionInfos(connection, regionInfos, now);
-    // Why sleep? This is the easiest way to ensure that the previous deletes does not
-    // eclipse the following puts, that might happen in the same ts from the server.
-    // See HBASE-9906, and HBASE-9879. Once either HBASE-9879, HBASE-8770 is fixed,
-    // or HBASE-9905 is fixed and meta uses seqIds, we do not need the sleep.
-    //
-    // HBASE-13875 uses master timestamp for the mutations. The 20ms sleep is not needed
-    addRegionsToMeta(connection, regionInfos, regionReplication, now + 1);
-    LOG.info("Overwritten " + regionInfos.size() + " regions to Meta");
-    LOG.debug("Overwritten regions: {} ", regionInfos);
-  }
-
-
-  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(List<? extends Mutation> mutations) throws IOException {
-    if (!METALOG.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(mutation);
-    }
-  }
-
-  private static void debugLogMutation(Mutation p) throws IOException {
-    METALOG.debug("{} {}", p.getClass().getSimpleName(), p.toJSON());
+      int replicaIndexToDeleteFrom, int numReplicasToRemove, Connection connection)
+      throws IOException {
+    removeRegionReplicasFromCatalog(metaRows, replicaIndexToDeleteFrom, numReplicasToRemove,
+        connection, TableName.META_TABLE_NAME);
   }
 }
diff --git a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java
index c86a60e..2b1fc67 100644
--- a/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java
+++ b/hbase-balancer/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java
@@ -35,6 +35,19 @@ public enum ServerState {
   CRASHED,
 
   /**
+   * Only server which carries root can have this state. We will split wal for root and then
+   * assign root first before splitting other wals.
+   */
+  SPLITTING_ROOT,
+
+  /**
+   * Indicate that the root splitting is done. We need this state so that the UnassignProcedure
+   * for root can safely quit. See the comments in UnassignProcedure.remoteCallFailed for more
+   * details.
+   */
+  SPLITTING_ROOT_DONE,
+
+  /**
    * Only server which carries meta can have this state. We will split wal for meta and then
    * assign meta first before splitting other wals.
    */
@@ -57,4 +70,4 @@ public enum ServerState {
    * quit. See the comments in UnassignProcedure.remoteCallFailed for more details.
    */
   OFFLINE
-}
\ No newline at end of file
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClientMetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClientMetaTableAccessor.java
index ecc6573..972b36c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClientMetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClientMetaTableAccessor.java
@@ -193,15 +193,15 @@ public final class 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
-   *         {@link CompletableFuture}.
+   *         {@link CompletableFuture}.META_TABLE_NAME
    */
   private static CompletableFuture<List<Pair<RegionInfo, ServerName>>> getTableRegionsAndLocations(
     final AsyncTable<AdvancedScanResultConsumer> metaTable, final TableName tableName,
     final boolean excludeOfflinedSplitParents) {
     CompletableFuture<List<Pair<RegionInfo, ServerName>>> future = new CompletableFuture<>();
-    if (TableName.META_TABLE_NAME.equals(tableName)) {
+    if (TableName.ROOT_TABLE_NAME.equals(tableName)) {
       future.completeExceptionally(new IOException(
-        "This method can't be used to locate meta regions;" + " use MetaTableLocator instead"));
+        "This method can't be used to locate meta regions;" + " use RootTableLocator instead"));
     }
 
     // Make a version of CollectingVisitor that collects RegionInfo and ServerAddress
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllRootRegionsOnlineException.java
similarity index 87%
copy from hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java
copy to hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllRootRegionsOnlineException.java
index c51fccb..6bf5da3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllRootRegionsOnlineException.java
@@ -25,19 +25,19 @@ import org.apache.yetus.audience.InterfaceAudience;
  * 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 {
   private static final long serialVersionUID = 6439786157874827523L;
   /**
    * default constructor
    */
-  public NotAllMetaRegionsOnlineException() {
+  public NotAllRootRegionsOnlineException() {
     super();
   }
 
   /**
    * @param message
    */
-  public NotAllMetaRegionsOnlineException(String message) {
+  public NotAllRootRegionsOnlineException(String message) {
     super(message);
   }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonRootRegionLocator.java
similarity index 92%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonRootRegionLocator.java
index b202168..5e9360b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonRootRegionLocator.java
@@ -23,6 +23,7 @@ import static org.apache.hadoop.hbase.HConstants.NINES;
 import static org.apache.hadoop.hbase.HConstants.USE_META_REPLICAS;
 import static org.apache.hadoop.hbase.HConstants.ZEROES;
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.apache.hadoop.hbase.TableName.ROOT_TABLE_NAME;
 import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.canUpdateOnError;
 import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.createRegionLocations;
 import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.isGood;
@@ -30,11 +31,11 @@ import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.removeRegi
 import static org.apache.hadoop.hbase.client.ConnectionUtils.createClosestRowAfter;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
 import static org.apache.hadoop.hbase.client.RegionInfo.createRegionName;
-import static org.apache.hadoop.hbase.util.Bytes.BYTES_COMPARATOR;
 import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
@@ -51,6 +52,7 @@ import org.apache.hadoop.hbase.CatalogFamilyFormat;
 import org.apache.hadoop.hbase.HBaseIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -68,9 +70,9 @@ import org.apache.hbase.thirdparty.com.google.common.base.Objects;
  * The asynchronous locator for regions other than meta.
  */
 @InterfaceAudience.Private
-class AsyncNonMetaRegionLocator {
+class AsyncNonRootRegionLocator {
 
-  private static final Logger LOG = LoggerFactory.getLogger(AsyncNonMetaRegionLocator.class);
+  private static final Logger LOG = LoggerFactory.getLogger(AsyncNonRootRegionLocator.class);
 
   @VisibleForTesting
   static final String MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE =
@@ -121,14 +123,22 @@ class AsyncNonMetaRegionLocator {
 
   private static final class TableCache {
 
-    private final ConcurrentNavigableMap<byte[], RegionLocations> cache =
-      new ConcurrentSkipListMap<>(BYTES_COMPARATOR);
+    private final ConcurrentNavigableMap<byte[], RegionLocations> cache;
 
     private final Set<LocateRequest> pendingRequests = new HashSet<>();
 
     private final Map<LocateRequest, CompletableFuture<RegionLocations>> allRequests =
       new LinkedHashMap<>();
 
+    private TableCache(TableName tableName) {
+      final KeyValue.KVComparator comparator = getComparator(tableName);
+      cache = new ConcurrentSkipListMap<>(new Comparator<byte[]>() {
+        @Override public int compare(byte[] left, byte[] right) {
+          return comparator.compareRows(left, 0, left.length, right, 0, right.length);
+        }
+      });
+    }
+
     public boolean hasQuota(int max) {
       return pendingRequests.size() < max;
     }
@@ -174,10 +184,14 @@ class AsyncNonMetaRegionLocator {
         // 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);
         completed = c == 0 || ((c > 0 || Bytes.equals(EMPTY_END_ROW, endKey)) &&
-          Bytes.compareTo(loc.getRegion().getStartKey(), req.row) < 0);
+          comparator.compareRows(
+            loc.getRegion().getStartKey(), 0, loc.getRegion().getStartKey().length,
+            req.row, 0, req.row.length) < 0);
       } else {
         completed = loc.getRegion().containsRow(req.row);
       }
@@ -190,7 +204,7 @@ class AsyncNonMetaRegionLocator {
     }
   }
 
-  AsyncNonMetaRegionLocator(AsyncConnectionImpl conn) {
+  AsyncNonRootRegionLocator(AsyncConnectionImpl conn) {
     this.conn = conn;
     this.maxConcurrentLocateRequestPerTable = conn.getConfiguration().getInt(
       MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE, DEFAULT_MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE);
@@ -201,7 +215,7 @@ class AsyncNonMetaRegionLocator {
   }
 
   private TableCache getTableCache(TableName tableName) {
-    return computeIfAbsent(cache, tableName, TableCache::new);
+    return computeIfAbsent(cache, tableName, ()->  new TableCache(tableName));
   }
 
   private boolean isEqual(RegionLocations locs1, RegionLocations locs2) {
@@ -369,8 +383,10 @@ class AsyncNonMetaRegionLocator {
       recordCacheMiss();
       return null;
     }
+    KeyValue.KVComparator comparator = getComparator(tableName);
     byte[] endKey = loc.getRegion().getEndKey();
-    if (isEmptyStopRow(endKey) || Bytes.compareTo(row, endKey) < 0) {
+    if (isEmptyStopRow(endKey) ||
+      comparator.compareRows(row, 0, row.length, endKey, 0, endKey.length) < 0) {
       if (LOG.isTraceEnabled()) {
         LOG.trace("Found {} in cache for {}, row='{}', locateType={}, replicaId={}", loc, tableName,
           Bytes.toStringBinary(row), RegionLocateType.CURRENT, replicaId);
@@ -398,8 +414,12 @@ class AsyncNonMetaRegionLocator {
       recordCacheMiss();
       return null;
     }
+    KeyValue.KVComparator comparator = getComparator(tableName);
     if (isEmptyStopRow(loc.getRegion().getEndKey()) ||
-      (!isEmptyStopRow && Bytes.compareTo(loc.getRegion().getEndKey(), row) >= 0)) {
+      (!isEmptyStopRow &&
+        comparator.compareRows(
+          loc.getRegion().getEndKey(), 0, loc.getRegion().getEndKey().length,
+          row, 0, row.length) >= 0)) {
       if (LOG.isTraceEnabled()) {
         LOG.trace("Found {} in cache for {}, row='{}', locateType={}, replicaId={}", loc, tableName,
           Bytes.toStringBinary(row), RegionLocateType.BEFORE, replicaId);
@@ -413,9 +433,11 @@ class AsyncNonMetaRegionLocator {
   }
 
   private void locateInMeta(TableName tableName, LocateRequest req) {
+    TableName parentTableName =
+      META_TABLE_NAME.equals(tableName) ? ROOT_TABLE_NAME : META_TABLE_NAME;
     if (LOG.isTraceEnabled()) {
       LOG.trace("Try locate '" + tableName + "', row='" + Bytes.toStringBinary(req.row) +
-        "', locateType=" + req.locateType + " in meta");
+        "', locateType=" + req.locateType + " in "+parentTableName);
     }
     byte[] metaStartKey;
     if (req.locateType.equals(RegionLocateType.BEFORE)) {
@@ -436,7 +458,7 @@ class AsyncNonMetaRegionLocator {
     if (useMetaReplicas) {
       scan.setConsistency(Consistency.TIMELINE);
     }
-    conn.getTable(META_TABLE_NAME).scan(scan, new AdvancedScanResultConsumer() {
+    conn.getTable(parentTableName).scan(scan, new AdvancedScanResultConsumer() {
 
       private boolean completeNormally = false;
 
@@ -667,4 +689,14 @@ class AsyncNonMetaRegionLocator {
     }
     return tableCache.cache.values().stream().mapToInt(RegionLocations::numNonNullElements).sum();
   }
+
+  private static KeyValue.KVComparator getComparator(TableName tableName) {
+    if (TableName.ROOT_TABLE_NAME.equals(tableName)) {
+      return KeyValue.ROOT_COMPARATOR;
+    }
+    if (META_TABLE_NAME.equals(tableName)) {
+      return KeyValue.META_COMPARATOR;
+    }
+    return KeyValue.COMPARATOR;
+  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
index 09eabfc..39150b7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.apache.hadoop.hbase.TableName.ROOT_TABLE_NAME;
 import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 
 import java.util.concurrent.CompletableFuture;
@@ -50,14 +50,14 @@ class AsyncRegionLocator {
 
   private final AsyncConnectionImpl conn;
 
-  private final AsyncMetaRegionLocator metaRegionLocator;
+  private final AsyncRootRegionLocator metaRegionLocator;
 
-  private final AsyncNonMetaRegionLocator nonMetaRegionLocator;
+  private final AsyncNonRootRegionLocator nonMetaRegionLocator;
 
   AsyncRegionLocator(AsyncConnectionImpl conn, HashedWheelTimer retryTimer) {
     this.conn = conn;
-    this.metaRegionLocator = new AsyncMetaRegionLocator(conn.registry);
-    this.nonMetaRegionLocator = new AsyncNonMetaRegionLocator(conn);
+    this.metaRegionLocator = new AsyncRootRegionLocator(conn.registry);
+    this.nonMetaRegionLocator = new AsyncNonRootRegionLocator(conn);
     this.retryTimer = retryTimer;
   }
 
@@ -81,13 +81,17 @@ class AsyncRegionLocator {
     return future;
   }
 
+  private boolean isRoot(TableName tableName) {
+    return TableName.isRootTableName(tableName);
+  }
+
   private boolean isMeta(TableName tableName) {
     return TableName.isMetaTableName(tableName);
   }
 
   CompletableFuture<RegionLocations> getRegionLocations(TableName tableName, byte[] row,
       RegionLocateType type, boolean reload, long timeoutNs) {
-    CompletableFuture<RegionLocations> future = isMeta(tableName)
+    CompletableFuture<RegionLocations> future = isRoot(tableName)
       ? metaRegionLocator.getRegionLocations(RegionReplicaUtil.DEFAULT_REPLICA_ID, reload)
       : nonMetaRegionLocator.getRegionLocations(tableName, row,
         RegionReplicaUtil.DEFAULT_REPLICA_ID, type, reload);
@@ -103,7 +107,7 @@ class AsyncRegionLocator {
     // Change it later if the meta table can have more than one regions.
     CompletableFuture<HRegionLocation> future = new CompletableFuture<>();
     CompletableFuture<RegionLocations> locsFuture =
-      isMeta(tableName) ? metaRegionLocator.getRegionLocations(replicaId, reload)
+      isRoot(tableName) ? metaRegionLocator.getRegionLocations(replicaId, reload)
         : nonMetaRegionLocator.getRegionLocations(tableName, row, replicaId, type, reload);
     addListener(locsFuture, (locs, error) -> {
       if (error != null) {
@@ -147,7 +151,7 @@ class AsyncRegionLocator {
   }
 
   void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) {
-    if (loc.getRegion().isMetaRegion()) {
+    if (loc.getRegion().isRootRegion()) {
       metaRegionLocator.updateCachedLocationOnError(loc, exception);
     } else {
       nonMetaRegionLocator.updateCachedLocationOnError(loc, exception);
@@ -156,7 +160,7 @@ class AsyncRegionLocator {
 
   void clearCache(TableName tableName) {
     LOG.debug("Clear meta cache for {}", tableName);
-    if (tableName.equals(META_TABLE_NAME)) {
+    if (tableName.equals(ROOT_TABLE_NAME)) {
       metaRegionLocator.clearCache();
     } else {
       nonMetaRegionLocator.clearCache(tableName);
@@ -175,15 +179,14 @@ class AsyncRegionLocator {
     nonMetaRegionLocator.clearCache();
   }
 
-  @VisibleForTesting
-  AsyncNonMetaRegionLocator getNonMetaRegionLocator() {
+  @VisibleForTesting AsyncNonRootRegionLocator getNonMetaRegionLocator() {
     return nonMetaRegionLocator;
   }
 
   // only used for testing whether we have cached the location for a region.
   @VisibleForTesting
   RegionLocations getRegionLocationInCache(TableName tableName, byte[] row) {
-    if (TableName.isMetaTableName(tableName)) {
+    if (TableName.isRootTableName(tableName)) {
       return metaRegionLocator.getRegionLocationInCache();
     } else {
       return nonMetaRegionLocator.getRegionLocationInCache(tableName, row);
@@ -193,7 +196,7 @@ class AsyncRegionLocator {
   // only used for testing whether we have cached the location for a table.
   @VisibleForTesting
   int getNumberOfCachedRegionLocations(TableName tableName) {
-    if (TableName.isMetaTableName(tableName)) {
+    if (TableName.isRootTableName(tableName)) {
       return metaRegionLocator.getNumberOfCachedRegionLocations();
     } else {
       return nonMetaRegionLocator.getNumberOfCachedRegionLocations(tableName);
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRootRegionLocator.java
similarity index 98%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRootRegionLocator.java
index 3571f960..6dff531 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRootRegionLocator.java
@@ -36,7 +36,7 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
  * The asynchronous locator for meta region.
  */
 @InterfaceAudience.Private
-class AsyncMetaRegionLocator {
+class AsyncRootRegionLocator {
 
   private final ConnectionRegistry registry;
 
@@ -45,7 +45,7 @@ class AsyncMetaRegionLocator {
   private final AtomicReference<CompletableFuture<RegionLocations>> metaRelocateFuture =
     new AtomicReference<>();
 
-  AsyncMetaRegionLocator(ConnectionRegistry registry) {
+  AsyncRootRegionLocator(ConnectionRegistry registry) {
     this.registry = registry;
   }
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
index ad6a051..c1cc6b2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
@@ -54,10 +54,14 @@ class AsyncTableRegionLocatorImpl implements AsyncTableRegionLocator {
 
   @Override
   public CompletableFuture<List<HRegionLocation>> getAllRegionLocations() {
-    if (TableName.isMetaTableName(tableName)) {
+    if (TableName.isRootTableName(tableName)) {
       return conn.registry.getMetaRegionLocations()
         .thenApply(locs -> Arrays.asList(locs.getRegionLocations()));
     }
+    if (TableName.isMetaTableName(tableName)) {
+      return ClientMetaTableAccessor
+        .getTableHRegionLocations(conn.getTable(TableName.ROOT_TABLE_NAME), tableName);
+    }
     return ClientMetaTableAccessor
       .getTableHRegionLocations(conn.getTable(TableName.META_TABLE_NAME), tableName);
   }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java
index 5d48991..cd57a41 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MutableRegionInfo.java
@@ -224,9 +224,19 @@ class MutableRegionInfo implements RegionInfo {
    */
   @Override
   public boolean containsRow(byte[] row) {
-    return Bytes.compareTo(row, startKey) >= 0 &&
-      (Bytes.compareTo(row, endKey) < 0 ||
-       Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY));
+    return containsRow(row, 0, (short)row.length);
+  }
+
+  @Override
+  public boolean containsRow(byte[] row, int offset, short length) {
+    return Bytes.compareTo(row, offset, length, startKey, 0, startKey.length) >= 0 &&
+      (Bytes.compareTo(row, offset, length, endKey, 0, endKey.length) < 0 ||
+        Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY));
+  }
+
+  @Override
+  public boolean isRootRegion() {
+    return tableName.equals(TableName.ROOT_TABLE_NAME);
   }
 
   /** @return true if this region is a meta region */
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index d740a3a..2554c19 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.HConstants.HIGH_QOS;
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.apache.hadoop.hbase.TableName.ROOT_TABLE_NAME;
 import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 import static org.apache.hadoop.hbase.util.FutureUtils.unwrapCompletionException;
 
@@ -354,6 +355,8 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
 
   private final HashedWheelTimer retryTimer;
 
+  private final AsyncTable<AdvancedScanResultConsumer> rootTable;
+
   private final AsyncTable<AdvancedScanResultConsumer> metaTable;
 
   private final long rpcTimeoutNs;
@@ -374,6 +377,7 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
       AsyncAdminBuilderBase builder) {
     this.connection = connection;
     this.retryTimer = retryTimer;
+    this.rootTable = connection.getTable(ROOT_TABLE_NAME);
     this.metaTable = connection.getTable(META_TABLE_NAME);
     this.rpcTimeoutNs = builder.rpcTimeoutNs;
     this.operationTimeoutNs = builder.operationTimeoutNs;
@@ -2391,32 +2395,46 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
       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
+        if (encodedName.equals(RegionInfoBuilder.ROOT_REGIONINFO.getEncodedName())) {
           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) {
@@ -2424,9 +2442,33 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
           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()) {
+              LOG.debug(
+                "Didn't find encoded name in hbase:meta, trying hbase:root for region :" +
+                  Bytes.toStringBinary(regionNameOrEncodedRegionName));
+            }
+            CompletableFuture<Optional<HRegionLocation>> innerfuture =
+              ClientMetaTableAccessor.getRegionLocationWithEncodedName(rootTable,
+                regionNameOrEncodedRegionName);
+            addListener(innerfuture, (innerlocation, innererr) -> {
+              if (innererr != null) {
+                returnedFuture.completeExceptionally(innererr);
+                return;
+              }
+              if (!innerlocation.isPresent() || innerlocation.get().getRegion() == null) {
+                  returnedFuture.completeExceptionally(new UnknownRegionException(
+                    "Invalid region name or encoded region name: " +
+                      Bytes.toStringBinary(regionNameOrEncodedRegionName)));
+              } else {
+                returnedFuture.complete(innerlocation.get());
+              }
+            });
+          } else {
+            returnedFuture.completeExceptionally(new UnknownRegionException(
+              "Invalid region name or encoded region name: " +
+                Bytes.toStringBinary(regionNameOrEncodedRegionName)));
+          }
         } else {
           returnedFuture.complete(location.get());
         }
@@ -2449,6 +2491,13 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
     }
 
     if (Bytes.equals(regionNameOrEncodedRegionName,
+      RegionInfoBuilder.ROOT_REGIONINFO.getRegionName()) ||
+      Bytes.equals(regionNameOrEncodedRegionName,
+        RegionInfoBuilder.ROOT_REGIONINFO.getEncodedNameAsBytes())) {
+      return CompletableFuture.completedFuture(RegionInfoBuilder.ROOT_REGIONINFO);
+    }
+
+    if (Bytes.equals(regionNameOrEncodedRegionName,
       RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()) ||
       Bytes.equals(regionNameOrEncodedRegionName,
         RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
index 7a3a9af..acc2102 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
@@ -27,6 +27,7 @@ import java.util.Comparator;
 import java.util.List;
 import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.util.ByteArrayHashKey;
@@ -102,6 +103,7 @@ public interface RegionInfo extends Comparable<RegionInfo> {
   @InterfaceAudience.Private
   Comparator<RegionInfo> COMPARATOR
     = (RegionInfo lhs, RegionInfo rhs) -> {
+      KeyValue.KVComparator comparator = getComparator(rhs.getTable());
       if (rhs == null) {
         return 1;
       }
@@ -113,13 +115,17 @@ public interface RegionInfo extends Comparable<RegionInfo> {
       }
 
       // Compare start keys.
-      result = Bytes.compareTo(lhs.getStartKey(), rhs.getStartKey());
+      result = comparator.compareRows(
+        lhs.getStartKey(), 0, lhs.getStartKey().length,
+        rhs.getStartKey(), 0, rhs.getStartKey().length);
       if (result != 0) {
         return result;
       }
 
       // Compare end keys.
-      result = Bytes.compareTo(lhs.getEndKey(), rhs.getEndKey());
+      result = comparator.compareRows(
+        lhs.getEndKey(), 0, lhs.getEndKey().length,
+        rhs.getEndKey(), 0, rhs.getEndKey().length);
 
       if (result != 0) {
         if (lhs.getStartKey().length != 0
@@ -227,6 +233,11 @@ public interface RegionInfo extends Comparable<RegionInfo> {
   /**
    * @return true if this region is a meta region.
    */
+  boolean isRootRegion();
+
+  /**
+   * @return true if this region is a meta region.
+   */
   boolean isMetaRegion();
 
   /**
@@ -244,6 +255,11 @@ public interface RegionInfo extends Comparable<RegionInfo> {
   boolean containsRow(byte[] row);
 
   /**
+   * @return true if the given row falls in this region.
+   */
+  boolean containsRow(byte[] row, int offset, short length);
+
+  /**
    * Does region name contain its encoded name?
    * @param regionName region name
    * @return boolean indicating if this a new format region
@@ -422,6 +438,7 @@ public interface RegionInfo extends Comparable<RegionInfo> {
    * @return true if two regions are adjacent
    */
   static boolean areAdjacent(RegionInfo regionA, RegionInfo regionB) {
+    KeyValue.KVComparator comparator = getComparator(regionA.getTable());
     if (regionA == null || regionB == null) {
       throw new IllegalArgumentException(
       "Can't check whether adjacent for null region");
@@ -431,11 +448,13 @@ public interface RegionInfo extends Comparable<RegionInfo> {
     }
     RegionInfo a = regionA;
     RegionInfo b = regionB;
-    if (Bytes.compareTo(a.getStartKey(), b.getStartKey()) > 0) {
+    if (comparator.compareRows(a.getStartKey(), 0, a.getStartKey().length,
+      b.getStartKey(), 0, b.getStartKey().length) > 0) {
       a = regionB;
       b = regionA;
     }
-    return Bytes.equals(a.getEndKey(), b.getStartKey());
+    return comparator.compareRows(a.getEndKey(), 0, a.getEndKey().length,
+      b.getStartKey(), 0, b.getStartKey().length) == 0;
   }
 
   /**
@@ -803,13 +822,15 @@ public interface RegionInfo extends Comparable<RegionInfo> {
    * @see #isDegenerate()
    */
   default boolean isOverlap(RegionInfo other) {
+    KeyValue.KVComparator comparator = getComparator(other.getTable());
     if (other == null) {
       return false;
     }
     if (!getTable().equals(other.getTable())) {
       return false;
     }
-    int startKeyCompare = Bytes.compareTo(getStartKey(), other.getStartKey());
+    int startKeyCompare = comparator.compareRows(getStartKey(), 0, getStartKey().length,
+      other.getStartKey(), 0, other.getStartKey().length);
     if (startKeyCompare == 0) {
       return true;
     }
@@ -817,15 +838,27 @@ public interface RegionInfo extends Comparable<RegionInfo> {
       if (isLast()) {
         return true;
       }
-      return Bytes.compareTo(getEndKey(), other.getStartKey()) > 0;
+      return comparator.compareRows(getEndKey(), 0, getEndKey().length,
+        other.getStartKey(), 0, other.getStartKey().length) > 0;
     }
     if (other.isLast()) {
       return true;
     }
-    return Bytes.compareTo(getStartKey(), other.getEndKey()) < 0;
+    return comparator.compareRows(getStartKey(), 0, getStartKey().length,
+      other.getEndKey(), 0, other.getEndKey().length) < 0;
   }
 
   default int compareTo(RegionInfo other) {
     return RegionInfo.COMPARATOR.compare(this, other);
   }
+
+  static KeyValue.KVComparator getComparator(TableName tableName) {
+    if (tableName.equals(TableName.ROOT_TABLE_NAME)) {
+      return KeyValue.ROOT_COMPARATOR;
+    }
+    if (tableName.equals(TableName.META_TABLE_NAME)) {
+      return KeyValue.META_COMPARATOR;
+    }
+    return KeyValue.COMPARATOR;
+  }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
index 8f045e0..69939f6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
@@ -34,6 +34,25 @@ public class RegionInfoBuilder {
   public static final String NO_HASH = null;
 
   /**
+   * 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);
+
+
+  /**
    * RegionInfo for first meta region
    * You cannot use this builder to make an instance of the {@link #FIRST_META_REGIONINFO}.
    * Just refer to this instance. Also, while the instance is actually a MutableRI, its type is
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
index a452387..80bdb8c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptor.java
@@ -271,6 +271,20 @@ public interface TableDescriptor {
    *
    * @return true if this table is <code> hbase:meta </code> region
    */
+  boolean isRootRegion();
+
+  /**
+   * Checks if the table is a <code>hbase:meta</code> table
+   *
+   * @return true if table is <code> hbase:meta </code> region.
+   */
+  boolean isRootTable();
+
+  /**
+   * Checks if this table is <code> hbase:meta </code> region.
+   *
+   * @return true if this table is <code> hbase:meta </code> region
+   */
   boolean isMetaRegion();
 
   /**
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
index 1328f7d..52a5d0f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableDescriptorBuilder.java
@@ -124,6 +124,16 @@ public class TableDescriptorBuilder {
   @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
+    = new Bytes(Bytes.toBytes(IS_ROOT));
+
   /**
    * Used by rest interface to access this metadata attribute
    * which denotes if it is a catalog table, either <code> hbase:meta </code>.
@@ -250,6 +260,7 @@ public class TableDescriptorBuilder {
     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);
   }
 
@@ -624,6 +635,8 @@ public class TableDescriptorBuilder {
       this.name = name;
       families.forEach(c -> this.families.put(c.getName(), ColumnFamilyDescriptorBuilder.copy(c)));
       this.values.putAll(values);
+      this.values.put(IS_ROOT_KEY,
+        new Bytes(Bytes.toBytes(Boolean.toString(name.equals(TableName.ROOT_TABLE_NAME)))));
       this.values.put(IS_META_KEY,
         new Bytes(Bytes.toBytes(Boolean.toString(name.equals(TableName.META_TABLE_NAME)))));
     }
@@ -634,6 +647,26 @@ public class TableDescriptorBuilder {
      * @return true if this table is <code> hbase:meta </code> region
      */
     @Override
+    public boolean isRootRegion() {
+      return getOrDefault(IS_ROOT_KEY, Boolean::valueOf, false);
+    }
+
+    /**
+     * Checks if the table is a <code>hbase:meta</code> table
+     *
+     * @return true if table is <code> hbase:meta </code> region.
+     */
+    @Override
+    public boolean isRootTable() {
+      return isRootRegion();
+    }
+
+    /**
+     * Checks if this table is <code> hbase:meta </code> region.
+     *
+     * @return true if this table is <code> hbase:meta </code> region
+     */
+    @Override
     public boolean isMetaRegion() {
       return getOrDefault(IS_META_KEY, Boolean::valueOf, false);
     }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
index 42a4188..c0e09f9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.client.RegionInfo.DEFAULT_REPLICA_ID;
-import static org.apache.hadoop.hbase.client.RegionInfoBuilder.FIRST_META_REGIONINFO;
+import static org.apache.hadoop.hbase.client.RegionInfoBuilder.ROOT_REGIONINFO;
 import static org.apache.hadoop.hbase.client.RegionReplicaUtil.getRegionInfoForDefaultReplica;
 import static org.apache.hadoop.hbase.client.RegionReplicaUtil.getRegionInfoForReplica;
 import static org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.lengthOfPBMagic;
@@ -158,7 +158,7 @@ class ZKConnectionRegistry implements ConnectionRegistry {
             LOG.warn("Meta region is in state " + stateAndServerName.getFirst());
           }
           locs[DEFAULT_REPLICA_ID] = new HRegionLocation(
-            getRegionInfoForDefaultReplica(FIRST_META_REGIONINFO), stateAndServerName.getSecond());
+            getRegionInfoForDefaultReplica(ROOT_REGIONINFO), stateAndServerName.getSecond());
           tryComplete(remaining, locs, future);
         });
       } else {
@@ -180,7 +180,7 @@ class ZKConnectionRegistry implements ConnectionRegistry {
               locs[replicaId] = null;
             } else {
               locs[replicaId] =
-                new HRegionLocation(getRegionInfoForReplica(FIRST_META_REGIONINFO, replicaId),
+                new HRegionLocation(getRegionInfoForReplica(ROOT_REGIONINFO, replicaId),
                   stateAndServerName.getSecond());
             }
           }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index ff20291..d368823 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -3138,7 +3138,7 @@ public final class ProtobufUtil {
    * @return RegionState instance corresponding to the serialized data.
    * @throws DeserializationException if the data is invalid.
    */
-  public static RegionState parseMetaRegionStateFrom(final byte[] data, int replicaId)
+  public static RegionState parseRootRegionStateFrom(final byte[] data, int replicaId)
       throws DeserializationException {
     RegionState.State state = RegionState.State.OPEN;
     ServerName serverName;
@@ -3165,7 +3165,7 @@ public final class ProtobufUtil {
       state = RegionState.State.OFFLINE;
     }
     return new RegionState(RegionReplicaUtil.getRegionInfoForReplica(
-        RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), state, serverName);
+        RegionInfoBuilder.ROOT_REGIONINFO, replicaId), state, serverName);
   }
 
   /**
@@ -3281,7 +3281,10 @@ public final class ProtobufUtil {
     long regionId = proto.getRegionId();
     int defaultReplicaId = org.apache.hadoop.hbase.client.RegionInfo.DEFAULT_REPLICA_ID;
     int replicaId = proto.hasReplicaId()? proto.getReplicaId(): defaultReplicaId;
-    if (tableName.equals(TableName.META_TABLE_NAME) && replicaId == defaultReplicaId) {
+    //TODO francis room to streamline this logic
+    if (tableName.equals(TableName.META_TABLE_NAME)
+        && replicaId == defaultReplicaId && regionId == 1 && !proto.getSplit()
+        && !proto.getOffline()) {
       return RegionInfoBuilder.FIRST_META_REGIONINFO;
     }
     byte[] startKey = null;
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRootRegionLocatorFailFast.java
similarity index 90%
rename from hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java
rename to hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRootRegionLocatorFailFast.java
index b306500..af23fb9 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRootRegionLocatorFailFast.java
@@ -33,15 +33,15 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({ ClientTests.class, SmallTests.class })
-public class TestAsyncMetaRegionLocatorFailFast {
+public class TestAsyncRootRegionLocatorFailFast {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestAsyncMetaRegionLocatorFailFast.class);
+    HBaseClassTestRule.forClass(TestAsyncRootRegionLocatorFailFast.class);
 
   private static Configuration CONF = HBaseConfiguration.create();
 
-  private static AsyncMetaRegionLocator LOCATOR;
+  private static AsyncRootRegionLocator LOCATOR;
 
   private static final class FaultyConnectionRegistry extends DoNothingConnectionRegistry {
 
@@ -57,7 +57,7 @@ public class TestAsyncMetaRegionLocatorFailFast {
 
   @BeforeClass
   public static void setUp() {
-    LOCATOR = new AsyncMetaRegionLocator(new FaultyConnectionRegistry(CONF));
+    LOCATOR = new AsyncRootRegionLocator(new FaultyConnectionRegistry(CONF));
   }
 
   @Test(expected = DoNotRetryIOException.class)
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
index 4af035a..3af76f4 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
@@ -300,8 +300,12 @@ public class CellComparatorImpl implements CellComparator {
    * @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;
   }
 }
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
index b7ddbab..1f089f7 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
@@ -101,6 +101,7 @@ public class KeyValue implements ExtendedCell, Cloneable {
    */
   @Deprecated
   public static final KVComparator COMPARATOR = new KVComparator();
+
   /**
    * A {@link KVComparator} for <code>hbase:meta</code> catalog table
    * {@link KeyValue}s.
@@ -110,6 +111,15 @@ public class KeyValue implements ExtendedCell, Cloneable {
   @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();
+
   /** Size of the key length field in bytes*/
   public static final int KEY_LENGTH_SIZE = Bytes.SIZEOF_INT;
 
@@ -1605,6 +1615,79 @@ public class KeyValue implements ExtendedCell, Cloneable {
   }
 
   /**
+   * 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.
+   */
+  @Deprecated
+  public static class RootComparator extends MetaComparator {
+    public int compareRows(byte [] left, int loffset, int llength,
+      byte [] right, int roffset, int rlength) {
+      // Rows look like this: .META.,ROW_FROM_META,RID
+      //        LOG.info("ROOT " + Bytes.toString(left, loffset, llength) +
+      //          "---" + Bytes.toString(right, roffset, rlength));
+      final int metalength = TableName.META_TABLE_NAME.getName().length+1; // '.META.' length
+      int lmetaOffsetPlusDelimiter = loffset + metalength;
+      int leftFarDelimiter = getDelimiterInReverse(left,
+        lmetaOffsetPlusDelimiter,
+        llength - metalength, HConstants.DELIMITER);
+      int rmetaOffsetPlusDelimiter = roffset + metalength;
+      int rightFarDelimiter = getDelimiterInReverse(right,
+        rmetaOffsetPlusDelimiter, rlength - metalength,
+        HConstants.DELIMITER);
+      if (leftFarDelimiter < 0 && rightFarDelimiter >= 0) {
+        // Nothing between .META. and regionid.  Its first key.
+        return -1;
+      } else if (rightFarDelimiter < 0 && leftFarDelimiter >= 0) {
+        return 1;
+      } else if (leftFarDelimiter < 0 && rightFarDelimiter < 0) {
+        return 0;
+      }
+      int result = super.compareRows(left, lmetaOffsetPlusDelimiter,
+        leftFarDelimiter - lmetaOffsetPlusDelimiter,
+        right, rmetaOffsetPlusDelimiter,
+        rightFarDelimiter - rmetaOffsetPlusDelimiter);
+      if (result != 0) {
+        return result;
+      }
+      // Compare last part of row, the rowid.
+      leftFarDelimiter++;
+      rightFarDelimiter++;
+      result =  Bytes.compareTo(left, leftFarDelimiter,
+        llength - (leftFarDelimiter - loffset),
+        right, rightFarDelimiter, rlength - (rightFarDelimiter - roffset));
+      return result;
+    }
+
+    /**
+     * The HFileV2 file format's trailer contains this class name.  We reinterpret this and
+     * instantiate the appropriate comparator.
+     * TODO: With V3 consider removing this.
+     * @return legacy class name for FileFileTrailer#comparatorClassName
+     */
+    @Override
+    public String getLegacyKeyComparatorName() {
+      return "org.apache.hadoop.hbase.KeyValue$RootKeyComparator";
+    }
+
+    /**
+     * Compare key portion of a {@link KeyValue} for keys in <code>hbase:meta</code>
+     * table.
+     */
+    @Override
+    public int compare(final Cell left, final Cell right) {
+      return PrivateCellUtil.compareKeyIgnoresMvcc(RootCellComparator.ROOT_COMPARATOR, left, right);
+    }
+
+    @Override
+    protected Object clone() throws CloneNotSupportedException {
+      return new RootComparator();
+    }
+  }
+
+
+  /**
    * A {@link KVComparator} for <code>hbase:meta</code> catalog table
    * {@link KeyValue}s.
    * @deprecated : {@link MetaCellComparator#META_COMPARATOR} to be used.
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaCellComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaCellComparator.java
index 4c18cfe..4f85c51 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaCellComparator.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/MetaCellComparator.java
@@ -71,7 +71,7 @@ public class MetaCellComparator extends CellComparatorImpl {
     return ignoreSequenceid ? diff : Longs.compare(b.getSequenceId(), a.getSequenceId());
   }
 
-  private static int compareRows(byte[] left, int loffset, int llength, byte[] right, int roffset,
+  protected static int compareRows(byte[] left, int loffset, int llength, byte[] right, int roffset,
       int rlength) {
     int leftDelimiter = Bytes.searchDelimiterIndex(left, loffset, llength, HConstants.DELIMITER);
     int rightDelimiter = Bytes.searchDelimiterIndex(right, roffset, rlength, HConstants.DELIMITER);
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/RootCellComparator.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/RootCellComparator.java
new file mode 100644
index 0000000..2c5835e
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/RootCellComparator.java
@@ -0,0 +1,81 @@
+/*
+ * 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 java.util.Comparator;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
+
+/**
+ * A {@link CellComparatorImpl} for <code>hbase:meta</code> catalog table
+ * {@link KeyValue}s.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class RootCellComparator extends MetaCellComparator {
+
+  /**
+   * A {@link MetaCellComparator} for <code>hbase:meta</code> catalog table
+   * {@link KeyValue}s.
+   */
+  public static final RootCellComparator ROOT_COMPARATOR = new RootCellComparator();
+
+  protected static int compareRows(byte[] left, int loffset, int llength, byte[] right, int roffset,
+      int rlength) {
+    // Rows look like this: .META.,ROW_FROM_META,RID
+    //        LOG.info("ROOT " + Bytes.toString(left, loffset, llength) +
+    //          "---" + Bytes.toString(right, roffset, rlength));
+    final int metalength = TableName.META_TABLE_NAME.getName().length + 1; // '.META.' length
+    int lmetaOffsetPlusDelimiter = loffset + metalength;
+    int leftFarDelimiter = Bytes
+      .searchDelimiterIndexInReverse(left, lmetaOffsetPlusDelimiter, llength - metalength,
+        HConstants.DELIMITER);
+    int rmetaOffsetPlusDelimiter = roffset + metalength;
+    int rightFarDelimiter = Bytes
+      .searchDelimiterIndexInReverse(right, rmetaOffsetPlusDelimiter, rlength - metalength,
+        HConstants.DELIMITER);
+    if (leftFarDelimiter < 0 && rightFarDelimiter >= 0) {
+      // Nothing between .META. and regionid.  Its first key.
+      return -1;
+    } else if (rightFarDelimiter < 0 && leftFarDelimiter >= 0) {
+      return 1;
+    } else if (leftFarDelimiter < 0 && rightFarDelimiter < 0) {
+      return 0;
+    }
+    int result = org.apache.hadoop.hbase.MetaCellComparator
+      .compareRows(left, lmetaOffsetPlusDelimiter, leftFarDelimiter - lmetaOffsetPlusDelimiter,
+        right, rmetaOffsetPlusDelimiter, rightFarDelimiter - rmetaOffsetPlusDelimiter);
+    if (result != 0) {
+      return result;
+    }
+    // Compare last part of row, the rowid.
+    leftFarDelimiter++;
+    rightFarDelimiter++;
+    result = Bytes.compareTo(left, leftFarDelimiter, llength - (leftFarDelimiter - loffset), right,
+      rightFarDelimiter, rlength - (rightFarDelimiter - roffset));
+    return result;
+  }
+
+  @Override
+  public Comparator getSimpleComparator() {
+    return this;
+  }
+
+}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
index b659d14..30f6172 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
@@ -80,6 +80,10 @@ public final class TableName implements Comparable<TableName> {
          "(?:"+VALID_TABLE_QUALIFIER_REGEX+"))";
 
   /** The hbase:meta table's name. */
+  public static final TableName ROOT_TABLE_NAME =
+    valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "root");
+
+  /** The hbase:meta table's name. */
   public static final TableName META_TABLE_NAME =
       valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta");
 
@@ -102,6 +106,13 @@ public final class TableName implements Comparable<TableName> {
   /**
    * @return True if <code>tn</code> is the hbase:meta table name.
    */
+  public static boolean isRootTableName(final TableName tn) {
+    return tn.equals(TableName.ROOT_TABLE_NAME);
+  }
+
+  /**
+   * @return True if <code>tn</code> is the hbase:meta table name.
+   */
   public static boolean isMetaTableName(final TableName tn) {
     return tn.equals(TableName.META_TABLE_NAME);
   }
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
index 7762330..7bc67ea 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
@@ -166,6 +166,44 @@ public class TestCellComparator {
     assertTrue(c.compare(x, y) < 0);
   }
 
+  /**
+   * Test meta comparisons using our new ByteBufferKeyValue Cell type, the type we use everywhere
+   * in 2.0.
+   */
+  @Test
+  public void testRootComparisons() throws Exception {
+    long now = System.currentTimeMillis();
+
+    // Meta compares
+    Cell aaa = createByteBufferKeyValueFromKeyValue(new KeyValue(
+      Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+
+        ",TestScanMultipleVersions,row_0500,1236020145502,1236020145502"), now));
+    Cell bbb = createByteBufferKeyValueFromKeyValue(new KeyValue(
+      Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+
+        ",TestScanMultipleVersions,,99999999999999,99999999999999"), now));
+    CellComparator c = RootCellComparator.ROOT_COMPARATOR;
+    assertTrue(c.compare(bbb, aaa) < 0);
+
+    Cell ccc = createByteBufferKeyValueFromKeyValue(
+      new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+
+        ",TestScanMultipleVersions,,99999999999999,1236023996656"),
+        Bytes.toBytes("info"), Bytes.toBytes("regioninfo"), 1236024396271L,
+        (byte[])null));
+    assertTrue(c.compare(ccc, bbb) < 0);
+
+    Cell x = createByteBufferKeyValueFromKeyValue(
+      new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+
+        ",TestScanMultipleVersions,row_0500,1236034574162,1236034574162"),
+        Bytes.toBytes("info"), Bytes.toBytes(""), 9223372036854775807L,
+        (byte[])null));
+    Cell y = createByteBufferKeyValueFromKeyValue(
+      new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+
+        ",TestScanMultipleVersions,row_0500,1236034574162,1236034574162"),
+        Bytes.toBytes("info"), Bytes.toBytes("regioninfo"), 1236034574912L,
+        (byte[])null));
+    assertTrue(c.compare(x, y) < 0);
+  }
+
   private static Cell createByteBufferKeyValueFromKeyValue(KeyValue kv) {
     ByteBuffer bb = ByteBuffer.wrap(kv.getBuffer());
     return new ByteBufferKeyValue(bb, 0, bb.remaining());
@@ -206,6 +244,37 @@ public class TestCellComparator {
   }
 
   @Test
+  public void testRootComparisons2() {
+    long now = System.currentTimeMillis();
+    CellComparator c = RootCellComparator.ROOT_COMPARATOR;
+    assertTrue(c.compare(createByteBufferKeyValueFromKeyValue(new KeyValue(
+        Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",a,,0,1"), now)),
+      createByteBufferKeyValueFromKeyValue(new KeyValue(
+        Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",a,,0,1"), now))) == 0);
+    Cell a = createByteBufferKeyValueFromKeyValue(new KeyValue(
+      Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",a,,0,1"), now));
+    Cell b = createByteBufferKeyValueFromKeyValue(new KeyValue(
+      Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",a,,0,2"), now));
+    assertTrue(c.compare(a, b) < 0);
+    assertTrue(c.compare(createByteBufferKeyValueFromKeyValue(new KeyValue(
+        Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",a,,0,2"), now)),
+      createByteBufferKeyValueFromKeyValue(new KeyValue(
+        Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",a,,0,1"), now))) > 0);
+    assertTrue(c.compare(createByteBufferKeyValueFromKeyValue(new KeyValue(
+        Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",,1"), now)),
+      createByteBufferKeyValueFromKeyValue(new KeyValue(
+        Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",,1"), now))) == 0);
+    assertTrue(c.compare(createByteBufferKeyValueFromKeyValue(new KeyValue(
+        Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",,1"), now)),
+      createByteBufferKeyValueFromKeyValue(new KeyValue(
+        Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",,2"), now))) < 0);
+    assertTrue(c.compare(createByteBufferKeyValueFromKeyValue(new KeyValue(
+        Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",,2"), now)),
+      createByteBufferKeyValueFromKeyValue(new KeyValue(
+        Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",,1"), now))) > 0);
+  }
+
+  @Test
   public void testBinaryKeys() throws Exception {
     Set<Cell> set = new TreeSet<>(CellComparatorImpl.COMPARATOR);
     final byte [] fam = Bytes.toBytes("col");
diff --git a/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto
index b51b529..b1e21c6 100644
--- a/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/server/master/MasterProcedure.proto
@@ -290,6 +290,7 @@ message ServerCrashStateData {
   repeated RegionInfo regions_assigned = 4;
   optional bool carrying_meta = 5;
   optional bool should_split_wal = 6 [default = true];
+  optional bool carrying_root = 7;
 }
 
 message RecoverMetaStateData {
@@ -314,6 +315,10 @@ enum ServerCrashState {
   SERVER_CRASH_DELETE_SPLIT_WALS_DIR=13;
   SERVER_CRASH_HANDLE_RIT2 = 20[deprecated=true];
   SERVER_CRASH_FINISH = 100;
+  SERVER_CRASH_SPLIT_ROOT_LOGS = 21;
+  SERVER_CRASH_ASSIGN_ROOT = 22;
+  SERVER_CRASH_CHECK_CARRYING_META = 23;
+  SERVER_CRASH_DELETE_SPLIT_ROOT_WALS_DIR=24;
 }
 
 enum RecoverMetaState {
@@ -488,10 +493,19 @@ message ReopenTableRegionsStateData {
   repeated RegionLocation region = 2;
 }
 
+enum InitRootState{
+  INIT_ROOT_WRITE_FS_LAYOUT = 1;
+  INIT_ROOT_ASSIGN_ROOT = 2;
+  INIT_ROOT_LOAD_ROOT = 3;
+  INIT_ROOT_INIT_META = 4;
+}
+
+message InitRootStateData {
+}
+
 enum InitMetaState {
-  INIT_META_WRITE_FS_LAYOUT = 1;
-  INIT_META_ASSIGN_META = 2;
-  INIT_META_CREATE_NAMESPACES = 3;
+  INIT_META_ASSIGN_META = 1;
+  INIT_META_CREATE_NAMESPACES = 2;
 }
 
 message InitMetaStateData {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
index e1a4949..a8eabbd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
@@ -401,12 +401,22 @@ public class ZkSplitLogWorkerCoordination extends ZKListener implements
       Collections.shuffle(paths);
       // pick meta wal firstly
       int offset = 0;
+      int metaOffset = -1;
+      int rootOffset = -1;
       for (int i = 0; i < paths.size(); i++) {
+        if (AbstractFSWALProvider.isRootFile(paths.get(i))) {
+          offset = rootOffset;
+        }
         if (AbstractFSWALProvider.isMetaFile(paths.get(i))) {
-          offset = i;
-          break;
+          offset = metaOffset;
         }
       }
+      if (rootOffset != -1) {
+        offset = rootOffset;
+      }
+      if (metaOffset != -1) {
+        offset = metaOffset;
+      }
       int numTasks = paths.size();
       boolean taskGrabbed = false;
       for (int i = 0; i < numTasks; i++) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
index cae284a..4c22978 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MetaCellComparator;
+import org.apache.hadoop.hbase.RootCellComparator;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
@@ -596,6 +597,9 @@ public class FixedFileTrailer {
     if (comparator.equals(CellComparatorImpl.class.getName())) {
       return KeyValue.COMPARATOR.getClass().getName();
     }
+    if (comparator.equals(RootCellComparator.class.getName())) {
+      return KeyValue.ROOT_COMPARATOR.getClass().getName();
+    }
     if (comparator.equals(MetaCellComparator.class.getName())) {
       return KeyValue.META_COMPARATOR.getClass().getName();
     }
@@ -611,6 +615,11 @@ public class FixedFileTrailer {
       || comparatorClassName.equals(KeyValue.COMPARATOR.getClass().getName())
       || (comparatorClassName.equals("org.apache.hadoop.hbase.CellComparator"))) {
       comparatorKlass = CellComparatorImpl.class;
+    } else if (comparatorClassName.equals(KeyValue.ROOT_COMPARATOR.getLegacyKeyComparatorName())
+      || comparatorClassName.equals(KeyValue.ROOT_COMPARATOR.getClass().getName())
+      || (comparatorClassName
+      .equals("org.apache.hadoop.hbase.CellComparator$RootCellComparator"))) {
+      comparatorKlass = RootCellComparator.class;
     } else if (comparatorClassName.equals(KeyValue.META_COMPARATOR.getLegacyKeyComparatorName())
       || comparatorClassName.equals(KeyValue.META_COMPARATOR.getClass().getName())
       || (comparatorClassName.equals("org.apache.hadoop.hbase.MetaCellComparator"))) {
@@ -637,6 +646,9 @@ public class FixedFileTrailer {
     } else if (comparatorClassName.equals(
       MetaCellComparator.META_COMPARATOR.getClass().getName())) {
       return MetaCellComparator.META_COMPARATOR;
+    } else if (comparatorClassName.equals(
+      RootCellComparator.ROOT_COMPARATOR.getClass().getName())) {
+      return RootCellComparator.ROOT_COMPARATOR;
     }
     try {
       Class<? extends CellComparator> comparatorClass = getComparatorClass(comparatorClassName);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
index 60aa65d..d093e4d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.MetaCellComparator;
 import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.RootCellComparator;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.crypto.Encryption;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
@@ -379,7 +380,8 @@ public class HFileWriterImpl implements HFile.Writer {
     // If Cells from meta table, don't mess around. meta table Cells have schema
     // (table,startrow,hash) so can't be treated as plain byte arrays. Just skip
     // out without trying to do this optimization.
-    if (comparator instanceof MetaCellComparator) {
+    if (comparator instanceof MetaCellComparator ||
+        comparator instanceof RootCellComparator) {
       return right;
     }
     int diff = comparator.compareRows(left, right);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 022f292..6a62709 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -61,6 +61,7 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CatalogAccessor;
 import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.ClusterId;
 import org.apache.hadoop.hbase.ClusterMetrics;
@@ -72,7 +73,6 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.InvalidFamilyOperationException;
 import org.apache.hadoop.hbase.MasterNotRunningException;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
@@ -123,7 +123,7 @@ import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
 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;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
 import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
@@ -1038,22 +1038,22 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
 
     // Checking if meta needs initializing.
-    status.setStatus("Initializing meta table if this is a new deploy");
-    InitMetaProcedure initMetaProc = null;
-    // Print out state of hbase:meta on startup; helps debugging.
-    RegionState rs = this.assignmentManager.getRegionStates().
-        getRegionState(RegionInfoBuilder.FIRST_META_REGIONINFO);
-    LOG.info("hbase:meta {}", rs);
-    if (rs != null && rs.isOffline()) {
-      Optional<InitMetaProcedure> optProc = procedureExecutor.getProcedures().stream()
-        .filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny();
-      initMetaProc = optProc.orElseGet(() -> {
-        // schedule an init meta procedure if meta has not been deployed yet
-        InitMetaProcedure temp = new InitMetaProcedure();
-        procedureExecutor.submitProcedure(temp);
-        return temp;
+    status.setStatus("Initializing Catalog tables if this is a new deploy");
+    // Print out state of hbase:root on startup; helps debugging.
+    RegionState rootRS = this.assignmentManager.getRegionStates().
+      getRegionState(RegionInfoBuilder.ROOT_REGIONINFO);
+    LOG.info("hbase:root {}", rootRS);
+    InitRootProcedure initRootProc = procedureExecutor.getProcedures().stream()
+      .filter(p -> p instanceof InitRootProcedure && !p.isFinished())
+      .map(o -> (InitRootProcedure) o).findAny()
+      .orElseGet(() -> {
+        if (rootRS != null && rootRS.isOffline()) {
+          InitRootProcedure temp = new InitRootProcedure();
+          procedureExecutor.submitProcedure(temp);
+          return temp;
+        }
+        return null;
       });
-    }
 
     // initialize load balancer
     this.balancer.setMasterServices(this);
@@ -1063,9 +1063,9 @@ public class HMaster extends HRegionServer implements MasterServices {
     // start up all service threads.
     status.setStatus("Initializing master service threads");
     startServiceThreads();
-    // wait meta to be initialized after we start procedure executor
-    if (initMetaProc != null) {
-      initMetaProc.await();
+    // wait catalog tables to be initialized after we start procedure executor
+    if (initRootProc != null) {
+      initRootProc.await();
     }
     // Wake up this server to check in
     sleeper.skipSleepCycle();
@@ -1084,15 +1084,6 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
 
     status.setStatus("Starting assignment manager");
-    // FIRST HBASE:META READ!!!!
-    // The below cannot make progress w/o hbase:meta being online.
-    // This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background
-    // as procedures run -- in particular SCPs for crashed servers... One should put up hbase:meta
-    // if it is down. It may take a while to come online. So, wait here until meta if for sure
-    // available. That's what waitForMetaOnline does.
-    if (!waitForMetaOnline()) {
-      return;
-    }
     this.assignmentManager.joinCluster();
     // The below depends on hbase:meta being online.
     this.tableStateManager.start();
@@ -1160,10 +1151,10 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
 
     assignmentManager.checkIfShouldMoveSystemRegionAsync();
-    status.setStatus("Assign meta replicas");
-    MasterMetaBootstrap metaBootstrap = createMetaBootstrap();
+    status.setStatus("Assign catalog replicas");
+    MasterCatalogBootstrap catalogBootstrap = createCatalogBootstrap();
     try {
-      metaBootstrap.assignMetaReplicas();
+      catalogBootstrap.assignCatalogReplicas();
     } catch (IOException | KeeperException e){
       LOG.error("Assigning meta replica failed: ", e);
     }
@@ -1226,6 +1217,7 @@ public class HMaster extends HRegionServer implements MasterServices {
    *   and we will hold here until operator intervention.
    */
   @VisibleForTesting
+  //TODO francis reconcile this with AM?
   public boolean waitForMetaOnline() {
     return isRegionOnline(RegionInfoBuilder.FIRST_META_REGIONINFO);
   }
@@ -1234,6 +1226,7 @@ public class HMaster extends HRegionServer implements MasterServices {
    * @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).
    */
+  //TODO francis reconcile this with AM?
   private boolean isRegionOnline(RegionInfo ri) {
     RetryCounter rc = null;
     while (!isStopped()) {
@@ -1271,7 +1264,7 @@ public class HMaster extends HRegionServer implements MasterServices {
    */
   private boolean waitForNamespaceOnline() throws IOException {
     TableState nsTableState =
-      MetaTableAccessor.getTableState(getConnection(), TableName.NAMESPACE_TABLE_NAME);
+      CatalogAccessor.getTableState(getConnection(), TableName.NAMESPACE_TABLE_NAME);
     if (nsTableState == null || nsTableState.isDisabled()) {
       // this means we have already migrated the data and disabled or deleted the namespace table,
       // or this is a new deploy which does not have a namespace table from the beginning.
@@ -1285,7 +1278,7 @@ public class HMaster extends HRegionServer implements MasterServices {
     }
     // Else there are namespace regions up in meta. Ensure they are assigned before we go on.
     for (RegionInfo ri : ris) {
-      if (!isRegionOnline(ri)) {
+      if (!assignmentManager.isRegionOnline(ri)) {
         return false;
       }
     }
@@ -1330,10 +1323,10 @@ public class HMaster extends HRegionServer implements MasterServices {
    * </p>
    */
   @VisibleForTesting
-  protected MasterMetaBootstrap createMetaBootstrap() {
+  protected MasterCatalogBootstrap createCatalogBootstrap() {
     // We put this out here in a method so can do a Mockito.spy and stub it out
     // w/ a mocked up MasterMetaBootstrap.
-    return new MasterMetaBootstrap(this);
+    return new MasterCatalogBootstrap(this);
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCatalogBootstrap.java
similarity index 52%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
rename to hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCatalogBootstrap.java
index c676df8..be354f8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCatalogBootstrap.java
@@ -20,15 +20,22 @@ package org.apache.hadoop.hbase.master;
 
 import java.io.IOException;
 import java.util.List;
+import org.apache.hadoop.hbase.CatalogAccessor;
+import org.apache.hadoop.hbase.CatalogFamilyFormat;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
 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.master.assignment.AssignmentManager;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -38,12 +45,12 @@ import org.slf4j.LoggerFactory;
  * Used by the HMaster on startup to split meta logs and assign the meta table.
  */
 @InterfaceAudience.Private
-class MasterMetaBootstrap {
-  private static final Logger LOG = LoggerFactory.getLogger(MasterMetaBootstrap.class);
+class MasterCatalogBootstrap {
+  private static final Logger LOG = LoggerFactory.getLogger(MasterCatalogBootstrap.class);
 
   private final HMaster master;
 
-  public MasterMetaBootstrap(HMaster master) {
+  public MasterCatalogBootstrap(HMaster master) {
     this.master = master;
   }
 
@@ -52,7 +59,7 @@ class MasterMetaBootstrap {
    * TODO: The way this assign runs, nothing but chance to stop all replicas showing up on same
    * server as the hbase:meta region.
    */
-  void assignMetaReplicas()
+  void assignCatalogReplicas()
       throws IOException, InterruptedException, KeeperException {
     int numReplicas = master.getConfiguration().getInt(HConstants.META_REPLICAS_NUM,
            HConstants.DEFAULT_META_REPLICA_NUM);
@@ -61,23 +68,56 @@ class MasterMetaBootstrap {
       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);
+      LOG.debug(hri.getRegionNameAsString() + " replica region state from zookeeper=" + rootState);
+      if (rootServername.equals(rootState.getServerName())) {
+        rootState = null;
+        LOG.info(hri.getRegionNameAsString() +
+          " old location is same as current hbase:root location; setting location as null...");
+      }
+      // These assigns run inline. All is blocked till they complete. Only interrupt is shutting
+      // down hosting server which calls AM#stop.
+      if (rootState != null && rootState.getServerName() != null) {
+        // Try to retain old assignment.
+        assignmentManager.assign(hri, rootState.getServerName());
+      } else {
+        assignmentManager.assign(hri);
+      }
+    }
+
+    Result metaRegionResult = CatalogAccessor.getRegionResult(
+      master.getConnection(), RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName());
+    RegionLocations regionLocations = CatalogFamilyFormat.getRegionLocations(metaRegionResult);
+    ServerName metaRegionServerName =
+        regionLocations.getRegionLocation(0) == null ?
+            null :
+            regionLocations.getRegionLocation(0).getServerName();
+    for (int i = 1; i < numReplicas; i++) {
+      // Get current hbase:meta state for replica from zk.
+      RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
+        RegionInfoBuilder.FIRST_META_REGIONINFO, i);
+      LOG.debug(hri.getRegionNameAsString() + " replica region state from meta=" +
+        regionLocations.getRegionLocation(i));
+      boolean retain = true;
+      ServerName replicaServerName =
+        regionLocations.getRegionLocation(i) == null ?
+            null :
+            regionLocations.getRegionLocation(i).getServerName();
+      if (metaRegionServerName.equals(replicaServerName)) {
+        retain = false;
         LOG.info(hri.getRegionNameAsString() +
           " old location is same as current hbase:meta location; setting location as null...");
       }
       // These assigns run inline. All is blocked till they complete. Only interrupt is shutting
       // down hosting server which calls AM#stop.
-      if (metaState != null && metaState.getServerName() != null) {
+      if (replicaServerName != null && retain) {
         // Try to retain old assignment.
-        assignmentManager.assignAsync(hri, metaState.getServerName());
+        assignmentManager.assignAsync(hri, replicaServerName);
       } else {
         assignmentManager.assignAsync(hri);
       }
@@ -90,18 +130,37 @@ class MasterMetaBootstrap {
     // unassign the unneeded replicas (for e.g., if the previous master was configured
     // with a replication of 3 and now it is 2, we need to unassign the 1 unneeded replica)
     try {
-      List<String> metaReplicaZnodes = zooKeeper.getMetaReplicaNodes();
+      List<String> metaReplicaZnodes = zooKeeper.getRootReplicaNodes();
       for (String metaReplicaZnode : metaReplicaZnodes) {
         int replicaId = zooKeeper.getZNodePaths().getMetaReplicaIdFromZnode(metaReplicaZnode);
         if (replicaId >= numMetaReplicasConfigured) {
-          RegionState r = MetaTableLocator.getMetaRegionState(zooKeeper, replicaId);
-          LOG.info("Closing excess replica of meta region " + r.getRegion());
+          RegionState r = RootTableLocator.getRootRegionState(zooKeeper, replicaId);
+          LOG.info("Closing excess replica of root region " + r.getRegion());
           // send a close and wait for a max of 30 seconds
           ServerManager.closeRegionSilentlyAndWait(master.getAsyncClusterConnection(),
               r.getServerName(), r.getRegion(), 30000);
           ZKUtil.deleteNode(zooKeeper, zooKeeper.getZNodePaths().getZNodeForReplica(replicaId));
         }
       }
+
+      Result metaRegionResult = CatalogAccessor.getRegionResult(
+        master.getConnection(), RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName());
+      RegionLocations regionLocations = CatalogFamilyFormat.getRegionLocations(metaRegionResult);
+      if (regionLocations.size() > numMetaReplicasConfigured) {
+        for (int i = regionLocations.size() - 1; i >= numMetaReplicasConfigured ; i--) {
+          HRegionLocation loc = regionLocations.getRegionLocation(i);
+          LOG.info("Closing excess replica of meta region " + loc.getRegion());
+          // send a close and wait for a max of 30 seconds
+          ServerManager.closeRegionSilentlyAndWait(master.getAsyncClusterConnection(),
+              loc.getServerName(), loc.getRegion(), 30000);
+          CatalogAccessor.removeRegionReplicasFromCatalog(
+              Sets.newHashSet(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()),
+              i,
+              1,
+              master.getConnection(),
+              TableName.ROOT_TABLE_NAME);
+        }
+      }
     } catch (Exception ex) {
       // ignore the exception since we don't want the master to be wedged due to potential
       // issues in the cleanup of the extra regions. We can do that cleanup via hbck or manually
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index dde3f29..d1fc366 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -35,11 +35,11 @@ import java.util.Set;
 import java.util.stream.Collectors;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.CatalogAccessor;
 import org.apache.hadoop.hbase.ClusterMetricsBuilder;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.ServerMetrics;
@@ -116,7 +116,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -1711,10 +1711,10 @@ public class MasterRpcServices extends RSRpcServices implements
           + " actual: " + type);
       }
       Pair<RegionInfo, ServerName> pair =
-        MetaTableAccessor.getRegion(master.getConnection(), regionName);
-      if (Bytes.equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName(), regionName)) {
-        pair = new Pair<>(RegionInfoBuilder.FIRST_META_REGIONINFO,
-          MetaTableLocator.getMetaRegionLocation(master.getZooKeeper()));
+        CatalogAccessor.getRegion(master.getConnection(), regionName);
+      if (Bytes.equals(RegionInfoBuilder.ROOT_REGIONINFO.getRegionName(), regionName)) {
+        pair = new Pair<>(RegionInfoBuilder.ROOT_REGIONINFO,
+          RootTableLocator.getRootRegionLocation(master.getZooKeeper()));
       }
       if (pair == null) {
         throw new UnknownRegionException(Bytes.toString(regionName));
@@ -2522,6 +2522,7 @@ public class MasterRpcServices extends RSRpcServices implements
    *
    * @return previous states of the regions
    */
+  //TODO francis support root here
   @Override
   public SetRegionStateInMetaResponse setRegionStateInMeta(RpcController controller,
     SetRegionStateInMetaRequest request) throws ServiceException {
@@ -2536,21 +2537,22 @@ public class MasterRpcServices extends RSRpcServices implements
           // TODO: actually, a full region name can save a lot on meta scan, improve later.
           encodedName = RegionInfo.encodeRegionName(spec.getValue().toByteArray());
         }
-        RegionInfo info = this.master.getAssignmentManager().loadRegionFromMeta(encodedName);
+        RegionInfo info = this.master.getAssignmentManager().loadRegionFromCatalog(encodedName);
         LOG.trace("region info loaded from meta table: {}", info);
         RegionState prevState =
           this.master.getAssignmentManager().getRegionStates().getRegionState(info);
         RegionState.State newState = RegionState.State.convert(s.getState());
         LOG.info("{} set region={} state from {} to {}", master.getClientIdAuditPrefix(), info,
           prevState.getState(), newState);
-        Put metaPut = MetaTableAccessor.makePutFromRegionInfo(info, System.currentTimeMillis());
+        Put metaPut = CatalogAccessor.makePutFromRegionInfo(info, System.currentTimeMillis());
         metaPut.addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER,
           Bytes.toBytes(newState.name()));
         List<Put> putList = new ArrayList<>();
         putList.add(metaPut);
-        MetaTableAccessor.putsToMetaTable(this.master.getConnection(), putList);
-        // Loads from meta again to refresh AM cache with the new region state
-        this.master.getAssignmentManager().loadRegionFromMeta(encodedName);
+        CatalogAccessor.putsToCatalogTable(this.master.getConnection(),
+            TableName.META_TABLE_NAME, putList);
+        //Loads from meta again to refresh AM cache with the new region state
+        this.master.getAssignmentManager().loadRegionFromCatalog(info.getEncodedName());
         builder.addStates(RegionSpecifierAndState.newBuilder().setRegionSpecifier(spec)
           .setState(prevState.getState().convert()));
       }
@@ -2576,7 +2578,7 @@ public class MasterRpcServices extends RSRpcServices implements
         RegionState regionState = this.master.getAssignmentManager().getRegionStates().
             getRegionState(encodedRegionName);
         ri = regionState == null ?
-          this.master.getAssignmentManager().loadRegionFromMeta(encodedRegionName) :
+          this.master.getAssignmentManager().loadRegionFromCatalog(encodedRegionName) :
             regionState.getRegion();
         break;
       default:
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterStatusServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterStatusServlet.java
index 58e57c4..6a1454b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterStatusServlet.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterStatusServlet.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.tmpl.master.MasterStatusTmpl;
 import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
@@ -80,7 +80,7 @@ public class MasterStatusServlet extends HttpServlet {
   }
 
   private ServerName getMetaLocationOrNull(HMaster master) {
-    return MetaTableLocator.getMetaRegionLocation(master.getZooKeeper());
+    return RootTableLocator.getRootRegionLocation(master.getZooKeeper());
   }
 
   private Map<String, Integer> getFragmentationInfo(
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
index 6001c8f..a6d75cf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
@@ -54,6 +54,16 @@ public class MasterWalManager {
   private static final Logger LOG = LoggerFactory.getLogger(MasterWalManager.class);
 
   /**
+   * Filter *in* WAL files that are for the hbase:root Region.
+   */
+  final static PathFilter ROOT_FILTER = new PathFilter() {
+    @Override
+    public boolean accept(Path p) {
+      return AbstractFSWALProvider.isRootFile(p);
+    }
+  };
+
+  /**
    * Filter *in* WAL files that are for the hbase:meta Region.
    */
   final static PathFilter META_FILTER = new PathFilter() {
@@ -67,10 +77,10 @@ public class MasterWalManager {
    * Filter *out* WAL files that are for the hbase:meta Region; i.e. return user-space WALs only.
    */
   @VisibleForTesting
-  public final static PathFilter NON_META_FILTER = new PathFilter() {
+  public final static PathFilter NON_CATALOG_FILTER = new PathFilter() {
     @Override
     public boolean accept(Path p) {
-      return !AbstractFSWALProvider.isMetaFile(p);
+      return !AbstractFSWALProvider.isCatalogFile(p);
     }
   };
 
@@ -286,6 +296,22 @@ public class MasterWalManager {
   }
 
   /**
+   * Specialized method to handle the splitting for root WAL
+   * @param serverName logs belonging to this server will be split
+   */
+  public void splitRootLog(final ServerName serverName) throws IOException {
+    splitRootLog(Collections.<ServerName>singleton(serverName));
+  }
+
+  /**
+   * Specialized method to handle the splitting for root WAL
+   * @param serverNames logs belonging to these servers will be split
+   */
+  public void splitRootLog(final Set<ServerName> serverNames) throws IOException {
+    splitLog(serverNames, ROOT_FILTER);
+  }
+
+  /**
    * Specialized method to handle the splitting for meta WAL
    * @param serverName logs belonging to this server will be split
    */
@@ -345,7 +371,7 @@ public class MasterWalManager {
   }
 
   public void splitLog(final Set<ServerName> serverNames) throws IOException {
-    splitLog(serverNames, NON_META_FILTER);
+    splitLog(serverNames, NON_CATALOG_FILTER);
   }
 
   /**
@@ -364,7 +390,9 @@ public class MasterWalManager {
     splitTime = EnvironmentEdgeManager.currentTime() - splitTime;
 
     if (this.metricsMasterFilesystem != null) {
-      if (filter == META_FILTER) {
+      if (filter == ROOT_FILTER) {
+        //TODO francis add metrics
+      } else  if (filter == META_FILTER) {
         this.metricsMasterFilesystem.addMetaWALSplit(splitTime, splitLogSize);
       } else {
         this.metricsMasterFilesystem.addSplit(splitTime, splitLogSize);
@@ -382,13 +410,19 @@ public class MasterWalManager {
    * WAL dir can be deleted.
    * @param serverName the server to archive meta log
    */
-  public void archiveMetaLog(final ServerName serverName) {
+  public void archiveCatalogLog(final ServerName serverName, boolean isRoot) {
+    String regionType = "meta";
+    PathFilter filterType = META_FILTER;
+    if (isRoot) {
+      regionType = "root";
+      filterType = ROOT_FILTER;
+    }
     try {
       Path logDir = new Path(this.rootDir,
           AbstractFSWALProvider.getWALDirectoryName(serverName.toString()));
       Path splitDir = logDir.suffix(AbstractFSWALProvider.SPLITTING_EXT);
       if (fs.exists(splitDir)) {
-        FileStatus[] logfiles = CommonFSUtils.listStatus(fs, splitDir, META_FILTER);
+        FileStatus[] logfiles = CommonFSUtils.listStatus(fs, splitDir, filterType);
         if (logfiles != null) {
           for (FileStatus status : logfiles) {
             if (!status.isDir()) {
@@ -397,7 +431,7 @@ public class MasterWalManager {
               if (!CommonFSUtils.renameAndSetModifyTime(fs, status.getPath(), newPath)) {
                 LOG.warn("Unable to move  " + status.getPath() + " to " + newPath);
               } else {
-                LOG.debug("Archived meta log " + status.getPath() + " to " + newPath);
+                LOG.debug("Archived "+regionType+" log " + status.getPath() + " to " + newPath);
               }
             }
           }
@@ -407,7 +441,7 @@ public class MasterWalManager {
         }
       }
     } catch (IOException ie) {
-      LOG.warn("Failed archiving meta log for server " + serverName, ie);
+      LOG.warn("Failed archiving "+regionType+" log for server " + serverName, ie);
     }
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
index f4e91b5..d900a59 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
@@ -102,7 +102,7 @@ public class MetaRegionLocationCache extends ZKListener {
     List<String> znodes = null;
     while (retryCounter.shouldRetry()) {
       try {
-        znodes = watcher.getMetaReplicaNodesAndWatchChildren();
+        znodes = watcher.getRootReplicaNodesAndWatchChildren();
         break;
       } catch (KeeperException ke) {
         LOG.debug("Error populating initial meta locations", ke);
@@ -149,7 +149,7 @@ public class MetaRegionLocationCache extends ZKListener {
     try {
       byte[] data = ZKUtil.getDataAndWatch(watcher,
           watcher.getZNodePaths().getZNodeForReplica(replicaId));
-      metaRegionState = ProtobufUtil.parseMetaRegionStateFrom(data, replicaId);
+      metaRegionState = ProtobufUtil.parseRootRegionStateFrom(data, replicaId);
     } catch (DeserializationException e) {
       throw ZKUtil.convert(e);
     }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitWALManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitWALManager.java
index 9ff84dc..b7c8c33 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitWALManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitWALManager.java
@@ -19,7 +19,10 @@ package org.apache.hadoop.hbase.master;
 import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_SPLIT_WAL_MAX_SPLITTER;
 import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_MAX_SPLITTER;
 import static org.apache.hadoop.hbase.master.MasterWalManager.META_FILTER;
-import static org.apache.hadoop.hbase.master.MasterWalManager.NON_META_FILTER;
+import static org.apache.hadoop.hbase.master.MasterWalManager.NON_CATALOG_FILTER;
+import static org.apache.hadoop.hbase.master.MasterWalManager.ROOT_FILTER;
+
+
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collections;
@@ -32,6 +35,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
@@ -74,6 +78,22 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 public class SplitWALManager {
   private static final Logger LOG = LoggerFactory.getLogger(SplitWALManager.class);
 
+  public enum SplitType {
+    ROOT(ROOT_FILTER),
+    META(META_FILTER),
+    USER(NON_CATALOG_FILTER);
+
+    private PathFilter filter;
+
+    SplitType(PathFilter filter) {
+      this.filter = filter;
+    }
+
+    public PathFilter getFilter() {
+      return filter;
+    }
+  }
+
   private final MasterServices master;
   private final SplitWorkerAssigner splitWorkerAssigner;
   private final Path rootDir;
@@ -92,11 +112,19 @@ public class SplitWALManager {
     this.walArchiveDir = new Path(this.rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
   }
 
-  public List<Procedure> splitWALs(ServerName crashedServer, boolean splitMeta)
+  //TODO francis temp added this to avoid updating all tests
+  public List<Procedure> splitWALs(ServerName crashedServer, boolean splitMeta) throws IOException {
+    if (splitMeta) {
+      return splitWALs(crashedServer, SplitType.META);
+    }
+    return splitWALs(crashedServer, SplitType.USER);
+  }
+
+  public List<Procedure> splitWALs(ServerName crashedServer, SplitType splitType)
       throws IOException {
     try {
       // 1. list all splitting files
-      List<FileStatus> splittingFiles = getWALsToSplit(crashedServer, splitMeta);
+      List<FileStatus> splittingFiles = getWALsToSplit(crashedServer, splitType);
       // 2. create corresponding procedures
       return createSplitWALProcedures(splittingFiles, crashedServer);
     } catch (IOException e) {
@@ -105,12 +133,22 @@ public class SplitWALManager {
     }
   }
 
+
+  //TODO francis temp added this to avoid updating tests
   public List<FileStatus> getWALsToSplit(ServerName serverName, boolean splitMeta)
+    throws IOException {
+    if (splitMeta) {
+      return getWALsToSplit(serverName, SplitType.META);
+    }
+    return getWALsToSplit(serverName, SplitType.USER);
+  }
+
+  public List<FileStatus> getWALsToSplit(ServerName serverName, SplitType splitType)
       throws IOException {
     List<Path> logDirs = master.getMasterWalManager().getLogDirs(Collections.singleton(serverName));
     FileStatus[] fileStatuses =
-        SplitLogManager.getFileList(this.conf, logDirs, splitMeta ? META_FILTER : NON_META_FILTER);
-    LOG.info("{} WAL count={}, meta={}", serverName, fileStatuses.length, splitMeta);
+        SplitLogManager.getFileList(this.conf, logDirs, splitType.getFilter());
+    LOG.info("{} WAL count={}, SplitType={}", serverName, fileStatuses.length, splitType);
     return Lists.newArrayList(fileStatuses);
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index 5638af5..e854d57 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -25,6 +25,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
@@ -76,9 +77,11 @@ import org.apache.hadoop.hbase.rsgroup.RSGroupBasedLoadBalancer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.hadoop.hbase.util.RetryCounterFactory;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.util.VersionInfo;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
@@ -152,6 +155,9 @@ public class AssignmentManager {
   private static final int DEFAULT_RIT_STUCK_WARNING_THRESHOLD = 60 * 1000;
   public static final String UNEXPECTED_STATE_REGION = "Unexpected state for ";
 
+  private final ProcedureEvent<?> rootAssignEvent = new ProcedureEvent<>("root assign");
+  private final ProcedureEvent<?> rootLoadEvent = new ProcedureEvent<>("root load");
+
   private final ProcedureEvent<?> metaAssignEvent = new ProcedureEvent<>("meta assign");
   private final ProcedureEvent<?> metaLoadEvent = new ProcedureEvent<>("meta load");
 
@@ -232,18 +238,18 @@ public class AssignmentManager {
       // here we are still in the early steps of active master startup. There is only one thread(us)
       // can access AssignmentManager and create region node, so here we do not need to lock the
       // region node.
-      RegionState regionState = MetaTableLocator.getMetaRegionState(zkw);
-      RegionStateNode regionNode =
-        regionStates.getOrCreateRegionStateNode(RegionInfoBuilder.FIRST_META_REGIONINFO);
-      regionNode.setRegionLocation(regionState.getServerName());
-      regionNode.setState(regionState.getState());
-      if (regionNode.getProcedure() != null) {
-        regionNode.getProcedure().stateLoaded(this, regionNode);
-      }
-      if (regionState.getServerName() != null) {
-        regionStates.addRegionToServer(regionNode);
-      }
-      setMetaAssigned(regionState.getRegion(), regionState.getState() == State.OPEN);
+      RegionState rootRegionState = RootTableLocator.getRootRegionState(zkw);
+      RegionStateNode rootRegionNode =
+        regionStates.getOrCreateRegionStateNode(RegionInfoBuilder.ROOT_REGIONINFO);
+        rootRegionNode.setRegionLocation(rootRegionState.getServerName());
+        rootRegionNode.setState(rootRegionState.getState());
+        if (rootRegionNode.getProcedure() != null) {
+          rootRegionNode.getProcedure().stateLoaded(this, rootRegionNode);
+        }
+        if (rootRegionState.getServerName() != null) {
+          regionStates.addRegionToServer(rootRegionNode);
+        }
+        setRootAssigned(rootRegionState.getRegion(), rootRegionState.getState() == State.OPEN);
     }
   }
 
@@ -312,6 +318,10 @@ public class AssignmentManager {
       for (RegionInfo hri: getMetaRegionSet()) {
         setMetaAssigned(hri, false);
       }
+      rootLoadEvent.suspend();
+      for(RegionInfo hri: getRootRegionSet()) {
+        setRootAssigned(hri, false);
+      }
     }
   }
 
@@ -400,6 +410,130 @@ public class AssignmentManager {
   }
 
   // ============================================================================================
+  //  ROOT Helpers
+  // ============================================================================================
+  private boolean isRootRegion(final RegionInfo regionInfo) {
+    return regionInfo.isRootRegion();
+  }
+
+  public boolean isRootRegion(final byte[] regionName) {
+    return getRootRegionFromName(regionName) != null;
+  }
+
+  public RegionInfo getRootRegionFromName(final byte[] regionName) {
+    for (RegionInfo hri: getRootRegionSet()) {
+      if (Bytes.equals(hri.getRegionName(), regionName)) {
+        return hri;
+      }
+    }
+    return null;
+  }
+
+  public boolean isCarryingRoot(final ServerName serverName) {
+    // TODO: handle multiple root
+    return isCarryingRegion(serverName, RegionInfoBuilder.ROOT_REGIONINFO);
+  }
+
+  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);
+  }
+
+  // TODO: handle multiple root.
+  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;
+    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.
+    return rootAssignEvent;
+  }
+
+  /**
+   * Wait until AM finishes the root loading, i.e, the region states rebuilding.
+   * @see #isRootLoaded()
+   * @see #waitRootAssigned(Procedure, RegionInfo)
+   */
+  public boolean waitRootLoaded(Procedure<?> proc) {
+    if (rootLoadEvent.suspendIfNotReady(proc)) {
+      LOG.debug("Waiting for root to be loaded: "+proc);
+      return true;
+    }
+    return false;
+  }
+
+  @VisibleForTesting
+  void wakeRootLoadedEvent() {
+    synchronized (rootLoadEvent) {
+      rootLoadEvent.wake(getProcedureScheduler());
+      assert isRootLoaded() : "expected root to be loaded";
+    }
+  }
+
+  /**
+   * Return whether AM finishes the root loading, i.e, the region states rebuilding.
+   * @see #isRootAssigned()
+   * @see #waitRootLoaded(Procedure)
+   */
+  public boolean isRootLoaded() {
+    return rootLoadEvent.isReady();
+  }
+
+
+  // ============================================================================================
   //  META Helpers
   // ============================================================================================
   private boolean isMetaRegion(final RegionInfo regionInfo) {
@@ -432,11 +566,20 @@ public class AssignmentManager {
 
   private RegionInfo getMetaForRegion(final RegionInfo regionInfo) {
     //if (regionInfo.isMetaRegion()) return regionInfo;
-    // TODO: handle multiple meta. if the region provided is not meta lookup
+    // TODO francis 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.
+   */
+  public boolean waitForMetaOnline() {
+    return isRegionOnline(RegionInfoBuilder.FIRST_META_REGIONINFO);
+  }
+
   // TODO: handle multiple meta.
   private static final Set<RegionInfo> META_REGION_SET =
       Collections.singleton(RegionInfoBuilder.FIRST_META_REGIONINFO);
@@ -475,8 +618,10 @@ public class AssignmentManager {
     assert isMetaRegion(metaRegionInfo) : "unexpected non-meta region " + metaRegionInfo;
     ProcedureEvent<?> metaAssignEvent = getMetaAssignEvent(metaRegionInfo);
     if (assigned) {
+      LOG.debug("Setting hbase:meta region assigned: "+metaRegionInfo);
       metaAssignEvent.wake(getProcedureScheduler());
     } else {
+      LOG.debug("Setting hbase:meta region unassigned: "+metaRegionInfo);
       metaAssignEvent.suspend();
     }
   }
@@ -493,7 +638,11 @@ public class AssignmentManager {
    * @see #waitMetaAssigned(Procedure, RegionInfo)
    */
   public boolean waitMetaLoaded(Procedure<?> proc) {
-    return metaLoadEvent.suspendIfNotReady(proc);
+    if (metaLoadEvent.suspendIfNotReady(proc)) {
+      LOG.debug("Waiting for meta to be loaded: "+proc);
+      return true;
+    }
+    return false;
   }
 
   @VisibleForTesting
@@ -511,6 +660,39 @@ public class AssignmentManager {
     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).
+   */
+  public boolean isRegionOnline(RegionInfo ri) {
+    RetryCounter rc = null;
+    while (!master.isStopped()) {
+      RegionState rs = regionStates.getRegionState(ri);
+      if (rs.isOpened()) {
+        if (master.getServerManager().isServerOnline(rs.getServerName())) {
+          return true;
+        }
+      }
+      // Region is not OPEN.
+      Optional<Procedure<MasterProcedureEnv>> optProc =
+        master.getMasterProcedureExecutor().getProcedures().
+        stream().filter(p -> p instanceof ServerCrashProcedure).findAny();
+      // TODO: Add a page to refguide on how to do repair. Have this log message point to it.
+      // Page will talk about loss of edits, how to schedule at least the meta WAL recovery, and
+      // then how to assign including how to break region lock if one held.
+      LOG.warn("{} is NOT online; state={}; ServerCrashProcedures={}. Master startup cannot " +
+          "progress, in holding-pattern until region onlined.",
+        ri.getRegionNameAsString(), rs, optProc.isPresent());
+      // Check once-a-minute.
+      if (rc == null) {
+        rc = new RetryCounterFactory(1000).create();
+      }
+      Threads.sleep(rc.getBackoffTimeAndIncrementAttempts());
+    }
+    return false;
+  }
+
   /**
    * Start a new thread to check if there are region servers whose versions are higher than others.
    * If so, move all system table regions to RS with the highest version to keep compatibility.
@@ -1004,7 +1186,7 @@ public class AssignmentManager {
 
   private void updateRegionTransition(ServerName serverName, TransitionCode state,
       RegionInfo regionInfo, long seqId, long procId) throws IOException {
-    checkMetaLoaded(regionInfo);
+    checkParentCatalogLoaded(regionInfo);
 
     RegionStateNode regionNode = regionStates.getRegionStateNode(regionInfo);
     if (regionNode == null) {
@@ -1056,7 +1238,7 @@ public class AssignmentManager {
   private void updateRegionSplitTransition(final ServerName serverName, final TransitionCode state,
       final RegionInfo parent, final RegionInfo hriA, final RegionInfo hriB)
       throws IOException {
-    checkMetaLoaded(parent);
+    checkParentCatalogLoaded(parent);
 
     if (state != TransitionCode.READY_TO_SPLIT) {
       throw new UnexpectedStateException("unsupported split regionState=" + state +
@@ -1094,7 +1276,7 @@ public class AssignmentManager {
 
   private void updateRegionMergeTransition(final ServerName serverName, final TransitionCode state,
       final RegionInfo merged, final RegionInfo hriA, final RegionInfo hriB) throws IOException {
-    checkMetaLoaded(merged);
+    checkParentCatalogLoaded(merged);
 
     if (state != TransitionCode.READY_TO_MERGE) {
       throw new UnexpectedStateException("Unsupported merge regionState=" + state +
@@ -1163,7 +1345,7 @@ public class AssignmentManager {
       LOG.trace("no online region found on {}", serverName);
       return;
     }
-    if (!isMetaLoaded()) {
+    if (!isRootLoaded() || !isMetaLoaded()) {
       // we are still on startup, skip checking
       return;
     }
@@ -1444,6 +1626,28 @@ public class AssignmentManager {
     long startTime = System.nanoTime();
     LOG.debug("Joining cluster...");
 
+    // FIRST Catalog tables READ!!!!
+    // The below cannot make progress w/o hbase:meta being online.
+    // This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background
+    // as procedures run -- in particular SCPs for crashed servers... One should put up hbase:meta
+    // if it is down. It may take a while to come online. So, wait here until meta if for sure
+    // available. That's what waitForXXXXOnline does.
+
+
+
+    LOG.debug("Waiting for hbase:root to be online.");
+    if (!waitForRootOnline()) {
+      throw new IOException("Waited too long for hbase:root to be online");
+    }
+
+    //load hbase:root to build regionstate for hbase:meta regions
+    loadRoot();
+
+    LOG.debug("Waiting for hbase:meta to be online.");
+    if (!waitForMetaOnline()) {
+      throw new IOException("Waited too long for hbase:meta to be online");
+    }
+
     // Scan hbase:meta to build list of existing regions, servers, and assignment.
     // hbase:meta is online now or will be. Inside loadMeta, we keep trying. Can't make progress
     // w/o  meta.
@@ -1486,13 +1690,13 @@ public class AssignmentManager {
   }
 
   /* AM internal RegionStateStore.RegionStateVisitor implementation. To be used when
-   * scanning META table for region rows, using RegionStateStore utility methods. RegionStateStore
+   * scanning Catalog table for region rows, using RegionStateStore utility methods. RegionStateStore
    * methods will convert Result into proper RegionInfo instances, but those would still need to be
    * added into AssignmentManager.regionStates in-memory cache.
    * RegionMetaLoadingVisitor.visitRegionState method provides the logic for adding RegionInfo
    * instances as loaded from latest META scan into AssignmentManager.regionStates.
    */
-  private class RegionMetaLoadingVisitor implements RegionStateStore.RegionStateVisitor  {
+  private class RegionCatalogLoadingVisitor implements RegionStateStore.RegionStateVisitor  {
 
     @Override
     public void visitRegionState(Result result, final RegionInfo regionInfo, final State state,
@@ -1533,6 +1737,13 @@ public class AssignmentManager {
       if (regionNode.getProcedure() != null) {
         regionNode.getProcedure().stateLoaded(AssignmentManager.this, regionNode);
       }
+      if (isMetaRegion(regionInfo)) {
+        if (localState.matches(State.OPEN)) {
+          setMetaAssigned(regionInfo, true);
+        } else if (localState.matches(State.CLOSING, State.CLOSED)){
+          setMetaAssigned(regionInfo, false);
+        }
+      }
     }
   };
 
@@ -1546,10 +1757,10 @@ public class AssignmentManager {
    *          cache, <b>null</b> otherwise.
    * @throws UnknownRegionException if any errors occur while querying meta.
    */
-  public RegionInfo loadRegionFromMeta(String regionEncodedName) throws UnknownRegionException {
+  public RegionInfo loadRegionFromCatalog(String regionEncodedName) throws UnknownRegionException {
     try {
-      RegionMetaLoadingVisitor visitor = new RegionMetaLoadingVisitor();
-      regionStateStore.visitMetaForRegion(regionEncodedName, visitor);
+      RegionCatalogLoadingVisitor visitor = new RegionCatalogLoadingVisitor();
+      regionStateStore.visitCatalogForRegion(regionEncodedName, visitor);
       return regionStates.getRegionState(regionEncodedName) == null ? null :
         regionStates.getRegionState(regionEncodedName).getRegion();
     } catch(IOException e) {
@@ -1558,9 +1769,25 @@ public class AssignmentManager {
     }
   }
 
+  public void loadRoot() throws IOException {
+    synchronized (rootLoadEvent) {
+      if (!isRootLoaded()) {
+        // TODO: use a thread pool
+        LOG.debug("Loaded hbase:root");
+        regionStateStore.visitCatalogTable(TableName.ROOT_TABLE_NAME,
+            new RegionCatalogLoadingVisitor());
+        wakeRootLoadedEvent();
+      } else {
+        LOG.debug("Not loading hbase:root, already loaded");
+      }
+    }
+  }
+
   private void loadMeta() throws IOException {
     // TODO: use a thread pool
-    regionStateStore.visitMeta(new RegionMetaLoadingVisitor());
+    LOG.debug("Loaded hbase:meta");
+    regionStateStore.visitCatalogTable(TableName.META_TABLE_NAME,
+        new RegionCatalogLoadingVisitor());
     // every assignment is blocked until meta is loaded.
     wakeMetaLoadedEvent();
   }
@@ -1572,6 +1799,25 @@ public class AssignmentManager {
    * @param hri region to check if it is already rebuild
    * @throws PleaseHoldException if meta has not been loaded yet
    */
+  private void checkRootLoaded(RegionInfo hri) throws PleaseHoldException {
+    if (!isRunning()) {
+      throw new PleaseHoldException("AssignmentManager not running");
+    }
+    boolean root = isRootRegion(hri);
+    boolean rootLoaded = isRootLoaded();
+    if (!root && !rootLoaded) {
+      throw new PleaseHoldException(
+        "Master not fully online; hbase:root=" + root + ", rootLoaded=" + rootLoaded);
+    }
+  }
+
+  /**
+   * Used to check if the meta loading is done.
+   * <p/>
+   * if not we throw PleaseHoldException since we are rebuilding the RegionStates
+   * @param hri region to check if it is already rebuild
+   * @throws PleaseHoldException if meta has not been loaded yet
+   */
   private void checkMetaLoaded(RegionInfo hri) throws PleaseHoldException {
     if (!isRunning()) {
       throw new PleaseHoldException("AssignmentManager not running");
@@ -1584,6 +1830,15 @@ public class AssignmentManager {
     }
   }
 
+
+  private void checkParentCatalogLoaded(RegionInfo regionInfo) throws PleaseHoldException {
+    if (regionInfo.isMetaRegion()) {
+      checkRootLoaded(regionInfo);
+    } else if (!regionInfo.isRootRegion()) {
+      checkMetaLoaded(regionInfo);
+    }
+  }
+
   // ============================================================================================
   //  TODO: Metrics
   // ============================================================================================
@@ -1619,14 +1874,16 @@ public class AssignmentManager {
     if (serverNode != null) {
       serverNode.writeLock().lock();
     }
+    boolean carryingRoot;
     boolean carryingMeta;
     long pid;
     try {
       ProcedureExecutor<MasterProcedureEnv> procExec = this.master.getMasterProcedureExecutor();
+      carryingRoot = isCarryingRoot(serverName);
       carryingMeta = isCarryingMeta(serverName);
       if (!force && serverNode != null && !serverNode.isInState(ServerState.ONLINE)) {
-        LOG.info("Skip adding ServerCrashProcedure for {} (meta={}) -- running?",
-          serverNode, carryingMeta);
+        LOG.info("Skip adding ServerCrashProcedure for {} (root={}, meta={}) -- running?",
+          serverNode, carryingRoot, carryingMeta);
         return Procedure.NO_PROC_ID;
       } else {
         MasterProcedureEnv mpe = procExec.getEnvironment();
@@ -1642,14 +1899,17 @@ public class AssignmentManager {
 
         if (force) {
           pid = procExec.submitProcedure(
-              new HBCKServerCrashProcedure(mpe, serverName, shouldSplitWal, carryingMeta));
+              new HBCKServerCrashProcedure(
+                mpe, serverName, shouldSplitWal, carryingRoot, carryingMeta));
         } else {
           pid = procExec.submitProcedure(
-              new ServerCrashProcedure(mpe, serverName, shouldSplitWal, carryingMeta));
+              new ServerCrashProcedure(
+                mpe, serverName, shouldSplitWal, carryingRoot, carryingMeta));
         }
-        LOG.info("Scheduled ServerCrashProcedure pid={} for {} (carryingMeta={}){}.",
-          pid, serverName, carryingMeta,
-          serverNode == null? "": " " + serverNode.toString() + ", oldState=" + oldState);
+        LOG.info(
+          "Scheduled ServerCrashProcedure pid={} for {} (carryingRoot={}, carryingMeta={}){}.",
+          pid, serverName, carryingRoot, carryingMeta,
+            serverNode == null? "": " " + serverNode.toString() + ", oldState=" + oldState);
       }
     } finally {
       if (serverNode != null) {
@@ -1810,8 +2070,11 @@ public class AssignmentManager {
     transitStateAndUpdate(regionNode, State.CLOSING, STATES_EXPECTED_ON_CLOSING);
 
     RegionInfo hri = regionNode.getRegionInfo();
-    // Set meta has not initialized early. so people trying to create/edit tables will wait
-    if (isMetaRegion(hri)) {
+    // Set root has not initialized early. so people trying to create/edit tables will wait
+    if (isRootRegion(hri)) {
+      setRootAssigned(hri, false);
+    } else if (isMetaRegion(hri)) {
+      // Set meta has not initialized early. so people trying to create/edit tables will wait
       setMetaAssigned(hri, false);
     }
     regionStates.addRegionToServer(regionNode);
@@ -1877,6 +2140,13 @@ public class AssignmentManager {
       // on table that contains state.
       setMetaAssigned(regionInfo, true);
     }
+    if (isRootRegion(regionInfo) && regionNode.getState() == State.OPEN) {
+      // Usually we'd set a table ENABLED at this stage but hbase:meta is ALWAYs enabled, it
+      // can't be disabled -- so skip the RPC (besides... enabled is managed by TableStateManager
+      // which is backed by hbase:meta... Avoid setting ENABLED to avoid having to update state
+      // on table that contains state.
+      setRootAssigned(regionInfo, true);
+    }
   }
 
   // ============================================================================================
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
index f1b3329..f36b424 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java
@@ -26,6 +26,7 @@ import java.util.List;
 import java.util.stream.Stream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MetaMutationAnnotation;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -159,6 +160,8 @@ public class MergeTableRegionsProcedure
    * to have an extent sufficient to cover all regions-to-merge.
    */
   private static RegionInfo createMergedRegionInfo(final RegionInfo[] regionsToMerge) {
+    KeyValue.KVComparator comparator =
+      RegionInfo.getComparator(regionsToMerge[0].getTable());
     byte [] lowestStartKey = null;
     byte [] highestEndKey = null;
     // Region Id is a timestamp. Merged region's id can't be less than that of
@@ -167,12 +170,16 @@ public class MergeTableRegionsProcedure
     for (RegionInfo ri: regionsToMerge) {
       if (lowestStartKey == null) {
         lowestStartKey = ri.getStartKey();
-      } else if (Bytes.compareTo(ri.getStartKey(), lowestStartKey) < 0) {
+      } else if (comparator.compareRows(
+        ri.getStartKey(), 0, ri.getStartKey().length,
+        lowestStartKey, 0, lowestStartKey.length) < 0) {
         lowestStartKey = ri.getStartKey();
       }
       if (highestEndKey == null) {
         highestEndKey = ri.getEndKey();
-      } else if (ri.isLast() || Bytes.compareTo(ri.getEndKey(), highestEndKey) > 0) {
+      } else if (ri.isLast() || comparator.compareRows(
+          ri.getEndKey(), 0, ri.getEndKey().length,
+          highestEndKey, 0, highestEndKey.length) > 0) {
         highestEndKey = ri.getEndKey();
       }
       highestRegionId = ri.getRegionId() > highestRegionId? ri.getRegionId(): highestRegionId;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java
index 1c90d81..caafb64 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java
@@ -148,13 +148,20 @@ public abstract class RegionRemoteProcedureBase extends Procedure<MasterProcedur
 
   @Override
   protected boolean waitInitialized(MasterProcedureEnv env) {
-    if (TableName.isMetaTableName(getTableName())) {
+    if (TableName.isRootTableName(getTableName())) {
       return false;
     }
-    // First we need meta to be loaded, and second, if meta is not online then we will likely to
-    // fail when updating meta so we wait until it is assigned.
-    AssignmentManager am = env.getAssignmentManager();
-    return am.waitMetaLoaded(this) || am.waitMetaAssigned(this, region);
+    if (TableName.isMetaTableName(getTableName())) {
+      // First we need rot to be loaded, and second, if root is not online then we will likely to
+      // fail when updating root so we wait until it is assigned.
+      AssignmentManager am = env.getAssignmentManager();
+      return am.waitRootLoaded(this) || am.waitRootAssigned(this, region);
+    } else {
+      // First we need meta to be loaded, and second, if meta is not online then we will likely to
+      // fail when updating meta so we wait until it is assigned.
+      AssignmentManager am = env.getAssignmentManager();
+      return am.waitMetaLoaded(this) || am.waitMetaAssigned(this, region);
+    }
   }
 
   @Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
index 7271ff7..83d71ca 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.ClientMetaTableAccessor;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.CatalogAccessor;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -61,7 +61,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hadoop.hbase.wal.WALSplitUtil;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -98,8 +98,9 @@ public class RegionStateStore {
       ServerName regionLocation, ServerName lastHost, long openSeqNum);
   }
 
-  public void visitMeta(final RegionStateVisitor visitor) throws IOException {
-    MetaTableAccessor.fullScanRegions(master.getConnection(),
+  public void visitCatalogTable(TableName catalogTableName, final RegionStateVisitor visitor)
+    throws IOException {
+    CatalogAccessor.fullScanRegions(master.getConnection(), catalogTableName,
       new ClientMetaTableAccessor.Visitor() {
         final boolean isDebugEnabled = LOG.isDebugEnabled();
 
@@ -110,7 +111,7 @@ public class RegionStateStore {
             if (LOG.isTraceEnabled()) {
               st = System.currentTimeMillis();
             }
-            visitMetaEntry(visitor, r);
+            visitCatalogEntry(visitor, r);
             if (LOG.isTraceEnabled()) {
               long et = System.currentTimeMillis();
               LOG.trace("[T] LOAD META PERF " + StringUtils.humanTimeDiff(et - st));
@@ -131,16 +132,16 @@ public class RegionStateStore {
    * @param visitor The <code>RegionStateVisitor</code> instance to react over the query results.
    * @throws IOException If some error occurs while querying META or parsing results.
    */
-  public void visitMetaForRegion(final String regionEncodedName, final RegionStateVisitor visitor)
+  public void visitCatalogForRegion(final String regionEncodedName, final RegionStateVisitor visitor)
       throws IOException {
-    Result result = MetaTableAccessor.
+    Result result = CatalogAccessor.
       scanByRegionEncodedName(master.getConnection(), regionEncodedName);
     if (result != null) {
-      visitMetaEntry(visitor, result);
+      visitCatalogEntry(visitor, result);
     }
   }
 
-  private void visitMetaEntry(final RegionStateVisitor visitor, final Result result)
+  private void visitCatalogEntry(final RegionStateVisitor visitor, final Result result)
       throws IOException {
     final RegionLocations rl = CatalogFamilyFormat.getRegionLocations(result);
     if (rl == null) return;
@@ -159,7 +160,7 @@ public class RegionStateStore {
       final State state = getRegionState(result, regionInfo);
 
       final ServerName lastHost = hrl.getServerName();
-      ServerName regionLocation = MetaTableAccessor.getTargetServerName(result, replicaId);
+      ServerName regionLocation = CatalogAccessor.getTargetServerName(result, replicaId);
       final long openSeqNum = hrl.getSeqNum();
 
       // TODO: move under trace, now is visible for debugging
@@ -172,13 +173,13 @@ public class RegionStateStore {
   }
 
   void updateRegionLocation(RegionStateNode regionStateNode) throws IOException {
-    if (regionStateNode.getRegionInfo().isMetaRegion()) {
-      updateMetaLocation(regionStateNode.getRegionInfo(), regionStateNode.getRegionLocation(),
+    if (regionStateNode.getRegionInfo().isRootRegion()) {
+      updateRootLocation(regionStateNode.getRegionInfo(), regionStateNode.getRegionLocation(),
         regionStateNode.getState());
     } else {
       long openSeqNum = regionStateNode.getState() == State.OPEN ? regionStateNode.getOpenSeqNum()
         : HConstants.NO_SEQNUM;
-      updateUserRegionLocation(regionStateNode.getRegionInfo(), regionStateNode.getState(),
+      updateRegionLocation(regionStateNode.getRegionInfo(), regionStateNode.getState(),
         regionStateNode.getRegionLocation(), openSeqNum,
         // The regionStateNode may have no procedure in a test scenario; allow for this.
         regionStateNode.getProcedure() != null ? regionStateNode.getProcedure().getProcId()
@@ -186,30 +187,32 @@ public class RegionStateStore {
     }
   }
 
-  private void updateMetaLocation(RegionInfo regionInfo, ServerName serverName, State state)
+  private void updateRootLocation(RegionInfo regionInfo, ServerName serverName, State state)
       throws IOException {
     try {
-      MetaTableLocator.setMetaLocation(master.getZooKeeper(), serverName, regionInfo.getReplicaId(),
+      RootTableLocator.setRootLocation(master.getZooKeeper(), serverName, regionInfo.getReplicaId(),
         state);
     } catch (KeeperException e) {
       throw new IOException(e);
     }
   }
 
-  private void updateUserRegionLocation(RegionInfo regionInfo, State state,
+  private void updateRegionLocation(RegionInfo regionInfo, State state,
       ServerName regionLocation, long openSeqNum,
        long pid) throws IOException {
+    TableName catalogTableName = regionInfo.isMetaRegion() ?
+        TableName.ROOT_TABLE_NAME : TableName.META_TABLE_NAME;
     long time = EnvironmentEdgeManager.currentTime();
     final int replicaId = regionInfo.getReplicaId();
     final Put put = new Put(CatalogFamilyFormat.getMetaKeyForRegion(regionInfo), time);
-    MetaTableAccessor.addRegionInfo(put, regionInfo);
+    CatalogAccessor.addRegionInfo(put, regionInfo);
     final StringBuilder info =
-      new StringBuilder("pid=").append(pid).append(" updating hbase:meta row=")
+      new StringBuilder("pid=").append(pid).append(" updating "+catalogTableName+" row=")
         .append(regionInfo.getEncodedName()).append(", regionState=").append(state);
     if (openSeqNum >= 0) {
       Preconditions.checkArgument(state == State.OPEN && regionLocation != null,
           "Open region should be on a server");
-      MetaTableAccessor.addLocation(put, regionLocation, openSeqNum, replicaId);
+      CatalogAccessor.addLocation(put, regionLocation, openSeqNum, replicaId);
       // only update replication barrier for default replica
       if (regionInfo.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID &&
         hasGlobalReplicationScope(regionInfo.getTable())) {
@@ -240,12 +243,12 @@ public class RegionStateStore {
         .setValue(Bytes.toBytes(state.name()))
         .build());
     LOG.info(info.toString());
-    updateRegionLocation(regionInfo, state, put);
+    updateRegionLocation(catalogTableName, regionInfo, state, put);
   }
 
-  private void updateRegionLocation(RegionInfo regionInfo, State state, Put put)
-      throws IOException {
-    try (Table table = master.getConnection().getTable(TableName.META_TABLE_NAME)) {
+  private void updateRegionLocation(TableName catalogTableName, RegionInfo regionInfo,
+      State state, Put put) throws IOException {
+    try (Table table = master.getConnection().getTable(catalogTableName)) {
       table.put(put);
     } catch (IOException e) {
       // TODO: Revist!!!! Means that if a server is loaded, then we will abort our host!
@@ -329,13 +332,13 @@ public class RegionStateStore {
     }
     long time = EnvironmentEdgeManager.currentTime();
     // Put for parent
-    Put putParent = MetaTableAccessor.makePutFromRegionInfo(
+    Put putParent = CatalogAccessor.makePutFromRegionInfo(
       RegionInfoBuilder.newBuilder(parent).setOffline(true).setSplit(true).build(), time);
-    MetaTableAccessor.addDaughtersToPut(putParent, splitA, splitB);
+    CatalogAccessor.addDaughtersToPut(putParent, splitA, splitB);
 
     // Puts for daughters
-    Put putA = MetaTableAccessor.makePutFromRegionInfo(splitA, time);
-    Put putB = MetaTableAccessor.makePutFromRegionInfo(splitB, time);
+    Put putA = CatalogAccessor.makePutFromRegionInfo(splitA, time);
+    Put putB = CatalogAccessor.makePutFromRegionInfo(splitB, time);
     if (parentOpenSeqNum > 0) {
       ReplicationBarrierFamilyFormat.addReplicationBarrier(putParent, parentOpenSeqNum);
       ReplicationBarrierFamilyFormat.addReplicationParent(putA, Collections.singletonList(parent));
@@ -346,8 +349,8 @@ public class RegionStateStore {
     // default OFFLINE state. If Master gets restarted after this step, start up sequence of
     // master tries to assign these offline regions. This is followed by re-assignments of the
     // daughter regions from resumed {@link SplitTableRegionProcedure}
-    MetaTableAccessor.addRegionStateToPut(putA, RegionState.State.CLOSED);
-    MetaTableAccessor.addRegionStateToPut(putB, RegionState.State.CLOSED);
+    CatalogAccessor.addRegionStateToPut(putA, RegionState.State.CLOSED);
+    CatalogAccessor.addRegionStateToPut(putB, RegionState.State.CLOSED);
 
     // new regions, openSeqNum = 1 is fine.
     addSequenceNum(putA, 1, splitA.getReplicaId());
@@ -357,8 +360,8 @@ public class RegionStateStore {
     // cached whenever the primary region is looked up from meta
     int regionReplication = getRegionReplication(htd);
     for (int i = 1; i < regionReplication; i++) {
-      MetaTableAccessor.addEmptyLocation(putA, i);
-      MetaTableAccessor.addEmptyLocation(putB, i);
+      CatalogAccessor.addEmptyLocation(putA, i);
+      CatalogAccessor.addEmptyLocation(putB, i);
     }
 
     multiMutate(parent, Arrays.asList(putParent, putA, putB));
@@ -376,7 +379,7 @@ public class RegionStateStore {
     for (RegionInfo ri: parents) {
       long seqNum = globalScope ? getOpenSeqNumForParentRegion(ri) : -1;
       // Deletes for merging regions
-      mutations.add(MetaTableAccessor.makeDeleteFromRegionInfo(ri, time));
+      mutations.add(CatalogAccessor.makeDeleteFromRegionInfo(ri, time));
       if (seqNum > 0) {
         mutations
           .add(ReplicationBarrierFamilyFormat.makePutForReplicationBarrier(ri, seqNum, time));
@@ -384,28 +387,28 @@ public class RegionStateStore {
       }
     }
     // Put for parent
-    Put putOfMerged = MetaTableAccessor.makePutFromRegionInfo(child, time);
+    Put putOfMerged = CatalogAccessor.makePutFromRegionInfo(child, time);
     putOfMerged = addMergeRegions(putOfMerged, Arrays.asList(parents));
     // Set initial state to CLOSED.
     // NOTE: If initial state is not set to CLOSED then merged region gets added with the
     // default OFFLINE state. If Master gets restarted after this step, start up sequence of
     // master tries to assign this offline region. This is followed by re-assignments of the
     // merged region from resumed {@link MergeTableRegionsProcedure}
-    MetaTableAccessor.addRegionStateToPut(putOfMerged, RegionState.State.CLOSED);
+    CatalogAccessor.addRegionStateToPut(putOfMerged, RegionState.State.CLOSED);
     mutations.add(putOfMerged);
     // The merged is a new region, openSeqNum = 1 is fine. ServerName may be null
     // if crash after merge happened but before we got to here.. means in-memory
     // locations of offlined merged, now-closed, regions is lost. Should be ok. We
     // assign the merged region later.
     if (serverName != null) {
-      MetaTableAccessor.addLocation(putOfMerged, serverName, 1, child.getReplicaId());
+      CatalogAccessor.addLocation(putOfMerged, serverName, 1, child.getReplicaId());
     }
 
     // Add empty locations for region replicas of the merged region so that number of replicas
     // can be cached whenever the primary region is looked up from meta
     int regionReplication = getRegionReplication(htd);
     for (int i = 1; i < regionReplication; i++) {
-      MetaTableAccessor.addEmptyLocation(putOfMerged, i);
+      CatalogAccessor.addEmptyLocation(putOfMerged, i);
     }
     // add parent reference for serial replication
     if (!replicationParents.isEmpty()) {
@@ -495,7 +498,7 @@ public class RegionStateStore {
   }
 
   public void deleteRegions(final List<RegionInfo> regions) throws IOException {
-    MetaTableAccessor.deleteRegionInfos(master.getConnection(), regions);
+    CatalogAccessor.deleteRegionInfos(master.getConnection(), regions);
   }
 
   // ==========================================================================
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
index 3bb3c4c..31f7364 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
@@ -397,6 +397,22 @@ public class RegionStates {
   }
 
   /**
+   * Call this when we start root log splitting a crashed Server.
+   * @see #metaLogSplit(ServerName)
+   */
+  public void rootLogSplitting(ServerName serverName) {
+    setServerState(serverName, ServerState.SPLITTING_ROOT);
+  }
+
+  /**
+   * Called after we've split the meta logs on a crashed Server.
+   * @see #metaLogSplitting(ServerName)
+   */
+  public void rootLogSplit(ServerName serverName) {
+    setServerState(serverName, ServerState.SPLITTING_ROOT_DONE);
+  }
+
+  /**
    * Call this when we start meta log splitting a crashed Server.
    * @see #metaLogSplit(ServerName)
    */
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java
index b0a697d..b9327db 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java
@@ -166,13 +166,20 @@ public class TransitRegionStateProcedure
 
   @Override
   protected boolean waitInitialized(MasterProcedureEnv env) {
-    if (TableName.isMetaTableName(getTableName())) {
+    if (TableName.isRootTableName(getTableName())) {
       return false;
     }
-    // First we need meta to be loaded, and second, if meta is not online then we will likely to
-    // fail when updating meta so we wait until it is assigned.
-    AssignmentManager am = env.getAssignmentManager();
-    return am.waitMetaLoaded(this) || am.waitMetaAssigned(this, getRegion());
+    if (TableName.isMetaTableName(getTableName())) {
+      // First we need root to be loaded, and second, if root is not online then we will likely to
+      // fail when updating root so we wait until it is assigned.
+      AssignmentManager am = env.getAssignmentManager();
+      return am.waitRootLoaded(this) || am.waitRootAssigned(this, getRegion());
+    } else {
+      // First we need meta to be loaded, and second, if meta is not online then we will likely to
+      // fail when updating meta so we wait until it is assigned.
+      AssignmentManager am = env.getAssignmentManager();
+      return am.waitMetaLoaded(this) || am.waitMetaAssigned(this, getRegion());
+    }
   }
 
   private void queueAssign(MasterProcedureEnv env, RegionStateNode regionNode)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java
index 725a138..faef6d0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java
@@ -21,10 +21,10 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.CatalogAccessor;
 import org.apache.hadoop.hbase.CatalogFamilyFormat;
 import org.apache.hadoop.hbase.ClientMetaTableAccessor;
 import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.Connection;
@@ -66,8 +66,8 @@ public class HBCKServerCrashProcedure extends ServerCrashProcedure {
    * @param carryingMeta True if carrying hbase:meta table region.
    */
   public HBCKServerCrashProcedure(final MasterProcedureEnv env, final ServerName serverName,
-                              final boolean shouldSplitWal, final boolean carryingMeta) {
-    super(env, serverName, shouldSplitWal, carryingMeta);
+            final boolean shouldSplitWal, final boolean carryingRoot, final boolean carryingMeta) {
+    super(env, serverName, shouldSplitWal, carryingRoot, carryingMeta);
   }
 
   /**
@@ -99,7 +99,7 @@ public class HBCKServerCrashProcedure extends ServerCrashProcedure {
     UnknownServerVisitor visitor =
         new UnknownServerVisitor(env.getMasterServices().getConnection(), getServerName());
     try {
-      MetaTableAccessor.scanMetaForTableRegions(env.getMasterServices().getConnection(),
+      CatalogAccessor.scanMetaForTableRegions(env.getMasterServices().getConnection(),
           visitor, null);
     } catch (IOException ioe) {
       LOG.warn("Failed scan of hbase:meta for 'Unknown Servers'", ioe);
@@ -152,7 +152,7 @@ public class HBCKServerCrashProcedure extends ServerCrashProcedure {
           LOG.info("Moving {} from CLOSING to CLOSED in hbase:meta",
               hrl.getRegion().getRegionNameAsString());
           try {
-            MetaTableAccessor.updateRegionState(this.connection, hrl.getRegion(),
+            CatalogAccessor.updateRegionState(this.connection, hrl.getRegion(),
                 RegionState.State.CLOSED);
           } catch (IOException ioe) {
             LOG.warn("Failed moving {} from CLOSING to CLOSED",
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
index f158452..56a7db2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
@@ -72,39 +72,12 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMe
     return TableOperationType.CREATE;
   }
 
-  private static TableDescriptor writeFsLayout(Path rootDir, Configuration conf) throws IOException {
-    LOG.info("BOOTSTRAP: creating hbase:meta region");
-    FileSystem fs = rootDir.getFileSystem(conf);
-    Path tableDir = CommonFSUtils.getTableDir(rootDir, TableName.META_TABLE_NAME);
-    if (fs.exists(tableDir) && !fs.delete(tableDir, true)) {
-      LOG.warn("Can not delete partial created meta table, continue...");
-    }
-    // Bootstrapping, make sure blockcache is off. Else, one will be
-    // created here in bootstrap and it'll need to be cleaned up. Better to
-    // not make it in first place. Turn off block caching for bootstrap.
-    // Enable after.
-    TableDescriptor metaDescriptor = FSTableDescriptors.tryUpdateAndGetMetaTableDescriptor(conf, fs,
-      rootDir, builder -> builder.setRegionReplication(
-        conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM)));
-    HRegion
-      .createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, rootDir, conf, metaDescriptor, null)
-      .close();
-    return metaDescriptor;
-  }
-
   @Override
   protected Flow executeFromState(MasterProcedureEnv env, InitMetaState state)
     throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
     LOG.debug("Execute {}", this);
     try {
       switch (state) {
-        case INIT_META_WRITE_FS_LAYOUT:
-          Configuration conf = env.getMasterConfiguration();
-          Path rootDir = CommonFSUtils.getRootDir(conf);
-          TableDescriptor td = writeFsLayout(rootDir, conf);
-          env.getMasterServices().getTableDescriptors().update(td, true);
-          setNextState(InitMetaState.INIT_META_ASSIGN_META);
-          return Flow.HAS_MORE_STATE;
         case INIT_META_ASSIGN_META:
           LOG.info("Going to assign meta");
           addChildProcedure(env.getAssignmentManager()
@@ -169,7 +142,7 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMe
 
   @Override
   protected InitMetaState getInitialState() {
-    return InitMetaState.INIT_META_WRITE_FS_LAYOUT;
+    return InitMetaState.INIT_META_ASSIGN_META;
   }
 
   @Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitRootProcedure.java
similarity index 50%
copy from hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
copy to hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitRootProcedure.java
index f158452..43e2f51 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitRootProcedure.java
@@ -17,36 +17,39 @@
  */
 package org.apache.hadoop.hbase.master.procedure;
 
-import static org.apache.hadoop.hbase.NamespaceDescriptor.DEFAULT_NAMESPACE;
-import static org.apache.hadoop.hbase.NamespaceDescriptor.SYSTEM_NAMESPACE;
-import static org.apache.hadoop.hbase.master.TableNamespaceManager.insertNamespaceToMeta;
-import static org.apache.hadoop.hbase.master.procedure.AbstractStateMachineNamespaceProcedure.createDirectory;
-
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
+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.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
 import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
 import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
 import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.RetryCounter;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.InitMetaState;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.InitMetaStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.InitRootState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.InitRootStateData;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
 
 /**
@@ -54,9 +57,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
  * {@link TransitRegionStateProcedure} to assign meta.
  */
 @InterfaceAudience.Private
-public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMetaState> {
+public class InitRootProcedure extends AbstractStateMachineTableProcedure<InitRootState> {
 
-  private static final Logger LOG = LoggerFactory.getLogger(InitMetaProcedure.class);
+  private static final Logger LOG = LoggerFactory.getLogger(InitRootProcedure.class);
 
   private CountDownLatch latch = new CountDownLatch(1);
 
@@ -64,7 +67,7 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMe
 
   @Override
   public TableName getTableName() {
-    return TableName.META_TABLE_NAME;
+    return TableName.ROOT_TABLE_NAME;
   }
 
   @Override
@@ -72,55 +75,74 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMe
     return TableOperationType.CREATE;
   }
 
-  private static TableDescriptor writeFsLayout(Path rootDir, Configuration conf) throws IOException {
-    LOG.info("BOOTSTRAP: creating hbase:meta region");
+  private static List<TableDescriptor> writeFsLayout(Path rootDir, Configuration conf)
+      throws IOException {
+    LOG.info("BOOTSTRAP: creating hbase:root and hbase:meta region");
     FileSystem fs = rootDir.getFileSystem(conf);
-    Path tableDir = CommonFSUtils.getTableDir(rootDir, TableName.META_TABLE_NAME);
+    Path tableDir = CommonFSUtils.getTableDir(rootDir, TableName.ROOT_TABLE_NAME);
+    if (fs.exists(tableDir) && !fs.delete(tableDir, true)) {
+      LOG.warn("Cannot delete partial created root table, continue...");
+    }
+    tableDir = CommonFSUtils.getTableDir(rootDir, TableName.META_TABLE_NAME);
     if (fs.exists(tableDir) && !fs.delete(tableDir, true)) {
-      LOG.warn("Can not delete partial created meta table, continue...");
+      LOG.warn("Cannot delete partial created meta table, continue...");
     }
+
     // Bootstrapping, make sure blockcache is off. Else, one will be
     // created here in bootstrap and it'll need to be cleaned up. Better to
     // not make it in first place. Turn off block caching for bootstrap.
     // Enable after.
-    TableDescriptor metaDescriptor = FSTableDescriptors.tryUpdateAndGetMetaTableDescriptor(conf, fs,
-      rootDir, builder -> builder.setRegionReplication(
-        conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM)));
+    List<TableDescriptor> tableDescriptors = FSTableDescriptors
+      .tryUpdateAndGetCatalogTableDescriptor(conf, fs,
+        rootDir, builder -> builder.setRegionReplication(
+          conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM)));
+    HRegion
+      .createHRegion(RegionInfoBuilder.ROOT_REGIONINFO, rootDir, conf,
+        tableDescriptors.get(0), null).close();
     HRegion
-      .createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, rootDir, conf, metaDescriptor, null)
-      .close();
-    return metaDescriptor;
+      .createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, rootDir, conf,
+        tableDescriptors.get(1), null).close();
+    return tableDescriptors;
   }
 
   @Override
-  protected Flow executeFromState(MasterProcedureEnv env, InitMetaState state)
-    throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+  protected Flow executeFromState(MasterProcedureEnv env, InitRootState state)
+      throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
     LOG.debug("Execute {}", this);
     try {
       switch (state) {
-        case INIT_META_WRITE_FS_LAYOUT:
-          Configuration conf = env.getMasterConfiguration();
-          Path rootDir = CommonFSUtils.getRootDir(conf);
-          TableDescriptor td = writeFsLayout(rootDir, conf);
-          env.getMasterServices().getTableDescriptors().update(td, true);
-          setNextState(InitMetaState.INIT_META_ASSIGN_META);
+        case INIT_ROOT_WRITE_FS_LAYOUT:
+            Configuration conf = env.getMasterConfiguration();
+            Path rootDir = CommonFSUtils.getRootDir(conf);
+            List<TableDescriptor> tds = writeFsLayout(rootDir, conf);
+            for (TableDescriptor td : tds) {
+              env.getMasterServices().getTableDescriptors().update(td, true);
+            }
+          setNextState(InitRootState.INIT_ROOT_ASSIGN_ROOT);
           return Flow.HAS_MORE_STATE;
-        case INIT_META_ASSIGN_META:
-          LOG.info("Going to assign meta");
+        case INIT_ROOT_ASSIGN_ROOT:
+          LOG.info("Going to assign root");
           addChildProcedure(env.getAssignmentManager()
-            .createAssignProcedures(Arrays.asList(RegionInfoBuilder.FIRST_META_REGIONINFO)));
-          setNextState(InitMetaState.INIT_META_CREATE_NAMESPACES);
+            .createAssignProcedures(Arrays.asList(RegionInfoBuilder.ROOT_REGIONINFO)));
+          setNextState(MasterProcedureProtos.InitRootState.INIT_ROOT_LOAD_ROOT);
           return Flow.HAS_MORE_STATE;
-        case INIT_META_CREATE_NAMESPACES:
-          LOG.info("Going to create {} and {} namespaces", DEFAULT_NAMESPACE, SYSTEM_NAMESPACE);
-          createDirectory(env, DEFAULT_NAMESPACE);
-          createDirectory(env, SYSTEM_NAMESPACE);
-          // here the TableNamespaceManager has not been initialized yet, so we have to insert the
-          // record directly into meta table, later the TableNamespaceManager will load these two
-          // namespaces when starting.
-          insertNamespaceToMeta(env.getMasterServices().getConnection(), DEFAULT_NAMESPACE);
-          insertNamespaceToMeta(env.getMasterServices().getConnection(), SYSTEM_NAMESPACE);
-
+        case INIT_ROOT_LOAD_ROOT:
+          try {
+            addMetaRegionToRoot(env);
+            env.getAssignmentManager().loadRoot();
+          } catch (IOException e) {
+            if (retryCounter == null) {
+              retryCounter = ProcedureUtil.createRetryCounter(env.getMasterConfiguration());
+            }
+            long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
+            LOG.warn("Failed to init default and system namespaces, suspend {}secs", backoff, e);
+            setTimeout(Math.toIntExact(backoff));
+            setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
+            skipPersistence();
+            throw new ProcedureSuspendedException();
+          }
+        case INIT_ROOT_INIT_META:
+          addChildProcedure(new InitMetaProcedure());
           return Flow.NO_MORE_STATE;
         default:
           throw new UnsupportedOperationException("unhandled state=" + state);
@@ -130,7 +152,7 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMe
         retryCounter = ProcedureUtil.createRetryCounter(env.getMasterConfiguration());
       }
       long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
-      LOG.warn("Failed to init meta, suspend {}secs", backoff, e);
+      LOG.warn("Failed to init root, suspend {}secs", backoff, e);
       setTimeout(Math.toIntExact(backoff));
       setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
       skipPersistence();
@@ -152,36 +174,44 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMe
   }
 
   @Override
-  protected void rollbackState(MasterProcedureEnv env, InitMetaState state)
-    throws IOException, InterruptedException {
+  protected LockState acquireLock(MasterProcedureEnv env) {
+    if (env.getProcedureScheduler().waitTableExclusiveLock(this, getTableName())) {
+      return LockState.LOCK_EVENT_WAIT;
+    }
+    return LockState.LOCK_ACQUIRED;
+  }
+
+  @Override
+  protected void rollbackState(MasterProcedureEnv env, InitRootState state)
+      throws IOException, InterruptedException {
     throw new UnsupportedOperationException();
   }
 
   @Override
-  protected InitMetaState getState(int stateId) {
-    return InitMetaState.forNumber(stateId);
+  protected InitRootState getState(int stateId) {
+    return InitRootState.forNumber(stateId);
   }
 
   @Override
-  protected int getStateId(InitMetaState state) {
+  protected int getStateId(InitRootState state) {
     return state.getNumber();
   }
 
   @Override
-  protected InitMetaState getInitialState() {
-    return InitMetaState.INIT_META_WRITE_FS_LAYOUT;
+  protected InitRootState getInitialState() {
+    return InitRootState.INIT_ROOT_WRITE_FS_LAYOUT;
   }
 
   @Override
   protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.serializeStateData(serializer);
-    serializer.serialize(InitMetaStateData.getDefaultInstance());
+    serializer.serialize(InitRootStateData.getDefaultInstance());
   }
 
   @Override
   protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
     super.deserializeStateData(serializer);
-    serializer.deserialize(InitMetaStateData.class);
+    serializer.deserialize(InitRootStateData.class);
   }
 
   @Override
@@ -192,4 +222,29 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMe
   public void await() throws InterruptedException {
     latch.await();
   }
+
+
+  public void addMetaRegionToRoot(MasterProcedureEnv env) throws IOException {
+    Table rooTable = env.getMasterServices().getConnection().getTable(TableName.ROOT_TABLE_NAME);
+    // The row key is the region name
+    byte[] row = RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName();
+    Put put = new Put(row);
+    final long now = EnvironmentEdgeManager.currentTime();
+    put.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
+      HConstants.REGIONINFO_QUALIFIER,
+      now,
+      RegionInfo.toByteArray(RegionInfoBuilder.FIRST_META_REGIONINFO)));
+    // Set into the root table the version of the meta table.
+    put.add(new KeyValue(row,
+      HConstants.CATALOG_FAMILY,
+      HConstants.META_VERSION_QUALIFIER,
+      now,
+      Bytes.toBytes(HConstants.META_VERSION)));
+    put.add(new KeyValue(row,
+      HConstants.CATALOG_FAMILY,
+      HConstants.STATE_QUALIFIER,
+      now,
+      Bytes.toBytes(RegionState.State.OFFLINE.name())));
+    rooTable.put(put);
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
index c6e77fd..5ee5f52 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
@@ -184,7 +184,13 @@ public final class MasterProcedureUtil {
    * server carrying meta, and 1 for others.
    */
   public static int getServerPriority(ServerProcedureInterface proc) {
-    return proc.hasMetaTableRegion() ? 100 : 1;
+    if (proc.hasRootTableRegion()) {
+      //TODO francis is 200 a reasonable number?
+      return 200;
+    } else if (proc.hasMetaTableRegion()) {
+      return 100;
+    }
+    return 1;
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
index 46621da..6d13587 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
@@ -26,7 +26,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
+import org.apache.hadoop.hbase.NotAllRootRegionsOnlineException;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.slf4j.Logger;
@@ -225,9 +225,11 @@ public final class ProcedureSyncWait {
   protected static void waitMetaRegions(final MasterProcedureEnv env) throws IOException {
     int timeout = env.getMasterConfiguration().getInt("hbase.client.catalog.timeout", 10000);
     try {
-      if (MetaTableLocator.waitMetaRegionLocation(env.getMasterServices().getZooKeeper(),
+      //TODO francis we actually need to check for meta here since it is called in
+      // CreateTableProcedure, we might need an equivalent root check too?
+      if (RootTableLocator.waitRootRegionLocation(env.getMasterServices().getZooKeeper(),
         timeout) == null) {
-        throw new NotAllMetaRegionsOnlineException();
+        throw new NotAllRootRegionsOnlineException();
       }
     } catch (InterruptedException e) {
       throw (InterruptedIOException) new InterruptedIOException().initCause(e);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
index 17606c3..4d3be8e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
@@ -80,6 +80,7 @@ public class ServerCrashProcedure
    */
   private List<RegionInfo> regionsOnCrashedServer;
 
+  private boolean carryingRoot = false;
   private boolean carryingMeta = false;
   private boolean shouldSplitWal;
   private MonitoredTask status;
@@ -91,12 +92,17 @@ public class ServerCrashProcedure
    * Call this constructor queuing up a Procedure.
    * @param serverName Name of the crashed server.
    * @param shouldSplitWal True if we should split WALs as part of crashed server processing.
-   * @param carryingMeta True if carrying hbase:meta table region.
+   * @param carryingRoot True if carrying hbase:root table region.
+   * @param carryingMeta True if carrying hbase:meta table region. Although carryingMeta is
+   *                     determined dynamically by an SCP instance. Caller can give the current
+   *                     state it sees, this information might be useful to SCP down the road or
+   *                     for debugging.
    */
   public ServerCrashProcedure(final MasterProcedureEnv env, final ServerName serverName,
-      final boolean shouldSplitWal, final boolean carryingMeta) {
+      final boolean shouldSplitWal, final boolean carryingRoot, final boolean carryingMeta) {
     this.serverName = serverName;
     this.shouldSplitWal = shouldSplitWal;
+    this.carryingRoot = carryingRoot;
     this.carryingMeta = carryingMeta;
     this.setOwner(env.getRequestUser());
   }
@@ -128,13 +134,34 @@ public class ServerCrashProcedure
 
     switch (state) {
       case SERVER_CRASH_START:
+        break;
+
+      //Don't block hbase:root processing states on hbase:meta being loaded
+      case SERVER_CRASH_SPLIT_ROOT_LOGS:
+      case SERVER_CRASH_DELETE_SPLIT_ROOT_WALS_DIR:
+      case SERVER_CRASH_ASSIGN_ROOT:
+        break;
+
+      case SERVER_CRASH_CHECK_CARRYING_META:
+        // If hbase:root is not loaded, we can't do the check so yield
+        if (env.getAssignmentManager().waitRootLoaded(this)) {
+          LOG.info("pid="+getProcId()+", waiting for root loaded: "+state+
+            ", carryingRoot="+carryingRoot+", carryingMeta="+carryingMeta);
+          throw new ProcedureSuspendedException();
+        }
+        break;
+
+      //Don't block hbase:meta processing states on hbase:meta being loaded
       case SERVER_CRASH_SPLIT_META_LOGS:
       case SERVER_CRASH_DELETE_SPLIT_META_WALS_DIR:
       case SERVER_CRASH_ASSIGN_META:
         break;
+
       default:
         // If hbase:meta is not assigned, yield.
         if (env.getAssignmentManager().waitMetaLoaded(this)) {
+          LOG.info("pid="+getProcId()+", waiting for meta loaded: "+state+
+            ", carryingRoot="+carryingRoot+", carryingMeta="+carryingMeta);
           throw new ProcedureSuspendedException();
         }
     }
@@ -143,7 +170,42 @@ public class ServerCrashProcedure
         case SERVER_CRASH_START:
           LOG.info("Start " + this);
           // If carrying meta, process it first. Else, get list of regions on crashed server.
-          if (this.carryingMeta) {
+          if (this.carryingRoot) {
+            setNextState(ServerCrashState.SERVER_CRASH_SPLIT_ROOT_LOGS);
+          } else  {
+            setNextState(ServerCrashState.SERVER_CRASH_CHECK_CARRYING_META);
+          }
+          break;
+        case SERVER_CRASH_SPLIT_ROOT_LOGS:
+          if (env.getMasterConfiguration().getBoolean(HBASE_SPLIT_WAL_COORDINATED_BY_ZK,
+            DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK)) {
+            zkCoordinatedSplitRootLogs(env);
+            setNextState(ServerCrashState.SERVER_CRASH_ASSIGN_ROOT);
+          } else {
+            am.getRegionStates().rootLogSplitting(serverName);
+            addChildProcedure(createSplittingWalProcedures(env, SplitWALManager.SplitType.ROOT));
+            setNextState(ServerCrashState.SERVER_CRASH_DELETE_SPLIT_ROOT_WALS_DIR);
+          }
+          break;
+        case SERVER_CRASH_DELETE_SPLIT_ROOT_WALS_DIR:
+          if (isSplittingDone(env, SplitWALManager.SplitType.ROOT)) {
+            setNextState(ServerCrashState.SERVER_CRASH_ASSIGN_ROOT);
+            am.getRegionStates().rootLogSplit(serverName);
+          } else {
+            setNextState(ServerCrashState.SERVER_CRASH_SPLIT_ROOT_LOGS);
+          }
+          break;
+        case SERVER_CRASH_ASSIGN_ROOT:
+          assignRegions(env, Arrays.asList(RegionInfoBuilder.ROOT_REGIONINFO));
+          setNextState(ServerCrashState.SERVER_CRASH_CHECK_CARRYING_META);
+          break;
+        case SERVER_CRASH_CHECK_CARRYING_META:
+          boolean currCarryingMeta = am.isCarryingMeta(serverName);
+          if (carryingMeta && !currCarryingMeta) {
+            LOG.error("pid="+getProcId()+", carryingMeta changed to false after SCP check");
+          }
+          carryingMeta = currCarryingMeta;
+          if (carryingMeta) {
             setNextState(ServerCrashState.SERVER_CRASH_SPLIT_META_LOGS);
           } else {
             setNextState(ServerCrashState.SERVER_CRASH_GET_REGIONS);
@@ -156,12 +218,12 @@ public class ServerCrashProcedure
             setNextState(ServerCrashState.SERVER_CRASH_ASSIGN_META);
           } else {
             am.getRegionStates().metaLogSplitting(serverName);
-            addChildProcedure(createSplittingWalProcedures(env, true));
+            addChildProcedure(createSplittingWalProcedures(env, SplitWALManager.SplitType.META));
             setNextState(ServerCrashState.SERVER_CRASH_DELETE_SPLIT_META_WALS_DIR);
           }
           break;
         case SERVER_CRASH_DELETE_SPLIT_META_WALS_DIR:
-          if (isSplittingDone(env, true)) {
+          if (isSplittingDone(env, SplitWALManager.SplitType.META)) {
             setNextState(ServerCrashState.SERVER_CRASH_ASSIGN_META);
             am.getRegionStates().metaLogSplit(serverName);
           } else {
@@ -195,12 +257,12 @@ public class ServerCrashProcedure
             setNextState(ServerCrashState.SERVER_CRASH_ASSIGN);
           } else {
             am.getRegionStates().logSplitting(this.serverName);
-            addChildProcedure(createSplittingWalProcedures(env, false));
+            addChildProcedure(createSplittingWalProcedures(env, SplitWALManager.SplitType.USER));
             setNextState(ServerCrashState.SERVER_CRASH_DELETE_SPLIT_WALS_DIR);
           }
           break;
         case SERVER_CRASH_DELETE_SPLIT_WALS_DIR:
-          if (isSplittingDone(env, false)) {
+          if (isSplittingDone(env, SplitWALManager.SplitType.USER)) {
             cleanupSplitDir(env);
             setNextState(ServerCrashState.SERVER_CRASH_ASSIGN);
             am.getRegionStates().logSplit(this.serverName);
@@ -257,7 +319,7 @@ public class ServerCrashProcedure
         // 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) {
@@ -265,11 +327,12 @@ public class ServerCrashProcedure
     }
   }
 
-  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);
@@ -277,11 +340,12 @@ public class ServerCrashProcedure
     }
   }
 
-  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()]);
   }
 
@@ -297,6 +361,16 @@ public class ServerCrashProcedure
     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);
+  }
+
   /**
    * Split hbase:meta logs using 'classic' zk-based coordination.
    * Superceded by procedure-based WAL splitting.
@@ -326,8 +400,11 @@ public class ServerCrashProcedure
     // of SCPs running because big cluster crashed down.
     am.getRegionStates().logSplitting(this.serverName);
     mwm.splitLog(this.serverName);
+    if (!carryingRoot) {
+      mwm.archiveCatalogLog(this.serverName, true);
+    }
     if (!carryingMeta) {
-      mwm.archiveMetaLog(this.serverName);
+      mwm.archiveCatalogLog(this.serverName, false);
     }
     am.getRegionStates().logSplit(this.serverName);
     LOG.debug("Done splitting WALs {}", this);
@@ -396,6 +473,8 @@ public class ServerCrashProcedure
     sb.append(getProcName());
     sb.append(", splitWal=");
     sb.append(shouldSplitWal);
+    sb.append(", root=");
+    sb.append(carryingRoot);
     sb.append(", meta=");
     sb.append(carryingMeta);
   }
@@ -412,6 +491,7 @@ public class ServerCrashProcedure
     MasterProcedureProtos.ServerCrashStateData.Builder state =
       MasterProcedureProtos.ServerCrashStateData.newBuilder().
       setServerName(ProtobufUtil.toServerName(this.serverName)).
+      setCarryingRoot(this.carryingRoot).
       setCarryingMeta(this.carryingMeta).
       setShouldSplitWal(this.shouldSplitWal);
     if (this.regionsOnCrashedServer != null && !this.regionsOnCrashedServer.isEmpty()) {
@@ -430,6 +510,7 @@ public class ServerCrashProcedure
     MasterProcedureProtos.ServerCrashStateData state =
         serializer.deserialize(MasterProcedureProtos.ServerCrashStateData.class);
     this.serverName = ProtobufUtil.toServerName(state.getServerName());
+    this.carryingRoot = state.hasCarryingRoot()? state.getCarryingRoot(): false;
     this.carryingMeta = state.hasCarryingMeta()? state.getCarryingMeta(): false;
     // shouldSplitWAL has a default over in pb so this invocation will always work.
     this.shouldSplitWal = state.getShouldSplitWal();
@@ -450,6 +531,11 @@ public class ServerCrashProcedure
   }
 
   @Override
+  public boolean hasRootTableRegion() {
+    return this.carryingRoot;
+  }
+
+  @Override
   public boolean hasMetaTableRegion() {
     return this.carryingMeta;
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerProcedureInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerProcedureInterface.java
index eb0583b..f769ee6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerProcedureInterface.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerProcedureInterface.java
@@ -49,6 +49,11 @@ public interface ServerProcedureInterface {
   /**
    * @return True if this server has an hbase:meta table region.
    */
+  boolean hasRootTableRegion();
+
+  /**
+   * @return True if this server has an hbase:meta table region.
+   */
   boolean hasMetaTableRegion();
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALProcedure.java
index 4ae408f..bcead65 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALProcedure.java
@@ -173,6 +173,10 @@ public class SplitWALProcedure
     return this.crashedServer;
   }
 
+  @Override public boolean hasRootTableRegion() {
+    return AbstractFSWALProvider.isRootFile(new Path(walPath));
+  }
+
   @Override
   public boolean hasMetaTableRegion() {
     return AbstractFSWALProvider.isMetaFile(new Path(walPath));
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALRemoteProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALRemoteProcedure.java
index 54607e6..912c623 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALRemoteProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALRemoteProcedure.java
@@ -123,6 +123,11 @@ public class SplitWALRemoteProcedure extends ServerRemoteProcedure
   }
 
   @Override
+  public boolean hasRootTableRegion() {
+    return AbstractFSWALProvider.isRootFile(new Path(walPath));
+  }
+
+  @Override
   public boolean hasMetaTableRegion() {
     return AbstractFSWALProvider.isMetaFile(new Path(walPath));
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchRpcThrottleProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchRpcThrottleProcedure.java
index 4dd84ca..c95cbfe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchRpcThrottleProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchRpcThrottleProcedure.java
@@ -143,6 +143,11 @@ public class SwitchRpcThrottleProcedure
   }
 
   @Override
+  public boolean hasRootTableRegion() {
+    return false;
+  }
+
+  @Override
   public boolean hasMetaTableRegion() {
     return false;
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchRpcThrottleRemoteProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchRpcThrottleRemoteProcedure.java
index 8ce9aef..2dc05f1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchRpcThrottleRemoteProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchRpcThrottleRemoteProcedure.java
@@ -90,6 +90,11 @@ public class SwitchRpcThrottleRemoteProcedure extends ServerRemoteProcedure
   }
 
   @Override
+  public boolean hasRootTableRegion() {
+    return false;
+  }
+
+  @Override
   public boolean hasMetaTableRegion() {
     return false;
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java
index 23d0263..06c0af8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/MasterSnapshotVerifier.java
@@ -23,7 +23,7 @@ import java.util.Map;
 import java.util.Set;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.CatalogAccessor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
@@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
 import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
 import org.slf4j.Logger;
@@ -158,10 +158,10 @@ public final class MasterSnapshotVerifier {
    */
   private void verifyRegions(final SnapshotManifest manifest) throws IOException {
     List<RegionInfo> regions;
-    if (TableName.META_TABLE_NAME.equals(tableName)) {
-      regions = MetaTableLocator.getMetaRegions(services.getZooKeeper());
+    if (TableName.ROOT_TABLE_NAME.equals(tableName)) {
+      regions = RootTableLocator.getRootRegions(services.getZooKeeper());
     } else {
-      regions = MetaTableAccessor.getTableRegions(services.getConnection(), tableName);
+      regions = CatalogAccessor.getTableRegions(services.getConnection(), tableName);
     }
     // Remove the non-default regions
     RegionReplicaUtil.removeNonDefaultRegions(regions);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
index 7c9496b..8f18ea9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
@@ -26,7 +26,7 @@ import java.util.concurrent.CancellationException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.CatalogAccessor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -49,7 +49,7 @@ import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -193,10 +193,10 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
 
       List<Pair<RegionInfo, ServerName>> regionsAndLocations;
       if (TableName.META_TABLE_NAME.equals(snapshotTable)) {
-        regionsAndLocations = MetaTableLocator.getMetaRegionsAndLocations(
+        regionsAndLocations = RootTableLocator.getRootRegionsAndLocations(
           server.getZooKeeper());
       } else {
-        regionsAndLocations = MetaTableAccessor.getTableRegionsAndLocations(
+        regionsAndLocations = CatalogAccessor.getTableRegionsAndLocations(
           server.getConnection(), snapshotTable, false);
       }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
index 65d88c6..420c715 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
@@ -247,7 +247,7 @@ public class MetaBrowser {
       HConstants.CATALOG_FAMILY,
       HConstants.STATE_QUALIFIER,
       CompareOperator.EQUAL,
-      // use the same serialization strategy as found in MetaTableAccessor#addRegionStateToPut
+      // use the same serialization strategy as found in CatalogAccessor#addRegionStateToPut
       Bytes.toBytes(state.name()));
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java
index c6a3b92..3fe20a6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/procedure/flush/MasterFlushTableProcedureManager.java
@@ -25,9 +25,9 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ThreadPoolExecutor;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CatalogAccessor;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
@@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -133,10 +133,10 @@ public class MasterFlushTableProcedureManager extends MasterProcedureManager {
     List<Pair<RegionInfo, ServerName>> regionsAndLocations;
 
     if (TableName.META_TABLE_NAME.equals(tableName)) {
-      regionsAndLocations = MetaTableLocator.getMetaRegionsAndLocations(
+      regionsAndLocations = RootTableLocator.getRootRegionsAndLocations(
         master.getZooKeeper());
     } else {
-      regionsAndLocations = MetaTableAccessor.getTableRegionsAndLocations(
+      regionsAndLocations = CatalogAccessor.getTableRegionsAndLocations(
         master.getConnection(), tableName, false);
     }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index cd5cecf..26abffd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -98,6 +98,8 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.RegionTooBusyException;
+import org.apache.hadoop.hbase.RootCellComparator;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.UnknownScannerException;
@@ -783,9 +785,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     // 'conf' renamed to 'confParam' b/c we use this.conf in the constructor
     this.baseConf = confParam;
     this.conf = new CompoundConfiguration().add(confParam).addBytesMap(htd.getValues());
-    this.cellComparator = htd.isMetaTable() ||
-      conf.getBoolean(USE_META_CELL_COMPARATOR, DEFAULT_USE_META_CELL_COMPARATOR) ?
-        MetaCellComparator.META_COMPARATOR : CellComparatorImpl.COMPARATOR;
+    if(htd.isRootTable()) {
+      this.cellComparator = RootCellComparator.ROOT_COMPARATOR;
+    } else if (htd.isMetaTable() ||
+      //TODO francis figure what support is needed for root for this
+      conf.getBoolean(USE_META_CELL_COMPARATOR, DEFAULT_USE_META_CELL_COMPARATOR)) {
+      this.cellComparator = MetaCellComparator.META_COMPARATOR;
+    } else {
+      this.cellComparator = CellComparatorImpl.COMPARATOR;
+    }
     this.lock = new ReentrantReadWriteLock(conf.getBoolean(FAIR_REENTRANT_CLOSE_LOCK,
         DEFAULT_FAIR_REENTRANT_CLOSE_LOCK));
     this.flushCheckInterval = conf.getInt(MEMSTORE_PERIODIC_FLUSH_INTERVAL,
@@ -7704,19 +7712,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * @return true if the row is within the range specified by the RegionInfo
    */
   public static boolean rowIsInRange(RegionInfo info, final byte [] row) {
-    return ((info.getStartKey().length == 0) ||
-        (Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
-        ((info.getEndKey().length == 0) ||
-            (Bytes.compareTo(info.getEndKey(), row) > 0));
+    return info.containsRow(row, 0, (short)row.length);
   }
 
   public static boolean rowIsInRange(RegionInfo info, final byte [] row, final int offset,
       final short length) {
-    return ((info.getStartKey().length == 0) ||
-        (Bytes.compareTo(info.getStartKey(), 0, info.getStartKey().length,
-          row, offset, length) <= 0)) &&
-        ((info.getEndKey().length == 0) ||
-          (Bytes.compareTo(info.getEndKey(), 0, info.getEndKey().length, row, offset, length) > 0));
+    return info.containsRow(row, offset, length);
   }
 
   @Override
@@ -8590,7 +8591,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    */
   public Optional<byte[]> checkSplit(boolean force) {
     // Can't split META
-    if (this.getRegionInfo().isMetaRegion()) {
+    if (this.getRegionInfo().isRootRegion()) {
       return Optional.empty();
     }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 6dd2e52..cabeb03 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -65,6 +65,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.CacheEvictionStats;
 import org.apache.hadoop.hbase.CallQueueTooBigException;
+import org.apache.hadoop.hbase.CatalogAccessor;
 import org.apache.hadoop.hbase.ChoreService;
 import org.apache.hadoop.hbase.ClockOutOfSyncException;
 import org.apache.hadoop.hbase.CoordinatedStateManager;
@@ -75,7 +76,6 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
 import org.apache.hadoop.hbase.HealthCheckChore;
-import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ScheduledChore;
@@ -177,7 +177,7 @@ import org.apache.hadoop.hbase.wal.WALFactory;
 import org.apache.hadoop.hbase.wal.WALProvider;
 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
 import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -1182,6 +1182,7 @@ public class HRegionServer extends Thread implements
     LOG.info("Exiting; stopping=" + this.serverName + "; zookeeper connection closed.");
   }
 
+  //TODO francis update this
   private boolean containsMetaTableRegions() {
     return onlineRegions.containsKey(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName());
   }
@@ -1990,6 +1991,8 @@ public class HRegionServer extends Thread implements
     // Start executor services
     this.executorService.startExecutorService(ExecutorType.RS_OPEN_REGION,
         conf.getInt("hbase.regionserver.executor.openregion.threads", 3));
+    this.executorService.startExecutorService(ExecutorType.RS_OPEN_ROOT,
+      conf.getInt("hbase.regionserver.executor.openroot.threads", 1));
     this.executorService.startExecutorService(ExecutorType.RS_OPEN_META,
         conf.getInt("hbase.regionserver.executor.openmeta.threads", 1));
     this.executorService.startExecutorService(ExecutorType.RS_OPEN_PRIORITY_REGION,
@@ -2344,7 +2347,7 @@ public class HRegionServer extends Thread implements
       Preconditions.checkArgument(hris != null && hris.length == 1);
       if (hris[0].isMetaRegion()) {
         try {
-          MetaTableLocator.setMetaLocation(getZooKeeper(), serverName,
+          RootTableLocator.setRootLocation(getZooKeeper(), serverName,
               hris[0].getReplicaId(), RegionState.State.OPEN);
         } catch (KeeperException e) {
           LOG.info("Failed to update meta location", e);
@@ -2352,7 +2355,7 @@ public class HRegionServer extends Thread implements
         }
       } else {
         try {
-          MetaTableAccessor.updateRegionLocation(asyncClusterConnection.toConnection(), hris[0],
+          CatalogAccessor.updateRegionLocation(asyncClusterConnection.toConnection(), hris[0],
               serverName, openSeqNum, masterSystemTime);
         } catch (IOException e) {
           LOG.info("Failed to update meta", e);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index a05d4a6..7de219a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -2116,7 +2116,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
     this.lock.readLock().lock();
     try {
       // Should already be enforced by the split policy!
-      assert !this.getRegionInfo().isMetaRegion();
+      assert !this.getRegionInfo().isRootRegion();
       // Not split-able if we find a reference store file present in the store.
       if (hasReferences()) {
         LOG.trace("Not splittable; has references: {}", this);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 4c0b621..28681a3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -128,6 +128,7 @@ import org.apache.hadoop.hbase.regionserver.handler.AssignRegionHandler;
 import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
 import org.apache.hadoop.hbase.regionserver.handler.OpenPriorityRegionHandler;
 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
+import org.apache.hadoop.hbase.regionserver.handler.OpenRootHandler;
 import org.apache.hadoop.hbase.regionserver.handler.UnassignRegionHandler;
 import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
 import org.apache.hadoop.hbase.replication.ReplicationUtils;
@@ -2143,7 +2144,10 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
           if (regionServer.executorService == null) {
             LOG.info("No executor executorService; skipping open request");
           } else {
-            if (region.isMetaRegion()) {
+            if (region.isRootRegion()) {
+              regionServer.executorService.submit(new OpenRootHandler(
+                  regionServer, regionServer,region, htd, masterSystemTime));
+            } else if (region.isMetaRegion()) {
               regionServer.executorService.submit(new OpenMetaHandler(
               regionServer, regionServer, region, htd, masterSystemTime));
             } else {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
index 737f165..e6b3c38 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/AssignRegionHandler.java
@@ -167,7 +167,9 @@ public class AssignRegionHandler extends EventHandler {
   public static AssignRegionHandler create(HRegionServer server, RegionInfo regionInfo,
       long openProcId, TableDescriptor tableDesc, long masterSystemTime) {
     EventType eventType;
-    if (regionInfo.isMetaRegion()) {
+    if (regionInfo.isRootRegion()) {
+      eventType = EventType.M_RS_OPEN_ROOT;
+    } else if (regionInfo.isMetaRegion()) {
       eventType = EventType.M_RS_OPEN_META;
     } else if (regionInfo.getTable().isSystemTable() ||
       (tableDesc != null && tableDesc.getPriority() >= HConstants.ADMIN_QOS)) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java
similarity index 51%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java
rename to hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java
index c51fccb..7a06733 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/NotAllMetaRegionsOnlineException.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/handler/OpenRootHandler.java
@@ -16,28 +16,25 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.hadoop.hbase.regionserver.handler;
 
-package org.apache.hadoop.hbase;
-
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.executor.EventType;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.yetus.audience.InterfaceAudience;
 
 /**
- * Thrown when an operation requires the root and all meta regions to be online
+ * Handles opening of a meta region on a region server.
+ * <p>
+ * This is executed after receiving an OPEN RPC from the master for meta.
  */
-@InterfaceAudience.Public
-public class NotAllMetaRegionsOnlineException extends DoNotRetryIOException {
-  private static final long serialVersionUID = 6439786157874827523L;
-  /**
-   * default constructor
-   */
-  public NotAllMetaRegionsOnlineException() {
-    super();
-  }
-
-  /**
-   * @param message
-   */
-  public NotAllMetaRegionsOnlineException(String message) {
-    super(message);
+@InterfaceAudience.Private
+public class OpenRootHandler extends OpenRegionHandler {
+  public OpenRootHandler(final Server server,
+                         final RegionServerServices rsServices, RegionInfo regionInfo,
+                         final TableDescriptor htd, long masterSystemTime) {
+    super(server, rsServices, regionInfo, htd, masterSystemTime, EventType.M_RS_OPEN_ROOT);
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
index afff1c1..02bb084 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.util;
 
 import edu.umd.cs.findbugs.annotations.Nullable;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
@@ -119,21 +120,43 @@ public class FSTableDescriptors implements TableDescriptors {
   }
 
   @VisibleForTesting
-  public static void tryUpdateMetaTableDescriptor(Configuration conf) throws IOException {
-    tryUpdateAndGetMetaTableDescriptor(conf, CommonFSUtils.getCurrentFileSystem(conf),
+  public static List<TableDescriptor> tryUpdateCatalogTableDescriptor(Configuration conf)
+      throws IOException {
+    return tryUpdateAndGetCatalogTableDescriptor(conf, CommonFSUtils.getCurrentFileSystem(conf),
       CommonFSUtils.getRootDir(conf), null);
   }
 
-  public static TableDescriptor tryUpdateAndGetMetaTableDescriptor(Configuration conf,
+  public static List<TableDescriptor> tryUpdateAndGetCatalogTableDescriptor(Configuration conf,
     FileSystem fs, Path rootdir,
-    Function<TableDescriptorBuilder, TableDescriptorBuilder> metaObserver) throws IOException {
+    Function<TableDescriptorBuilder, TableDescriptorBuilder> catalogObserver) throws IOException {
+    List<TableDescriptor> descriptors = new ArrayList<>();
+    // see if we already have root descriptor on fs. Write one if not.
+    try {
+      descriptors.add(getTableDescriptorFromFs(fs, rootdir, TableName.ROOT_TABLE_NAME));
+    } catch (TableInfoMissingException e) {
+      TableDescriptorBuilder builder = createRootTableDescriptorBuilder(conf);
+      if (catalogObserver != null) {
+        builder = catalogObserver.apply(builder);
+      }
+      TableDescriptor td = builder.build();
+      LOG.info("Creating new hbase:root table descriptor {}", td);
+      TableName tableName = td.getTableName();
+      Path tableDir = CommonFSUtils.getTableDir(rootdir, tableName);
+      Path p = writeTableDescriptor(fs, td, tableDir, getTableInfoPath(fs, tableDir, true));
+      if (p == null) {
+        throw new IOException("Failed update hbase:root  table descriptor");
+      }
+      LOG.info("Updated hbase:root table descriptor to {}", p);
+      descriptors.add(td);
+    }
+
     // see if we already have meta descriptor on fs. Write one if not.
     try {
-      return getTableDescriptorFromFs(fs, rootdir, TableName.META_TABLE_NAME);
+      descriptors.add(getTableDescriptorFromFs(fs, rootdir, TableName.META_TABLE_NAME));
     } catch (TableInfoMissingException e) {
       TableDescriptorBuilder builder = createMetaTableDescriptorBuilder(conf);
-      if (metaObserver != null) {
-        builder = metaObserver.apply(builder);
+      if (catalogObserver != null) {
+        builder = catalogObserver.apply(builder);
       }
       TableDescriptor td = builder.build();
       LOG.info("Creating new hbase:meta table descriptor {}", td);
@@ -144,8 +167,10 @@ public class FSTableDescriptors implements TableDescriptors {
         throw new IOException("Failed update hbase:meta table descriptor");
       }
       LOG.info("Updated hbase:meta table descriptor to {}", p);
-      return td;
+      descriptors.add(td);
     }
+    assert descriptors.size() == 2;
+    return descriptors;
   }
 
   private static TableDescriptorBuilder createMetaTableDescriptorBuilder(final Configuration conf)
@@ -197,6 +222,55 @@ public class FSTableDescriptors implements TableDescriptors {
         .setPriority(Coprocessor.PRIORITY_SYSTEM).build());
   }
 
+  private static TableDescriptorBuilder createRootTableDescriptorBuilder(final Configuration conf)
+    throws IOException {
+    // TODO We used to set CacheDataInL1 for ROOT table. When we have BucketCache in file mode, now
+    // the ROOT table data goes to File mode BC only. Test how that affect the system. If too much,
+    // we have to rethink about adding back the setCacheDataInL1 for ROOT table CFs.
+    return TableDescriptorBuilder.newBuilder(TableName.ROOT_TABLE_NAME)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(HConstants.CATALOG_FAMILY)
+        .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+          HConstants.DEFAULT_HBASE_META_VERSIONS))
+        .setInMemory(true)
+        .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+          HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
+        .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+        .setBloomFilterType(BloomType.ROWCOL)
+        .setDataBlockEncoding(org.apache.hadoop.hbase.io.encoding.DataBlockEncoding.ROW_INDEX_V1)
+        .build())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(HConstants.TABLE_FAMILY)
+        .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+          HConstants.DEFAULT_HBASE_META_VERSIONS))
+        .setInMemory(true)
+        .setBlocksize(8 * 1024)
+        .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+        .setDataBlockEncoding(org.apache.hadoop.hbase.io.encoding.DataBlockEncoding.ROW_INDEX_V1)
+        .setBloomFilterType(BloomType.ROWCOL)
+        .build())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder
+        .newBuilder(HConstants.REPLICATION_BARRIER_FAMILY)
+        .setMaxVersions(HConstants.ALL_VERSIONS)
+        .setInMemory(true)
+        .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+        .setDataBlockEncoding(org.apache.hadoop.hbase.io.encoding.DataBlockEncoding.ROW_INDEX_V1)
+        .setBloomFilterType(BloomType.ROWCOL)
+        .build())
+      .setColumnFamily(ColumnFamilyDescriptorBuilder
+        .newBuilder(HConstants.NAMESPACE_FAMILY)
+        .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+          HConstants.DEFAULT_HBASE_META_VERSIONS))
+        .setInMemory(true)
+        .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+          HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
+        .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+        .setDataBlockEncoding(org.apache.hadoop.hbase.io.encoding.DataBlockEncoding.ROW_INDEX_V1)
+        .setBloomFilterType(BloomType.ROWCOL)
+        .build())
+      .setCoprocessor(CoprocessorDescriptorBuilder.newBuilder(
+        MultiRowMutationEndpoint.class.getName())
+        .setPriority(Coprocessor.PRIORITY_SYSTEM).build());
+  }
+
   @VisibleForTesting
   protected boolean isUsecache() {
     return this.usecache;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
index 94ae704..6d2087d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
@@ -241,6 +241,11 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
 
   // should be package private; more visible for use in AbstractFSWAL
   public static final String WAL_FILE_NAME_DELIMITER = ".";
+
+  /** The hbase:root region's WAL filename extension */
+  @VisibleForTesting
+  public static final String ROOT_WAL_PROVIDER_ID = ".root";
+
   /** The hbase:meta region's WAL filename extension */
   @VisibleForTesting
   public static final String META_WAL_PROVIDER_ID = ".meta";
@@ -407,6 +412,25 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
     return serverName;
   }
 
+  public static boolean isCatalogFile(Path p) {
+    return isRootFile(p) || isMetaFile(p);
+  }
+
+  public static boolean isCatalogFile(String p) {
+    return isRootFile(p) || isMetaFile(p);
+  }
+
+  public static boolean isRootFile(Path p) {
+    return isRootFile(p.getName());
+  }
+
+  public static boolean isRootFile(String p) {
+    if (p != null && p.endsWith(ROOT_WAL_PROVIDER_ID)) {
+      return true;
+    }
+    return false;
+  }
+
   public static boolean isMetaFile(Path p) {
     return isMetaFile(p.getName());
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
index 062b368..f76a532 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
@@ -65,10 +65,16 @@ public class AsyncFSWALProvider extends AbstractFSWALProvider<AsyncFSWAL> {
 
   @Override
   protected AsyncFSWAL createWAL() throws IOException {
+    String suffix = null;
+    if (ROOT_WAL_PROVIDER_ID.equals(providerId)) {
+      suffix = ROOT_WAL_PROVIDER_ID;
+    } else if (META_WAL_PROVIDER_ID.equals(providerId)) {
+      suffix = META_WAL_PROVIDER_ID;
+    }
     return new AsyncFSWAL(CommonFSUtils.getWALFileSystem(conf), CommonFSUtils.getWALRootDir(conf),
         getWALDirectoryName(factory.factoryId),
         getWALArchiveDirectoryName(conf, factory.factoryId), conf, listeners, true, logPrefix,
-        META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null,
+        suffix,
         eventLoopGroup, channelClass);
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
index 9623bd1..f0ef555 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hbase.wal;
 
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.apache.hadoop.hbase.TableName.ROOT_TABLE_NAME;
 import java.io.IOException;
 import java.io.InterruptedIOException;
 import java.util.HashMap;
@@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.CellComparatorImpl;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.MetaCellComparator;
 import org.apache.hadoop.hbase.PrivateCellUtil;
+import org.apache.hadoop.hbase.RootCellComparator;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
@@ -77,6 +79,7 @@ public class BoundedRecoveredHFilesOutputSink extends OutputSink {
     Map<String, CellSet> familyCells = new HashMap<>();
     Map<String, Long> familySeqIds = new HashMap<>();
     boolean isMetaTable = buffer.tableName.equals(META_TABLE_NAME);
+    boolean isRootTable = buffer.tableName.equals(ROOT_TABLE_NAME);
     // First iterate all Cells to find which column families are present and to stamp Cell with
     // sequence id.
     for (WAL.Entry entry : buffer.entryBuffer) {
@@ -92,7 +95,9 @@ public class BoundedRecoveredHFilesOutputSink extends OutputSink {
         familyCells
             .computeIfAbsent(familyName,
               key -> new CellSet(
-                  isMetaTable ? MetaCellComparator.META_COMPARATOR : CellComparatorImpl.COMPARATOR))
+                isRootTable ? RootCellComparator.ROOT_COMPARATOR :
+                  (isMetaTable ? MetaCellComparator.META_COMPARATOR :
+                    CellComparatorImpl.COMPARATOR)))
             .add(cell);
         familySeqIds.compute(familyName, (k, v) -> v == null ? seqId : Math.max(v, seqId));
       }
@@ -103,7 +108,7 @@ public class BoundedRecoveredHFilesOutputSink extends OutputSink {
     for (Map.Entry<String, CellSet> cellsEntry : familyCells.entrySet()) {
       String familyName = cellsEntry.getKey();
       StoreFileWriter writer = createRecoveredHFileWriter(buffer.tableName, regionName,
-        familySeqIds.get(familyName), familyName, isMetaTable);
+        familySeqIds.get(familyName), familyName, isRootTable, isMetaTable);
       LOG.trace("Created {}", writer.getPath());
       openingWritersNum.incrementAndGet();
       try {
@@ -192,7 +197,7 @@ public class BoundedRecoveredHFilesOutputSink extends OutputSink {
    *   given hfile has metadata on how it was written.
    */
   private StoreFileWriter createRecoveredHFileWriter(TableName tableName, String regionName,
-      long seqId, String familyName, boolean isMetaTable) throws IOException {
+      long seqId, String familyName, boolean isRootTable, boolean isMetaTable) throws IOException {
     Path outputDir = WALSplitUtil.tryCreateRecoveredHFilesDir(walSplitter.rootFS, walSplitter.conf,
       tableName, regionName, familyName);
     StoreFileWriter.Builder writerBuilder =
@@ -201,8 +206,9 @@ public class BoundedRecoveredHFilesOutputSink extends OutputSink {
     HFileContext hFileContext = new HFileContextBuilder().
       withChecksumType(HStore.getChecksumType(walSplitter.conf)).
       withBytesPerCheckSum(HStore.getBytesPerChecksum(walSplitter.conf)).
-      withCellComparator(isMetaTable?
-        MetaCellComparator.META_COMPARATOR: CellComparatorImpl.COMPARATOR).build();
+      withCellComparator(
+        isRootTable ? RootCellComparator.ROOT_COMPARATOR :
+          (isMetaTable? MetaCellComparator.META_COMPARATOR: CellComparatorImpl.COMPARATOR)).build();
     return writerBuilder.withFileContext(hFileContext).build();
   }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/FSHLogProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/FSHLogProvider.java
index 3b91c24..458133f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/FSHLogProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/FSHLogProvider.java
@@ -101,10 +101,16 @@ public class FSHLogProvider extends AbstractFSWALProvider<FSHLog> {
 
   @Override
   protected FSHLog createWAL() throws IOException {
+    String suffix = null;
+    if (ROOT_WAL_PROVIDER_ID.equals(providerId)) {
+      suffix = ROOT_WAL_PROVIDER_ID;
+    } else if (META_WAL_PROVIDER_ID.equals(providerId)) {
+      suffix = META_WAL_PROVIDER_ID;
+    }
     return new FSHLog(CommonFSUtils.getWALFileSystem(conf), CommonFSUtils.getWALRootDir(conf),
         getWALDirectoryName(factory.factoryId),
         getWALArchiveDirectoryName(conf, factory.factoryId), conf, listeners, true, logPrefix,
-        META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null);
+        suffix);
   }
 
   @Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
index 764d3d5..ae71cb7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.wal;
 
 import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.META_WAL_PROVIDER_ID;
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.ROOT_WAL_PROVIDER_ID;
 import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
 
 import java.io.IOException;
@@ -124,6 +125,7 @@ public class RegionGroupingProvider implements WALProvider {
   public static final String DEFAULT_DELEGATE_PROVIDER = WALFactory.Providers.defaultProvider
       .name();
 
+  private static final String ROOT_WAL_GROUP_NAME = "root";
   private static final String META_WAL_GROUP_NAME = "meta";
 
   /** A group-provider mapping, make sure one-one rather than many-one mapping */
@@ -146,8 +148,8 @@ public class RegionGroupingProvider implements WALProvider {
     this.conf = conf;
     this.factory = factory;
 
-    if (META_WAL_PROVIDER_ID.equals(providerId)) {
-      // do not change the provider id if it is for meta
+    if (ROOT_WAL_PROVIDER_ID.equals(providerId) || META_WAL_PROVIDER_ID.equals(providerId)) {
+      // do not change the provider id if it is for root/meta
       this.providerId = providerId;
     } else {
       StringBuilder sb = new StringBuilder().append(factory.factoryId);
@@ -169,9 +171,15 @@ public class RegionGroupingProvider implements WALProvider {
   }
 
   private WALProvider createProvider(String group) throws IOException {
+    String suffix = group;
+    if (ROOT_WAL_PROVIDER_ID.equals(providerId)) {
+      suffix = ROOT_WAL_PROVIDER_ID;
+    } else if (META_WAL_PROVIDER_ID.equals(providerId)) {
+      suffix = META_WAL_PROVIDER_ID;
+    }
+
     WALProvider provider = WALFactory.createProvider(providerClass);
-    provider.init(factory, conf,
-      META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : group);
+    provider.init(factory, conf, suffix);
     provider.addWALActionsListener(new MetricsWAL());
     return provider;
   }
@@ -202,7 +210,9 @@ public class RegionGroupingProvider implements WALProvider {
   @Override
   public WAL getWAL(RegionInfo region) throws IOException {
     String group;
-    if (META_WAL_PROVIDER_ID.equals(this.providerId)) {
+    if (ROOT_WAL_PROVIDER_ID.equals(providerId)) {
+      group = ROOT_WAL_GROUP_NAME;
+    } else if (META_WAL_PROVIDER_ID.equals(this.providerId)) {
       group = META_WAL_GROUP_NAME;
     } else {
       byte[] id;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
index 26b8727..2e0bb71 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
@@ -81,12 +81,20 @@ public class WALFactory {
   public static final String WAL_PROVIDER = "hbase.wal.provider";
   static final String DEFAULT_WAL_PROVIDER = Providers.defaultProvider.name();
 
+  public static final String ROOT_WAL_PROVIDER = "hbase.wal.root_provider";
+
   public static final String META_WAL_PROVIDER = "hbase.wal.meta_provider";
 
   public static final String WAL_ENABLED = "hbase.regionserver.hlog.enabled";
 
   final String factoryId;
   private final WALProvider provider;
+
+  // The root updates are written to a different wal. If this
+  // regionserver holds root regions, then this ref will be non-null.
+  // lazily intialized; most RegionServers don't deal with ROOT
+  private final AtomicReference<WALProvider> rootProvider = new AtomicReference<>();
+
   // The meta updates are written to a different wal. If this
   // regionserver holds meta regions, then this ref will be non-null.
   // lazily intialized; most RegionServers don't deal with META
@@ -219,6 +227,10 @@ public class WALFactory {
    * factory.
    */
   public void close() throws IOException {
+    final WALProvider rootProvider = this.rootProvider.get();
+    if (null != rootProvider) {
+      rootProvider.close();
+    }
     final WALProvider metaProvider = this.metaProvider.get();
     if (null != metaProvider) {
       metaProvider.close();
@@ -237,6 +249,14 @@ public class WALFactory {
    */
   public void shutdown() throws IOException {
     IOException exception = null;
+    final WALProvider rootProvider = this.rootProvider.get();
+    if (null != rootProvider) {
+      try {
+        rootProvider.shutdown();
+      } catch(IOException ioe) {
+        exception = ioe;
+      }
+    }
     final WALProvider metaProvider = this.metaProvider.get();
     if (null != metaProvider) {
       try {
@@ -256,6 +276,36 @@ public class WALFactory {
   }
 
   @VisibleForTesting
+  WALProvider getRootProvider() throws IOException {
+    for (;;) {
+      WALProvider provider = this.rootProvider.get();
+      if (provider != null) {
+        return provider;
+      }
+      Class<? extends WALProvider> clz = null;
+      if (conf.get(ROOT_WAL_PROVIDER) == null) {
+        try {
+          clz = conf.getClass(WAL_PROVIDER, Providers.defaultProvider.clazz, WALProvider.class);
+        } catch (Throwable t) {
+          // the WAL provider should be an enum. Proceed
+        }
+      }
+      if (clz == null){
+        clz = getProviderClass(ROOT_WAL_PROVIDER, conf.get(WAL_PROVIDER, DEFAULT_WAL_PROVIDER));
+      }
+      provider = createProvider(clz);
+      provider.init(this, conf, AbstractFSWALProvider.ROOT_WAL_PROVIDER_ID);
+      provider.addWALActionsListener(new MetricsWAL());
+      if (rootProvider.compareAndSet(null, provider)) {
+        return provider;
+      } else {
+        // someone is ahead of us, close and try again.
+        provider.close();
+      }
+    }
+  }
+
+  @VisibleForTesting
   WALProvider getMetaProvider() throws IOException {
     for (;;) {
       WALProvider provider = this.metaProvider.get();
@@ -289,8 +339,11 @@ public class WALFactory {
    * @param region the region which we want to get a WAL for it. Could be null.
    */
   public WAL getWAL(RegionInfo region) throws IOException {
-    // use different WAL for hbase:meta
-    if (region != null && region.isMetaRegion() &&
+    // use different WAL for hbase:root and hbase:meta
+    if (region != null && region.isRootRegion() &&
+      region.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
+      return getRootProvider().getWAL(region);
+    } else if (region != null && region.isMetaRegion() &&
       region.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
       return getMetaProvider().getWAL(region);
     } else {
@@ -486,6 +539,10 @@ public class WALFactory {
     return this.provider;
   }
 
+  public final WALProvider getRootWALProvider() {
+    return this.rootProvider.get();
+  }
+
   public final WALProvider getMetaWALProvider() {
     return this.metaProvider.get();
   }
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
index e46b277..84c662b 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -65,7 +65,7 @@
 <%@ page import="org.apache.hadoop.hbase.quotas.ThrottleSettings" %>
 <%@ page import="org.apache.hadoop.hbase.util.Bytes" %>
 <%@ page import="org.apache.hadoop.hbase.util.FSUtils" %>
-<%@ page import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator" %>
+<%@ page import="org.apache.hadoop.hbase.zookeeper.RootTableLocator" %>
 <%@ page import="org.apache.hadoop.util.StringUtils" %>
 <%@ page import="org.apache.hbase.thirdparty.com.google.protobuf.ByteString" %>
 <%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos" %>
@@ -311,7 +311,7 @@
           for (int j = 0; j < numMetaReplicas; j++) {
             RegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica(
                                     RegionInfoBuilder.FIRST_META_REGIONINFO, j);
-            ServerName metaLocation = MetaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), j, 1);
+            ServerName metaLocation = RootTableLocator.waitRootRegionLocation(master.getZooKeeper(), j, 1);
             for (int i = 0; i < 1; i++) {
               String hostAndPort = "";
               String readReq = "N/A";
@@ -376,7 +376,7 @@
            for (int j = 0; j < numMetaReplicas; j++) {
              RegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica(
                                      RegionInfoBuilder.FIRST_META_REGIONINFO, j);
-             ServerName metaLocation = MetaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), j, 1);
+             ServerName metaLocation = RootTableLocator.waitRootRegionLocation(master.getZooKeeper(), j, 1);
              for (int i = 0; i < 1; i++) {
                String hostAndPort = "";
                float locality = 0.0f;
@@ -424,7 +424,7 @@
           for (int j = 0; j < numMetaReplicas; j++) {
             RegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica(
                                     RegionInfoBuilder.FIRST_META_REGIONINFO, j);
-            ServerName metaLocation = MetaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), j, 1);
+            ServerName metaLocation = RootTableLocator.waitRootRegionLocation(master.getZooKeeper(), j, 1);
             for (int i = 0; i < 1; i++) {
               String hostAndPort = "";
               long compactingCells = 0;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
index 85dff35..4c29b76 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
@@ -349,6 +349,13 @@ public abstract class HBaseCluster implements Closeable, Configurable {
   public boolean restoreClusterMetrics(ClusterMetrics desiredStatus) throws IOException {
     return true;
   }
+  /**
+   * Get the ServerName of region server serving the first hbase:meta region
+   */
+  public ServerName getServerHoldingRoot() throws IOException {
+    return getServerHoldingRegion(TableName.ROOT_TABLE_NAME,
+      RegionInfoBuilder.ROOT_REGIONINFO.getRegionName());
+  }
 
   /**
    * Get the ServerName of region server serving the first hbase:meta region
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCatalogAccessor.java
similarity index 85%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/TestCatalogAccessor.java
index dc4b6a8..05884d4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestCatalogAccessor.java
@@ -34,14 +34,18 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.ipc.CallRunner;
 import org.apache.hadoop.hbase.ipc.DelegatingRpcScheduler;
 import org.apache.hadoop.hbase.ipc.PriorityFunction;
@@ -53,9 +57,10 @@ import org.apache.hadoop.hbase.regionserver.SimpleRpcSchedulerFactory;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
@@ -70,16 +75,16 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 
 /**
- * Test {@link org.apache.hadoop.hbase.MetaTableAccessor}.
+ * Test {@link CatalogAccessor}.
  */
 @Category({ MiscTests.class, MediumTests.class })
 @SuppressWarnings("deprecation")
-public class TestMetaTableAccessor {
+public class TestCatalogAccessor {
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestMetaTableAccessor.class);
+    HBaseClassTestRule.forClass(TestCatalogAccessor.class);
 
-  private static final Logger LOG = LoggerFactory.getLogger(TestMetaTableAccessor.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestCatalogAccessor.class);
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
   private static Connection connection;
   private Random random = new Random();
@@ -97,6 +102,7 @@ public class TestMetaTableAccessor {
     c.setLong("hbase.client.pause", 1000);
     c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 10);
     connection = ConnectionFactory.createConnection(c);
+    CommonFSUtils.logFileSystemState(UTIL.getTestFileSystem(), UTIL.getDefaultRootDirPath(), LOG);
   }
 
   @AfterClass
@@ -108,7 +114,7 @@ public class TestMetaTableAccessor {
   @Test
   public void testIsMetaWhenAllHealthy() throws InterruptedException {
     HMaster m = UTIL.getMiniHBaseCluster().getMaster();
-    assertTrue(m.waitForMetaOnline());
+    assertTrue(m.getAssignmentManager().waitForMetaOnline());
   }
 
   @Test
@@ -117,11 +123,11 @@ public class TestMetaTableAccessor {
     int index = UTIL.getMiniHBaseCluster().getServerWithMeta();
     HRegionServer rsWithMeta = UTIL.getMiniHBaseCluster().getRegionServer(index);
     rsWithMeta.abort("TESTING");
-    assertTrue(m.waitForMetaOnline());
+    assertTrue(m.getAssignmentManager().waitForMetaOnline());
   }
 
   /**
-   * Does {@link MetaTableAccessor#getRegion(Connection, byte[])} and a write against hbase:meta
+   * Does {@link CatalogAccessor#getRegion(Connection, byte[])} and a write against hbase:meta
    * while its hosted server is restarted to prove our retrying works.
    */
   @Test
@@ -145,7 +151,7 @@ public class TestMetaTableAccessor {
     MetaTask writer = new MetaTask(connection, "writer") {
       @Override
       void metaTask() throws Throwable {
-        MetaTableAccessor.addRegionToMeta(connection, regions.get(0));
+        CatalogAccessor.addRegionToMeta(connection, regions.get(0));
         LOG.info("Wrote " + regions.get(0).getEncodedName());
       }
     };
@@ -174,6 +180,15 @@ public class TestMetaTableAccessor {
         } while (index == -1 && startTime + timeOut < System.currentTimeMillis());
 
         if (index != -1) {
+          //TODO francis check this
+          ServerName rootServerName =
+            UTIL.getMiniHBaseCluster().getRegionServer(index).getServerName();
+          UTIL.getAdmin().move(RegionInfoBuilder.ROOT_REGIONINFO.getEncodedNameAsBytes(),
+            rootServerName.toString().getBytes());
+          while (!UTIL.getConnection().getRegionLocator(TableName.ROOT_TABLE_NAME)
+            .getAllRegionLocations().get(0).getServerName().equals(rootServerName)) {
+            Thread.sleep(100);
+          }
           UTIL.getMiniHBaseCluster().abortRegionServer(index);
           UTIL.getMiniHBaseCluster().waitOnRegionServer(index);
         }
@@ -195,7 +210,7 @@ public class TestMetaTableAccessor {
   }
 
   /**
-   * Thread that runs a MetaTableAccessor task until asked stop.
+   * Thread that runs a CatalogAccessor task until asked stop.
    */
   abstract static class MetaTask extends Thread {
     boolean stop = false;
@@ -244,9 +259,9 @@ public class TestMetaTableAccessor {
 
   @Test
   public void testGetRegionsFromMetaTable() throws IOException, InterruptedException {
-    List<RegionInfo> regions = MetaTableLocator.getMetaRegions(UTIL.getZooKeeperWatcher());
+    List<RegionInfo> regions = RootTableLocator.getRootRegions(UTIL.getZooKeeperWatcher());
     assertTrue(regions.size() >= 1);
-    assertTrue(MetaTableLocator.getMetaRegionsAndLocations(UTIL.getZooKeeperWatcher()).size() >= 1);
+    assertTrue(RootTableLocator.getRootRegionsAndLocations(UTIL.getZooKeeperWatcher()).size() >= 1);
   }
 
   @Test
@@ -255,7 +270,7 @@ public class TestMetaTableAccessor {
     LOG.info("Started " + name);
     // Test get on non-existent region.
     Pair<RegionInfo, ServerName> pair =
-      MetaTableAccessor.getRegion(connection, Bytes.toBytes("nonexistent-region"));
+      CatalogAccessor.getRegion(connection, Bytes.toBytes("nonexistent-region"));
     assertNull(pair);
     LOG.info("Finished " + name);
   }
@@ -277,16 +292,16 @@ public class TestMetaTableAccessor {
 
     // Now make sure we only get the regions from 1 of the tables at a time
 
-    assertEquals(1, MetaTableAccessor.getTableRegions(connection, tableName).size());
-    assertEquals(1, MetaTableAccessor.getTableRegions(connection, greaterName).size());
+    assertEquals(1, CatalogAccessor.getTableRegions(connection, tableName).size());
+    assertEquals(1, CatalogAccessor.getTableRegions(connection, greaterName).size());
   }
 
   private static List<RegionInfo> testGettingTableRegions(final Connection connection,
     final TableName name, final int regionCount) throws IOException, InterruptedException {
-    List<RegionInfo> regions = MetaTableAccessor.getTableRegions(connection, name);
+    List<RegionInfo> regions = org.apache.hadoop.hbase.CatalogAccessor.getTableRegions(connection, name);
     assertEquals(regionCount, regions.size());
     Pair<RegionInfo, ServerName> pair =
-      MetaTableAccessor.getRegion(connection, regions.get(0).getRegionName());
+      CatalogAccessor.getRegion(connection, regions.get(0).getRegionName());
     assertEquals(regions.get(0).getEncodedName(), pair.getFirst().getEncodedName());
     return regions;
   }
@@ -294,7 +309,7 @@ public class TestMetaTableAccessor {
   private static void testGetRegion(final Connection connection, final RegionInfo region)
     throws IOException, InterruptedException {
     Pair<RegionInfo, ServerName> pair =
-      MetaTableAccessor.getRegion(connection, region.getRegionName());
+      CatalogAccessor.getRegion(connection, region.getRegionName());
     assertEquals(region.getEncodedName(), pair.getFirst().getEncodedName());
   }
 
@@ -319,15 +334,15 @@ public class TestMetaTableAccessor {
     long seqNum1 = random.nextLong();
     long seqNum100 = random.nextLong();
 
-    try (Table meta = MetaTableAccessor.getMetaHTable(connection)) {
-      MetaTableAccessor.updateRegionLocation(connection, primary, serverName0, seqNum0,
+    try (Table meta = CatalogAccessor.getCatalogHTable(connection, TableName.META_TABLE_NAME)) {
+      CatalogAccessor.updateRegionLocation(connection, primary, serverName0, seqNum0,
         EnvironmentEdgeManager.currentTime());
 
       // assert that the server, startcode and seqNum columns are there for the primary region
       assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true);
 
       // add replica = 1
-      MetaTableAccessor.updateRegionLocation(connection, replica1, serverName1, seqNum1,
+      CatalogAccessor.updateRegionLocation(connection, replica1, serverName1, seqNum1,
         EnvironmentEdgeManager.currentTime());
       // check whether the primary is still there
       assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true);
@@ -335,7 +350,7 @@ public class TestMetaTableAccessor {
       assertMetaLocation(meta, primary.getRegionName(), serverName1, seqNum1, 1, true);
 
       // add replica = 1
-      MetaTableAccessor.updateRegionLocation(connection, replica100, serverName100, seqNum100,
+      CatalogAccessor.updateRegionLocation(connection, replica100, serverName100, seqNum100,
         EnvironmentEdgeManager.currentTime());
       // check whether the primary is still there
       assertMetaLocation(meta, primary.getRegionName(), serverName0, seqNum0, 0, true);
@@ -384,12 +399,12 @@ public class TestMetaTableAccessor {
       .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(HConstants.EMPTY_END_ROW).setSplit(false)
       .setRegionId(regionId).setReplicaId(0).build();
 
-    Table meta = MetaTableAccessor.getMetaHTable(connection);
+    Table meta = CatalogAccessor.getCatalogHTable(connection, TableName.META_TABLE_NAME);
     try {
       List<RegionInfo> regionInfos = Lists.newArrayList(primary);
-      MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
-      MetaTableAccessor.removeRegionReplicasFromMeta(Sets.newHashSet(primary.getRegionName()), 1, 2,
-        connection);
+      CatalogAccessor.addRegionsToMeta(connection, regionInfos, 3);
+      CatalogAccessor.removeRegionReplicasFromCatalog(Sets.newHashSet(primary.getRegionName()),
+        1, 2, connection, TableName.META_TABLE_NAME);
       Get get = new Get(primary.getRegionName());
       Result result = meta.get(get);
       for (int replicaId = 0; replicaId < 3; replicaId++) {
@@ -422,10 +437,10 @@ public class TestMetaTableAccessor {
       .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(HConstants.EMPTY_END_ROW).setSplit(false)
       .setRegionId(regionId).setReplicaId(0).build();
 
-    Table meta = MetaTableAccessor.getMetaHTable(connection);
+    Table meta = CatalogAccessor.getCatalogHTable(connection, TableName.META_TABLE_NAME);
     try {
       List<RegionInfo> regionInfos = Lists.newArrayList(primary);
-      MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
+      CatalogAccessor.addRegionsToMeta(connection, regionInfos, 3);
 
       assertEmptyMetaLocation(meta, primary.getRegionName(), 1);
       assertEmptyMetaLocation(meta, primary.getRegionName(), 2);
@@ -460,27 +475,27 @@ public class TestMetaTableAccessor {
     doReturn(true).when(visitor).visit((Result) anyObject());
 
     // Scanning the entire table should give us three rows
-    MetaTableAccessor.scanMetaForTableRegions(connection, visitor, tableName);
+    CatalogAccessor.scanMetaForTableRegions(connection, visitor, tableName);
     verify(visitor, times(3)).visit((Result) anyObject());
 
     // Scanning the table with a specified empty start row should also
     // give us three hbase:meta rows
     reset(visitor);
     doReturn(true).when(visitor).visit((Result) anyObject());
-    MetaTableAccessor.scanMeta(connection, visitor, tableName, null, 1000);
+    CatalogAccessor.scanMeta(connection, visitor, tableName, null, 1000);
     verify(visitor, times(3)).visit((Result) anyObject());
 
     // Scanning the table starting in the middle should give us two rows:
     // region_a and region_b
     reset(visitor);
     doReturn(true).when(visitor).visit((Result) anyObject());
-    MetaTableAccessor.scanMeta(connection, visitor, tableName, Bytes.toBytes("region_ac"), 1000);
+    CatalogAccessor.scanMeta(connection, visitor, tableName, Bytes.toBytes("region_ac"), 1000);
     verify(visitor, times(2)).visit((Result) anyObject());
 
     // Scanning with a limit of 1 should only give us one row
     reset(visitor);
     doReturn(true).when(visitor).visit((Result) anyObject());
-    MetaTableAccessor.scanMeta(connection, visitor, tableName, Bytes.toBytes("region_ac"), 1);
+    CatalogAccessor.scanMeta(connection, visitor, tableName, Bytes.toBytes("region_ac"), 1);
     verify(visitor, times(1)).visit((Result) anyObject());
     table.close();
   }
@@ -496,12 +511,12 @@ public class TestMetaTableAccessor {
       .setRegionId(regionId).setReplicaId(0).build();
 
     ServerName sn = ServerName.valueOf("bar", 0, 0);
-    try (Table meta = MetaTableAccessor.getMetaHTable(connection)) {
+    try (Table meta = CatalogAccessor.getCatalogHTable(connection, TableName.META_TABLE_NAME)) {
       List<RegionInfo> regionInfos = Lists.newArrayList(regionInfo);
-      MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 1);
+      CatalogAccessor.addRegionsToMeta(connection, regionInfos, 1);
 
       long masterSystemTime = EnvironmentEdgeManager.currentTime() + 123456789;
-      MetaTableAccessor.updateRegionLocation(connection, regionInfo, sn, 1, masterSystemTime);
+      CatalogAccessor.updateRegionLocation(connection, regionInfo, sn, 1, masterSystemTime);
 
       Get get = new Get(regionInfo.getRegionName());
       Result result = meta.get(get);
@@ -556,7 +571,7 @@ public class TestMetaTableAccessor {
     final List<HRegion> regions = UTIL.getHBaseCluster().getRegions(tableName);
     final String encodedName = regions.get(0).getRegionInfo().getEncodedName();
     final Result result =
-      MetaTableAccessor.scanByRegionEncodedName(UTIL.getConnection(), encodedName);
+      CatalogAccessor.scanByRegionEncodedName(UTIL.getConnection(), encodedName);
     assertNotNull(result);
     assertTrue(result.advance());
     final String resultingRowKey = CellUtil.getCellKeyAsString(result.current());
@@ -568,7 +583,7 @@ public class TestMetaTableAccessor {
   public void testScanByRegionEncodedNameNonExistingRegion() throws Exception {
     final String encodedName = "nonexistingregion";
     final Result result =
-      MetaTableAccessor.scanByRegionEncodedName(UTIL.getConnection(), encodedName);
+      CatalogAccessor.scanByRegionEncodedName(UTIL.getConnection(), encodedName);
     assertNull(result);
   }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java
index b578ffe..0aeaaad 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java
@@ -204,7 +204,18 @@ public class TestClientClusterMetrics {
       .addColumn(CF, Bytes.toBytes("q3"), Bytes.toBytes("v3")));
 
     ClusterMetrics metrics = ADMIN.getClusterMetrics();
-    Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2);
+    Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 3);
+    Assert.assertEquals(metrics.getTableRegionStatesCount().get(TableName.ROOT_TABLE_NAME)
+      .getRegionsInTransition(), 0);
+    Assert.assertEquals(metrics.getTableRegionStatesCount().get(TableName.ROOT_TABLE_NAME)
+      .getOpenRegions(), 1);
+    Assert.assertEquals(metrics.getTableRegionStatesCount().get(TableName.ROOT_TABLE_NAME)
+      .getTotalRegions(), 1);
+    Assert.assertEquals(metrics.getTableRegionStatesCount().get(TableName.ROOT_TABLE_NAME)
+      .getClosedRegions(), 0);
+    Assert.assertEquals(metrics.getTableRegionStatesCount().get(TableName.ROOT_TABLE_NAME)
+      .getSplitRegions(), 0);
+
     Assert.assertEquals(metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME)
       .getRegionsInTransition(), 0);
     Assert.assertEquals(metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME)
@@ -215,6 +226,7 @@ public class TestClientClusterMetrics {
       .getClosedRegions(), 0);
     Assert.assertEquals(metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME)
       .getSplitRegions(), 0);
+
     Assert.assertEquals(metrics.getTableRegionStatesCount().get(TABLE_NAME)
       .getRegionsInTransition(), 0);
     Assert.assertEquals(metrics.getTableRegionStatesCount().get(TABLE_NAME)
@@ -238,7 +250,7 @@ public class TestClientClusterMetrics {
     insertData(TABLE_NAME, startRowNum, rowCount);
 
     ClusterMetrics metrics = ADMIN.getClusterMetrics();
-    Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2);
+    Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 3);
     Assert.assertEquals(metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME)
       .getRegionsInTransition(), 0);
     Assert.assertEquals(metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME)
@@ -259,13 +271,21 @@ public class TestClientClusterMetrics {
     ADMIN.split(TABLE_NAME, splitKey);
 
     metrics = ADMIN.getClusterMetrics();
-    Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 2);
+    Assert.assertEquals(metrics.getTableRegionStatesCount().size(), 3);
+    Assert.assertEquals(metrics.getTableRegionStatesCount().get(TableName.ROOT_TABLE_NAME)
+      .getRegionsInTransition(), 0);
+    Assert.assertEquals(metrics.getTableRegionStatesCount().get(TableName.ROOT_TABLE_NAME)
+      .getOpenRegions(), 1);
+    Assert.assertEquals(metrics.getTableRegionStatesCount().get(TableName.ROOT_TABLE_NAME)
+      .getTotalRegions(), 1);
+
     Assert.assertEquals(metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME)
       .getRegionsInTransition(), 0);
     Assert.assertEquals(metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME)
       .getOpenRegions(), 1);
     Assert.assertEquals(metrics.getTableRegionStatesCount().get(TableName.META_TABLE_NAME)
       .getTotalRegions(), 1);
+
     Assert.assertEquals(metrics.getTableRegionStatesCount().get(TABLE_NAME)
       .getRegionsInTransition(), 0);
     Assert.assertEquals(metrics.getTableRegionStatesCount().get(TABLE_NAME)
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java
index e73272e..bc55760 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java
@@ -23,8 +23,8 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
-import org.apache.hadoop.hbase.TestMetaTableAccessor.SpyingRpcScheduler;
-import org.apache.hadoop.hbase.TestMetaTableAccessor.SpyingRpcSchedulerFactory;
+import org.apache.hadoop.hbase.TestCatalogAccessor.SpyingRpcScheduler;
+import org.apache.hadoop.hbase.TestCatalogAccessor.SpyingRpcSchedulerFactory;
 import org.apache.hadoop.hbase.client.AsyncTable;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
@@ -122,13 +122,13 @@ public class TestMetaUpdatesGoToPriorityQueue {
     SpyingRpcScheduler scheduler = (SpyingRpcScheduler) rs.getRpcServer().getScheduler();
     long prevCalls = scheduler.numPriorityCalls;
     long time = System.currentTimeMillis();
-    Put putParent = MetaTableAccessor.makePutFromRegionInfo(
+    Put putParent = CatalogAccessor.makePutFromRegionInfo(
       RegionInfoBuilder.newBuilder(parent).setOffline(true).setSplit(true).build(), time);
-    MetaTableAccessor.addDaughtersToPut(putParent, splitA, splitB);
-    Put putA = MetaTableAccessor.makePutFromRegionInfo(splitA, time);
-    Put putB = MetaTableAccessor.makePutFromRegionInfo(splitB, time);
+    CatalogAccessor.addDaughtersToPut(putParent, splitA, splitB);
+    Put putA = CatalogAccessor.makePutFromRegionInfo(splitA, time);
+    Put putB = CatalogAccessor.makePutFromRegionInfo(splitB, time);
     multiMutate(putParent.getRow(), Arrays.asList(putParent, putA, putB));
 
     assertTrue(prevCalls < scheduler.numPriorityCalls);
   }
-}
\ No newline at end of file
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRootTableLocator.java
similarity index 79%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/TestRootTableLocator.java
index 9274fa0..1205bac 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestRootTableLocator.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.zookeeper.KeeperException;
 import org.junit.After;
@@ -48,16 +48,16 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse;
 
 /**
- * Test {@link org.apache.hadoop.hbase.zookeeper.MetaTableLocator}
+ * Test {@link RootTableLocator}
  */
 @Category({ MiscTests.class, MediumTests.class })
-public class TestMetaTableLocator {
+public class TestRootTableLocator {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestMetaTableLocator.class);
+    HBaseClassTestRule.forClass(TestRootTableLocator.class);
 
-  private static final Logger LOG = LoggerFactory.getLogger(TestMetaTableLocator.class);
+  private static final Logger LOG = LoggerFactory.getLogger(TestRootTableLocator.class);
   private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
   private static final ServerName SN =
     ServerName.valueOf("example.org", 1234, System.currentTimeMillis());
@@ -98,7 +98,7 @@ public class TestMetaTableLocator {
     try {
       // Clean out meta location or later tests will be confused... they presume
       // start fresh in zk.
-      MetaTableLocator.deleteMetaLocation(this.watcher);
+      RootTableLocator.deleteRootLocation(this.watcher);
     } catch (KeeperException e) {
       LOG.warn("Unable to delete hbase:meta location", e);
     }
@@ -118,30 +118,30 @@ public class TestMetaTableLocator {
     Mockito.when(client.get((RpcController) Mockito.any(), (GetRequest) Mockito.any()))
       .thenReturn(GetResponse.newBuilder().build());
 
-    assertNull(MetaTableLocator.getMetaRegionLocation(this.watcher));
+    assertNull(RootTableLocator.getRootRegionLocation(this.watcher));
     for (RegionState.State state : RegionState.State.values()) {
       if (state.equals(RegionState.State.OPEN)) {
         continue;
       }
-      MetaTableLocator.setMetaLocation(this.watcher, SN, state);
-      assertNull(MetaTableLocator.getMetaRegionLocation(this.watcher));
-      assertEquals(state, MetaTableLocator.getMetaRegionState(this.watcher).getState());
+      RootTableLocator.setRootLocation(this.watcher, SN, state);
+      assertNull(RootTableLocator.getRootRegionLocation(this.watcher));
+      assertEquals(state, RootTableLocator.getRootRegionState(this.watcher).getState());
     }
-    MetaTableLocator.setMetaLocation(this.watcher, SN, RegionState.State.OPEN);
-    assertEquals(SN, MetaTableLocator.getMetaRegionLocation(this.watcher));
+    RootTableLocator.setRootLocation(this.watcher, SN, RegionState.State.OPEN);
+    assertEquals(SN, RootTableLocator.getRootRegionLocation(this.watcher));
     assertEquals(RegionState.State.OPEN,
-      MetaTableLocator.getMetaRegionState(this.watcher).getState());
+      RootTableLocator.getRootRegionState(this.watcher).getState());
 
-    MetaTableLocator.deleteMetaLocation(this.watcher);
-    assertNull(MetaTableLocator.getMetaRegionState(this.watcher).getServerName());
+    RootTableLocator.deleteRootLocation(this.watcher);
+    assertNull(RootTableLocator.getRootRegionState(this.watcher).getServerName());
     assertEquals(RegionState.State.OFFLINE,
-      MetaTableLocator.getMetaRegionState(this.watcher).getState());
-    assertNull(MetaTableLocator.getMetaRegionLocation(this.watcher));
+      RootTableLocator.getRootRegionState(this.watcher).getState());
+    assertNull(RootTableLocator.getRootRegionLocation(this.watcher));
   }
 
-  @Test(expected = NotAllMetaRegionsOnlineException.class)
+  @Test(expected = NotAllRootRegionsOnlineException.class)
   public void testTimeoutWaitForMeta() throws IOException, InterruptedException {
-    MetaTableLocator.waitMetaRegionLocation(watcher, 100);
+    RootTableLocator.waitRootRegionLocation(watcher, 100);
   }
 
   /**
@@ -149,19 +149,19 @@ public class TestMetaTableLocator {
    */
   @Test
   public void testNoTimeoutWaitForMeta() throws IOException, InterruptedException, KeeperException {
-    ServerName hsa = MetaTableLocator.getMetaRegionLocation(watcher);
+    ServerName hsa = RootTableLocator.getRootRegionLocation(watcher);
     assertNull(hsa);
 
     // Now test waiting on meta location getting set.
     Thread t = new WaitOnMetaThread();
     startWaitAliveThenWaitItLives(t, 1);
     // Set a meta location.
-    MetaTableLocator.setMetaLocation(this.watcher, SN, RegionState.State.OPEN);
+    RootTableLocator.setRootLocation(this.watcher, SN, RegionState.State.OPEN);
     hsa = SN;
     // Join the thread... should exit shortly.
     t.join();
     // Now meta is available.
-    assertTrue(MetaTableLocator.getMetaRegionLocation(watcher).equals(hsa));
+    assertTrue(RootTableLocator.getRootRegionLocation(watcher).equals(hsa));
   }
 
   private void startWaitAliveThenWaitItLives(final Thread t, final int ms) {
@@ -194,11 +194,11 @@ public class TestMetaTableLocator {
     void doWaiting() throws InterruptedException {
       try {
         for (;;) {
-          if (MetaTableLocator.waitMetaRegionLocation(watcher, 10000) != null) {
+          if (RootTableLocator.waitRootRegionLocation(watcher, 10000) != null) {
             break;
           }
         }
-      } catch (NotAllMetaRegionsOnlineException e) {
+      } catch (NotAllRootRegionsOnlineException e) {
         // Ignore
       }
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java
index 78e3e54..ae132c4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
 import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil;
 import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
 import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.junit.AfterClass;
 import org.junit.Rule;
 import org.slf4j.Logger;
@@ -65,7 +65,7 @@ public class MetaWithReplicasTestBase {
     AssignmentManager am = TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager();
     Set<ServerName> sns = new HashSet<ServerName>();
     ServerName hbaseMetaServerName =
-      MetaTableLocator.getMetaRegionLocation(TEST_UTIL.getZooKeeperWatcher());
+      RootTableLocator.getRootRegionLocation(TEST_UTIL.getZooKeeperWatcher());
     LOG.info("HBASE:META DEPLOY: on " + hbaseMetaServerName);
     sns.add(hbaseMetaServerName);
     for (int replicaId = 1; replicaId < 3; replicaId++) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java
index 50111f7..3d11894 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncAdminWithRegionReplicas.java
@@ -137,16 +137,16 @@ public class TestAsyncAdminWithRegionReplicas extends TestAsyncAdminBase {
 
   @Test
   public void testGetTableRegions() throws InterruptedException, ExecutionException, IOException {
-    List<RegionInfo> metaRegions = admin.getRegions(TableName.META_TABLE_NAME).get();
-    assertEquals(3, metaRegions.size());
+    List<RegionInfo> rootRegions = admin.getRegions(TableName.META_TABLE_NAME).get();
+    assertEquals(3, rootRegions.size());
     for (int i = 0; i < 3; i++) {
-      RegionInfo metaRegion = metaRegions.get(i);
-      assertEquals(TableName.META_TABLE_NAME, metaRegion.getTable());
+      RegionInfo metaRegion = rootRegions.get(i);
+      assertEquals(TableName.ROOT_TABLE_NAME, metaRegion.getTable());
       assertEquals(i, metaRegion.getReplicaId());
     }
     createTableWithDefaultConf(tableName, 3);
     List<RegionInfo> regions = admin.getRegions(tableName).get();
-    assertEquals(3, metaRegions.size());
+    assertEquals(3, rootRegions.size());
     for (int i = 0; i < 3; i++) {
       RegionInfo region = regions.get(i);
       assertEquals(tableName, region.getTable());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonRootRegionLocator.java
similarity index 98%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonRootRegionLocator.java
index d8388de..b6357a8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonRootRegionLocator.java
@@ -57,11 +57,11 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({ MediumTests.class, ClientTests.class })
-public class TestAsyncNonMetaRegionLocator {
+public class TestAsyncNonRootRegionLocator {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestAsyncNonMetaRegionLocator.class);
+    HBaseClassTestRule.forClass(TestAsyncNonRootRegionLocator.class);
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
@@ -71,7 +71,7 @@ public class TestAsyncNonMetaRegionLocator {
 
   private static AsyncConnectionImpl CONN;
 
-  private static AsyncNonMetaRegionLocator LOCATOR;
+  private static AsyncNonRootRegionLocator LOCATOR;
 
   private static byte[][] SPLIT_KEYS;
 
@@ -83,7 +83,7 @@ public class TestAsyncNonMetaRegionLocator {
         ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
       registry.getClusterId().get(), null, User.getCurrent());
-    LOCATOR = new AsyncNonMetaRegionLocator(CONN);
+    LOCATOR = new AsyncNonRootRegionLocator(CONN);
     SPLIT_KEYS = new byte[8][];
     for (int i = 111; i < 999; i += 111) {
       SPLIT_KEYS[i / 111 - 1] = Bytes.toBytes(String.format("%03d", i));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonRootRegionLocatorConcurrenyLimit.java
similarity index 95%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonRootRegionLocatorConcurrenyLimit.java
index 88ab3ad..2d61d02 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonRootRegionLocatorConcurrenyLimit.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.hbase.client;
 
 import static java.util.stream.Collectors.toList;
-import static org.apache.hadoop.hbase.client.AsyncNonMetaRegionLocator.MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE;
+import static org.apache.hadoop.hbase.client.AsyncNonRootRegionLocator.MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStartRow;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
 import static org.apache.hadoop.hbase.coprocessor.CoprocessorHost.REGION_COPROCESSOR_CONF_KEY;
@@ -56,11 +56,11 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({ MediumTests.class, ClientTests.class })
-public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
+public class TestAsyncNonRootRegionLocatorConcurrenyLimit {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestAsyncNonMetaRegionLocatorConcurrenyLimit.class);
+    HBaseClassTestRule.forClass(TestAsyncNonRootRegionLocatorConcurrenyLimit.class);
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
@@ -70,7 +70,7 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
 
   private static AsyncConnectionImpl CONN;
 
-  private static AsyncNonMetaRegionLocator LOCATOR;
+  private static AsyncNonRootRegionLocator LOCATOR;
 
   private static byte[][] SPLIT_KEYS;
 
@@ -127,7 +127,7 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
         ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
       registry.getClusterId().get(), null, User.getCurrent());
-    LOCATOR = new AsyncNonMetaRegionLocator(CONN);
+    LOCATOR = new AsyncNonRootRegionLocator(CONN);
     SPLIT_KEYS = IntStream.range(1, 256).mapToObj(i -> Bytes.toBytes(String.format("%02x", i)))
       .toArray(byte[][]::new);
     TEST_UTIL.createTable(TABLE_NAME, FAMILY, SPLIT_KEYS);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRootRegionLocator.java
similarity index 91%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRootRegionLocator.java
index 003bef3..2e7372c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRootRegionLocator.java
@@ -36,17 +36,17 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 @Category({ MediumTests.class, ClientTests.class })
-public class TestAsyncMetaRegionLocator {
+public class TestAsyncRootRegionLocator {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-    HBaseClassTestRule.forClass(TestAsyncMetaRegionLocator.class);
+    HBaseClassTestRule.forClass(TestAsyncRootRegionLocator.class);
 
   private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
 
   private static ConnectionRegistry REGISTRY;
 
-  private static AsyncMetaRegionLocator LOCATOR;
+  private static AsyncRootRegionLocator LOCATOR;
 
   @BeforeClass
   public static void setUp() throws Exception {
@@ -56,7 +56,7 @@ public class TestAsyncMetaRegionLocator {
     REGISTRY = ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
     RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, REGISTRY);
     TEST_UTIL.getAdmin().balancerSwitch(false, true);
-    LOCATOR = new AsyncMetaRegionLocator(REGISTRY);
+    LOCATOR = new AsyncRootRegionLocator(REGISTRY);
   }
 
   @AfterClass
@@ -67,7 +67,7 @@ public class TestAsyncMetaRegionLocator {
 
   @Test
   public void test() throws Exception {
-    testLocator(TEST_UTIL, TableName.META_TABLE_NAME, new Locator() {
+    testLocator(TEST_UTIL, TableName.ROOT_TABLE_NAME, new Locator() {
 
       @Override
       public void updateCachedLocationOnError(HRegionLocation loc, Throwable error)
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java
index 6c6bb98..e4672073 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java
@@ -49,16 +49,16 @@ public class TestAsyncTableLocatePrefetch {
 
   private static AsyncConnection CONN;
 
-  private static AsyncNonMetaRegionLocator LOCATOR;
+  private static AsyncNonRootRegionLocator LOCATOR;
 
   @BeforeClass
   public static void setUp() throws Exception {
-    TEST_UTIL.getConfiguration().setInt(AsyncNonMetaRegionLocator.LOCATE_PREFETCH_LIMIT, 100);
+    TEST_UTIL.getConfiguration().setInt(AsyncNonRootRegionLocator.LOCATE_PREFETCH_LIMIT, 100);
     TEST_UTIL.startMiniCluster(3);
     TEST_UTIL.createMultiRegionTable(TABLE_NAME, FAMILY);
     TEST_UTIL.waitTableAvailable(TABLE_NAME);
     CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
-    LOCATOR = new AsyncNonMetaRegionLocator((AsyncConnectionImpl) CONN);
+    LOCATOR = new AsyncNonRootRegionLocator((AsyncConnectionImpl) CONN);
   }
 
   @AfterClass
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRSCrashPublish.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRSCrashPublish.java
index 461bf1b..53ea788 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRSCrashPublish.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRSCrashPublish.java
@@ -84,7 +84,7 @@ public class TestAsyncTableRSCrashPublish {
   public void test() throws IOException, ExecutionException, InterruptedException {
     Configuration conf = UTIL.getHBaseCluster().getMaster().getConfiguration();
     try (AsyncConnection connection = ConnectionFactory.createAsyncConnection(conf).get()) {
-      AsyncNonMetaRegionLocator locator =
+      AsyncNonRootRegionLocator locator =
         ((AsyncConnectionImpl) connection).getLocator().getNonMetaRegionLocator();
       connection.getTable(TABLE_NAME).get(new Get(Bytes.toBytes(0))).join();
       ServerName serverName =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogAccessorNoCluster.java
similarity index 94%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogAccessorNoCluster.java
index e160fb3..e32db1c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestCatalogAccessorNoCluster.java
@@ -39,15 +39,15 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 /**
- * Test MetaTableAccessor but without spinning up a cluster.
+ * Test CatalogAccessor but without spinning up a cluster.
  * We mock regionserver back and forth (we do spin up a zk cluster).
  */
 @Category({MiscTests.class, MediumTests.class})
-public class TestMetaTableAccessorNoCluster {
+public class TestCatalogAccessorNoCluster {
 
   @ClassRule
   public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestMetaTableAccessorNoCluster.class);
+      HBaseClassTestRule.forClass(TestCatalogAccessorNoCluster.class);
 
   private static final  HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
index abaf092..e78ef97 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
@@ -72,10 +72,10 @@ public class TestMetaRegionLocationCache {
 
   private List<HRegionLocation> getCurrentMetaLocations(ZKWatcher zk) throws Exception {
     List<HRegionLocation> result = new ArrayList<>();
-    for (String znode: zk.getMetaReplicaNodes()) {
+    for (String znode: zk.getRootReplicaNodes()) {
       String path = ZNodePaths.joinZNode(zk.getZNodePaths().baseZNode, znode);
       int replicaId = zk.getZNodePaths().getMetaReplicaIdFromPath(path);
-      RegionState state = MetaTableLocator.getMetaRegionState(zk, replicaId);
+      RegionState state = RootTableLocator.getRootRegionState(zk, replicaId);
       result.add(new HRegionLocation(state.getRegion(), state.getServerName()));
     }
     return result;
@@ -95,7 +95,7 @@ public class TestMetaRegionLocationCache {
         master.getMetaRegionLocationCache().getMetaRegionLocations().get();
     assertFalse(metaHRLs.isEmpty());
     ZKWatcher zk = master.getZooKeeper();
-    List<String> metaZnodes = zk.getMetaReplicaNodes();
+    List<String> metaZnodes = zk.getRootReplicaNodes();
     assertEquals(metaZnodes.size(), metaHRLs.size());
     List<HRegionLocation> actualHRLs = getCurrentMetaLocations(zk);
     Collections.sort(metaHRLs);
@@ -158,7 +158,7 @@ public class TestMetaRegionLocationCache {
         // assignment.
         for (int i = 0; i < 3; i++) {
           // Updates the meta znodes.
-          MetaTableLocator.setMetaLocation(zkWatcher, sn, i, RegionState.State.OPEN);
+          RootTableLocator.setRootLocation(zkWatcher, sn, i, RegionState.State.OPEN);
         }
         // Wait until the meta cache is populated.
         int iters = 0;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java
index 91754eb..fcadc9c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicasBasic.java
@@ -85,7 +85,7 @@ public class TestMetaWithReplicasBasic extends MetaWithReplicasTestBase {
   @Test
   public void testReplicaCleanup() throws Exception {
     ZKWatcher zkw = TEST_UTIL.getZooKeeperWatcher();
-    List<String> metaReplicaZnodes = zkw.getMetaReplicaNodes();
+    List<String> metaReplicaZnodes = zkw.getRootReplicaNodes();
     assertEquals(3, metaReplicaZnodes.size());
 
     final HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
@@ -99,7 +99,7 @@ public class TestMetaWithReplicasBasic extends MetaWithReplicasTestBase {
     //wait until new master finished meta replica assignment logic
     TEST_UTIL.waitFor(30000, () -> newMaster.getMasterQuotaManager() != null);
     zkw = TEST_UTIL.getZooKeeperWatcher();
-    metaReplicaZnodes = zkw.getMetaReplicaNodes();
+    metaReplicaZnodes = zkw.getRootReplicaNodes();
     assertEquals(1, metaReplicaZnodes.size());
 
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitMetaBasicOperations.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitMetaBasicOperations.java
new file mode 100644
index 0000000..dfe7dde
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSplitMetaBasicOperations.java
@@ -0,0 +1,412 @@
+/**
+ * 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.client;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NavigableMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CatalogFamilyFormat;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.CatalogAccessor;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter.Predicate;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HStore;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@Category(MediumTests.class)
+public class TestSplitMetaBasicOperations {
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestSplitMetaBasicOperations.class);
+
+  static Configuration conf = HBaseConfiguration.create();
+  private static final Logger LOG = LoggerFactory.getLogger(TestSplitMetaBasicOperations.class);
+
+  /**
+   * Tests basic split and merge operations on meta and user tables
+   * as well as tests that the tables are still accessible using basic operations.
+   * Then performs some tests which verify that the proper comparator is used
+   * when meta row keys are involved as the meta row key is broken up into
+   * 3 parts and lexicographical ordering is performed on each one individually.
+   * Given that the ',' is the delimiter for these 3 parts we have chosen split keys
+   * which have table splits keys that are lexicographically smaller than ','
+   * in certain scenarios to verify that the proper comparator is used.
+   */
+  @Test(timeout = 120000)
+  public void testBasicSplitMerge() throws Exception {
+    final TableName tableName = TableName.valueOf("testSplitAtSplitPoint");
+    conf = HBaseConfiguration.create();
+    conf.set("hbase.client.retries.by.server", "10");
+    conf.set("hbase.client.retries.number", "10");
+    HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
+    try {
+      TEST_UTIL.startMiniCluster(1, 3);
+      Connection conn = TEST_UTIL.getConnection();
+      MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
+      cluster.waitForActiveAndReadyMaster();
+
+      TableDescriptor desc =
+        TableDescriptorBuilder.newBuilder(tableName)
+          .setColumnFamily(ColumnFamilyDescriptorBuilder
+            .newBuilder(Bytes.toBytes("cf"))
+            .setBlocksize(30)
+            .build()).build();
+
+      byte[][] splits = {{0x02},{0x03},{0x04},{0x05}};
+      final Admin hbaseAdmin = conn.getAdmin();
+      hbaseAdmin.createTable(desc, splits);
+
+      List<Result> tableRegionList = CatalogAccessor.fullScanRegions(TEST_UTIL.getConnection());
+      LOG.info("Splitting meta");
+      List<HRegionLocation> tableRegionLocations =
+        conn.getTable(tableName).getRegionLocator().getAllRegionLocations();
+
+//      //Test malformed split key
+//      try {
+//        splitTable(TEST_UTIL, TableName.META_TABLE_NAME, Bytes.toBytesBinary("5555"));
+//        fail("Expected malformed split key related exception");
+//      } catch (Exception ex) {
+//      }
+//
+//      //Test malformed split key, empty table name
+//      try {
+//;        splitTable(TEST_UTIL, TableName.META_TABLE_NAME, Bytes.toBytesBinary(",,123"));
+//        fail("Expected malformed split key related exception");
+//      } catch (Exception ex) {
+//      }
+//
+//      //Test malformed split key, empty id component
+//      try {
+//        splitTable(TEST_UTIL, TableName.META_TABLE_NAME, Bytes.toBytesBinary("123,,"));
+//        fail("Expected malformed split key related exception");
+//      } catch (Exception ex) {
+//      }
+
+      splitTable(TEST_UTIL, TableName.META_TABLE_NAME,
+        Iterators.get(tableRegionLocations.iterator(), 0).getRegion().getRegionName());
+
+      TEST_UTIL.flush(TableName.ROOT_TABLE_NAME);
+      for(Result r : conn.getTable(TableName.ROOT_TABLE_NAME).getScanner(new Scan())) {
+        LOG.debug("hbase:root-->"+r);
+      }
+
+      // Root should have two entries as meta got split in 2
+      List<RegionInfo> regions =
+        CatalogAccessor.getTableRegions(
+          conn,
+          TableName.META_TABLE_NAME,
+          true);
+      assertEquals(regions.toString(), 2, regions.size());
+
+      TEST_UTIL.flush(TableName.ROOT_TABLE_NAME);
+      for(Result r : conn.getTable(TableName.ROOT_TABLE_NAME).getScanner(new Scan())) {
+        LOG.debug("hbase:root-->"+r);
+      }
+      final Table table = conn.getTable(TableName.META_TABLE_NAME);
+        table.getRegionLocator().getRegionLocation(
+          Iterators.get(tableRegionLocations.iterator(), 1).getRegion().getRegionName(),
+          true).getRegion();
+
+      splitTable(TEST_UTIL, TableName.META_TABLE_NAME,
+        Iterators.get(tableRegionLocations.iterator(), 1).getRegion().getRegionName());
+
+      splitTable(TEST_UTIL, TableName.META_TABLE_NAME,
+        Iterators.get(tableRegionLocations.iterator(), 2).getRegion().getRegionName());
+
+      assertEquals(4,
+        CatalogAccessor.getTableRegions(
+          TEST_UTIL.getConnection(),
+          TableName.META_TABLE_NAME,
+          true).size());
+
+      checkBasicOps(conn, tableName, tableRegionList);
+
+      ResultScanner resultScanner =
+        TEST_UTIL.getConnection().getTable(TableName.ROOT_TABLE_NAME).getScanner(new Scan());
+
+      splitTable(TEST_UTIL, tableName, new byte[]{0x02, 0x00});
+
+      //Always compact the tables before running a catalog scan
+      //to make sure everything can be cleaned up
+      TEST_UTIL.compact(tableName, true);
+      TEST_UTIL.compact(TableName.META_TABLE_NAME, true);
+      archiveStores(TEST_UTIL, TableName.META_TABLE_NAME);
+      hbaseAdmin.runCatalogJanitor();
+      //TODO need to see why we have to wait for catalog janitor to run
+      Thread.sleep(1000);
+
+      tableRegionList = CatalogAccessor.fullScanRegions(TEST_UTIL.getConnection());
+      List<RegionInfo> regionsList = new ArrayList<>();
+      for (Result res : tableRegionList) {
+        regionsList.add(CatalogFamilyFormat.getRegionInfo(res));
+      }
+      assertEquals("Got list :"+regionsList,splits.length + 2, tableRegionList.size());
+      checkBasicOps(conn, tableName, tableRegionList);
+
+      mergeRegions(TEST_UTIL, tableName, new byte[]{0x02}, new byte[]{0x02, 0x00});
+      TEST_UTIL.compact(tableName, true);
+      TEST_UTIL.compact(TableName.META_TABLE_NAME, true);
+      archiveStores(TEST_UTIL, TableName.META_TABLE_NAME);
+      hbaseAdmin.runCatalogJanitor();
+
+      tableRegionList = CatalogAccessor.fullScanRegions(TEST_UTIL.getConnection());
+      assertEquals(splits.length + 1, tableRegionList.size());
+
+      mergeRegions(TEST_UTIL,
+        TableName.META_TABLE_NAME,
+        Iterators.get(tableRegionLocations.iterator(), 0).getRegion().getRegionName(),
+        Iterators.get(tableRegionLocations.iterator(), 1).getRegion().getRegionName());
+
+      checkBasicOps(conn, tableName, tableRegionList);
+
+    } finally {
+      TEST_UTIL.deleteTable(tableName);
+      TEST_UTIL.shutdownMiniCluster();
+    }
+  }
+
+//  @Test(timeout = 120000)
+//  public void testSplitNoSplitPoint() throws Exception {
+//    final TableName tableName = TableName.valueOf("testSplitNoSplitPoint");
+//    conf = HBaseConfiguration.create();
+//    HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
+//    Admin hbaseAdmin = null;
+//    try {
+//      TEST_UTIL.startMiniCluster(1, 3);
+//      MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
+//      cluster.waitForActiveAndReadyMaster();
+//      hbaseAdmin = TEST_UTIL.getAdmin();
+//
+//      HTableDescriptor desc = new HTableDescriptor(tableName);
+//      desc.addFamily(new HColumnDescriptor("cf").setBlocksize(30));
+//      SplitAlgorithm algo = new RegionSplitter.HexStringSplit();
+//      // Have more splits so entries in meta span more than one block
+//      byte[][] splits = algo.split(20);
+//      hbaseAdmin.createTable(desc, splits);
+//      hbaseAdmin.flush(TableName.META_TABLE_NAME);
+//
+//      List<Result> list =
+//        CatalogAccessor.fullScanRegions(TEST_UTIL.getConnection());
+//
+//      LOG.info("Splitting meta");
+//      int metaServerIndex =
+//        TEST_UTIL.getHBaseCluster().getServerWith(
+//          HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
+//      HRegionServer server = TEST_UTIL.getHBaseCluster().getRegionServer(metaServerIndex);
+//      int regionCount = hbaseAdmin.getRegions(server.getServerName()).size();
+//      hbaseAdmin.split(HRegionInfo.FIRST_META_REGIONINFO.getTable());
+//
+//      int metaSize =0;
+//      for (int i = 0; i < 300; i++) {
+//        LOG.debug("Waiting on region to split");
+//        metaSize =
+//          CatalogAccessor.getTableRegions(
+//            TEST_UTIL.getConnection(),
+//            TableName.META_TABLE_NAME,
+//            true).size();
+//        if (metaSize > 1) {
+//          break;
+//        }
+//        Thread.sleep(1000);
+//      }
+//
+//      assertEquals(2, metaSize);
+//      LOG.info("Splitting done");
+//      checkBasicOps(TEST_UTIL.getConnection(), tableName, list);
+//    } finally {
+//      if (hbaseAdmin != null) {
+//        hbaseAdmin.disableTable(tableName);
+//        hbaseAdmin.deleteTable(tableName);
+//      }
+//      TEST_UTIL.shutdownMiniCluster();
+//    }
+//  }
+
+
+  private void checkBasicOps(Connection conn, TableName tableName,
+    List<Result> expectedList) throws Exception {
+    LOG.info("Splitting done");
+
+    // Scan meta after split
+    Table table = conn.getTable(TableName.META_TABLE_NAME);
+    Scan s = new Scan().withStartRow(HConstants.EMPTY_START_ROW)
+      .addFamily(HConstants.CATALOG_FAMILY);
+    ResultScanner scanner = table.getScanner(s);
+    Result r = scanner.next();
+    int i = 0;
+    while (r != null) {
+      assertEquals(CatalogFamilyFormat.getRegionInfo(r),
+        CatalogFamilyFormat.getRegionInfo(expectedList.get(i)));
+      r = scanner.next();
+      i++;
+    }
+
+    table = conn.getTable(tableName);
+    // try adding/retrieving a row to a user region referenced by the first meta
+    byte[] rowKey = Bytes.toBytes("f0000000");
+    byte[] family = Bytes.toBytes("cf");
+    Put put = new Put(rowKey);
+    put.addColumn(family, Bytes.toBytes("A"), Bytes.toBytes("1"));
+    table.put(put);
+    Get get = new Get(rowKey);
+    Result result = table.get(get);
+    assertTrue("Column A value should be 1",
+      Bytes.toString(result.getValue(family, Bytes.toBytes("A"))).equals("1"));
+
+    // try adding/retrieving a row to a user region referenced by the second meta
+    rowKey = Bytes.toBytes("10000000");
+    family = Bytes.toBytes("cf");
+    put = new Put(rowKey);
+    put.addColumn(family, Bytes.toBytes("A"), Bytes.toBytes("2"));
+    table.put(put);
+    get = new Get(rowKey);
+    result = conn.getTable(tableName).get(get);
+    assertTrue("Column A value should be 2",
+      Bytes.toString(result.getValue(family, Bytes.toBytes("A"))).equals("2"));
+
+    rowKey = Bytes.toBytes("f0000000");
+    Delete d = new Delete(rowKey);
+    table.delete(d);
+    assertTrue(table.get(new Get(rowKey)).isEmpty());
+
+    rowKey = Bytes.toBytes("10000000");
+    d = new Delete(rowKey);
+    table.delete(d);
+    assertTrue(table.get(new Get(rowKey)).isEmpty());
+  }
+
+  public void splitTable(HBaseTestingUtility util, TableName tableName, final byte[] splitKey)
+    throws Exception {
+    final Table table = util.getConnection().getTable(tableName);
+    final RegionInfo targetRegion =
+      table.getRegionLocator().getRegionLocation(splitKey, true).getRegion();
+
+    util.getAdmin().flush(tableName);
+    util.compact(tableName, true);
+    archiveStores(util, tableName);
+    util.getAdmin().split(targetRegion.getTable(), splitKey);
+
+    util.waitFor(60000, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        HRegionLocation loc1 =
+          table.getRegionLocator().getRegionLocation(targetRegion.getStartKey(), true);
+        HRegionLocation loc2 =
+          table.getRegionLocator().getRegionLocation(splitKey, true);
+        return !loc1.getRegion().getRegionNameAsString().equals(
+          loc2.getRegion().getRegionNameAsString());
+      }
+    });
+
+    //make sure regions are online
+    byte[][] keys = {targetRegion.getStartKey(), splitKey};
+    for(byte[] el : keys) {
+      byte[] key = el;
+      if (key.length == 0) {
+        key = new byte[]{0x00};
+      }
+      table.get(new Get(key));
+    }
+  }
+
+
+  public void mergeRegions(HBaseTestingUtility util, TableName tableName,
+    final byte[] startKey1,
+    final byte[] startKey2)
+    throws Exception {
+    Connection conn = util.getConnection();
+    final Table table = conn.getTable(tableName);
+    final RegionInfo targetRegion1 =
+      table.getRegionLocator().getRegionLocation(startKey1, true).getRegion();
+    final RegionInfo targetRegion2 =
+      table.getRegionLocator().getRegionLocation(startKey2, true).getRegion();
+
+    Table metaTable = conn.getTable(TableName.ROOT_TABLE_NAME);
+    if (!tableName.equals(TableName.META_TABLE_NAME)) {
+      metaTable = conn.getTable(TableName.META_TABLE_NAME);
+    }
+
+    Scan s = new Scan().withStartRow(HConstants.EMPTY_START_ROW)
+      .addFamily(HConstants.CATALOG_FAMILY);
+    ResultScanner scanner = metaTable.getScanner(s);
+    Result r = scanner.next();
+    List<RegionInfo> splitList = new ArrayList<RegionInfo>();
+    while (r != null) {
+      splitList.add(CatalogFamilyFormat.getRegionInfo(r));
+      r = scanner.next();
+    }
+    scanner.close();
+
+    util.getAdmin().flush(tableName);
+    util.compact(tableName, true);
+    archiveStores(util, tableName);
+
+    util.getAdmin().mergeRegionsAsync(new byte[][]{targetRegion1.getRegionName(),
+      targetRegion2.getRegionName()},false).get();
+    util.waitFor(60000, new Predicate<Exception>() {
+      @Override
+      public boolean evaluate() throws Exception {
+        HRegionLocation loc1 = table.getRegionLocator().getRegionLocation(startKey1, true);
+        HRegionLocation loc2 = table.getRegionLocator().getRegionLocation(startKey2, true);
+        LOG.debug("loc-->"+Bytes.toStringBinary(startKey1)+"="+loc1+" --- "+Bytes.toStringBinary(startKey2)+"="+loc2);
+        return loc1.getRegion().getRegionNameAsString().equals(
+          loc2.getRegion().getRegionNameAsString());
+      }
+    });
+
+
+    s = new Scan().withStartRow(HConstants.EMPTY_START_ROW).addFamily(HConstants.CATALOG_FAMILY);
+    scanner = metaTable.getScanner(s);
+    r = scanner.next();
+    splitList = new ArrayList<RegionInfo>();
+    while (r != null) {
+      splitList.add(CatalogFamilyFormat.getRegionInfo(r));
+      r = scanner.next();
+    }
+
+    //make sure region is online
+    byte[] key = startKey1;
+    if (key.length == 0) {
+      key = new byte[]{0x00};
+    }
+    table.get(new Get(key));
+  }
+
+  private void archiveStores(HBaseTestingUtility util, TableName tableName) throws IOException {
+    for (HRegion region : util.getMiniHBaseCluster().getRegions(tableName)) {
+      for (HStore store : region.getStores()) {
+        store.closeAndArchiveCompactedFiles();
+      }
+    }
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java
index 137cb28..886afdb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestZKConnectionRegistry.java
@@ -93,7 +93,7 @@ public class TestZKConnectionRegistry {
     IntStream.range(0, 3).forEach(i -> {
       HRegionLocation loc = locs.getRegionLocation(i);
       assertNotNull("Replica " + i + " doesn't have location", loc);
-      assertEquals(TableName.META_TABLE_NAME, loc.getRegion().getTable());
+      assertEquals(TableName.ROOT_TABLE_NAME, loc.getRegion().getTable());
       assertEquals(i, loc.getRegion().getReplicaId());
     });
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
index 4016e7b..858e0dd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
@@ -141,7 +141,7 @@ public class TestInfoServersACL {
 
     CLUSTER = new LocalHBaseCluster(conf, 1);
     CLUSTER.startup();
-    CLUSTER.getActiveMaster().waitForMetaOnline();
+    CLUSTER.getActiveMaster().getAssignmentManager().waitForMetaOnline();
   }
 
   /**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java
index f78d490..5ddc018 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java
@@ -108,7 +108,7 @@ public class TestDeadServer {
     HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
     final ProcedureExecutor<MasterProcedureEnv> pExecutor = master.getMasterProcedureExecutor();
     ServerCrashProcedure proc = new ServerCrashProcedure(
-      pExecutor.getEnvironment(), hostname123, false, false);
+      pExecutor.getEnvironment(), hostname123, false, false, false);
 
     ProcedureTestingUtility.submitAndWait(pExecutor, proc);
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
index a27936d..5ba2f3e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.testclassification.FlakeyTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
@@ -187,7 +187,7 @@ public class TestMasterFailover {
       LOG.info("Master has aborted");
 
       // meta should remain where it was
-      RegionState metaState = MetaTableLocator.getMetaRegionState(hrs.getZooKeeper());
+      RegionState metaState = RootTableLocator.getRootRegionState(hrs.getZooKeeper());
       assertEquals("hbase:meta should be online on RS",
           metaState.getServerName(), metaServerName);
       assertEquals("hbase:meta should be online on RS", State.OPEN, metaState.getState());
@@ -200,7 +200,7 @@ public class TestMasterFailover {
       LOG.info("Master is ready");
 
       // ensure meta is still deployed on RS
-      metaState = MetaTableLocator.getMetaRegionState(activeMaster.getZooKeeper());
+      metaState = RootTableLocator.getRootRegionState(activeMaster.getZooKeeper());
       assertEquals("hbase:meta should be online on RS",
           metaState.getServerName(), metaServerName);
       assertEquals("hbase:meta should be online on RS", State.OPEN, metaState.getState());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java
index 742734e..ae58308 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.master.assignment.RegionStates;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
 import org.apache.zookeeper.KeeperException;
@@ -99,7 +99,7 @@ public class TestMetaShutdownHandler {
       metaServerName =
         regionStates.getRegionServerOfRegion(RegionInfoBuilder.FIRST_META_REGIONINFO);
     }
-    RegionState metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper());
+    RegionState metaState = RootTableLocator.getRootRegionState(master.getZooKeeper());
     assertEquals("Wrong state for meta!", RegionState.State.OPEN, metaState.getState());
     assertNotEquals("Meta is on master!", metaServerName, master.getServerName());
 
@@ -126,7 +126,7 @@ public class TestMetaShutdownHandler {
     assertTrue("Meta should be assigned",
       regionStates.isRegionOnline(RegionInfoBuilder.FIRST_META_REGIONINFO));
     // Now, make sure meta is registered in zk
-    metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper());
+    metaState = RootTableLocator.getRootRegionState(master.getZooKeeper());
     assertEquals("Meta should not be in transition", RegionState.State.OPEN, metaState.getState());
     assertEquals("Meta should be assigned", metaState.getServerName(),
       regionStates.getRegionServerOfRegion(RegionInfoBuilder.FIRST_META_REGIONINFO));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java
index 50361ed..af25ebd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java
@@ -597,6 +597,11 @@ public class TestRegionsRecoveryChore {
       }
 
       @Override
+      public boolean isRootRegion() {
+        return false;
+      }
+
+      @Override
       public boolean isMetaRegion() {
         return false;
       }
@@ -611,6 +616,11 @@ public class TestRegionsRecoveryChore {
         return false;
       }
 
+      @Override
+      public boolean containsRow(byte[] row, int offset, short length) {
+        return false;
+      }
+
     };
     return regionInfo;
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitWALManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitWALManager.java
index 7edb011..2df2c09 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitWALManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitWALManager.java
@@ -158,7 +158,8 @@ public class TestSplitWALManager {
     Assert.assertFalse(TEST_UTIL.getTestFileSystem().exists(wals[0].getPath()));
 
     // Test splitting wal
-    wals = TEST_UTIL.getTestFileSystem().listStatus(metaWALDir, MasterWalManager.NON_META_FILTER);
+    wals =
+      TEST_UTIL.getTestFileSystem().listStatus(metaWALDir, MasterWalManager.NON_CATALOG_FILTER);
     Assert.assertEquals(1, wals.length);
     testProcedures =
         splitWALManager.createSplitWALProcedures(Lists.newArrayList(wals[0]), metaServer);
@@ -282,6 +283,11 @@ public class TestSplitWALManager {
     }
 
     @Override
+    public boolean hasRootTableRegion() {
+      return false;
+    }
+
+    @Override
     public boolean hasMetaTableRegion() {
       return false;
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
index 0f4e97f..0de8c53 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
@@ -25,7 +25,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.CatalogAccessor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
@@ -224,7 +224,7 @@ public class TestAssignmentManager extends TestAssignmentManagerBase {
    * back any response, which cause master startup hangs forever
    */
   @Test
-  public void testAssignMetaAndCrashBeforeResponse() throws Exception {
+  public void testAssignRootAndCrashBeforeResponse() throws Exception {
     tearDown();
     // See setUp(), start HBase until set up meta
     util = new HBaseTestingUtility();
@@ -237,8 +237,8 @@ public class TestAssignmentManager extends TestAssignmentManagerBase {
 
     // Assign meta
     rsDispatcher.setMockRsExecutor(new HangThenRSRestartExecutor());
-    am.assign(RegionInfoBuilder.FIRST_META_REGIONINFO);
-    assertEquals(true, am.isMetaAssigned());
+    am.assign(RegionInfoBuilder.ROOT_REGIONINFO);
+    assertEquals(true, am.isRootAssigned());
 
     // set it back as default, see setUpMeta()
     am.wakeMetaLoadedEvent();
@@ -299,10 +299,10 @@ public class TestAssignmentManager extends TestAssignmentManagerBase {
       RegionInfo hri = createRegionInfo(tableName, 1);
       assertNull("RegionInfo was just instantiated by the test, but "
         + "shouldn't be in AM regionStates yet.", am.getRegionStates().getRegionState(hri));
-      MetaTableAccessor.addRegionToMeta(this.util.getConnection(), hri);
+      CatalogAccessor.addRegionToMeta(this.util.getConnection(), hri);
       assertNull("RegionInfo was manually added in META, but "
         + "shouldn't be in AM regionStates yet.", am.getRegionStates().getRegionState(hri));
-      hri = am.loadRegionFromMeta(hri.getEncodedName());
+      hri = am.loadRegionFromCatalog(hri.getEncodedName());
       assertEquals(hri.getEncodedName(),
         am.getRegionStates().getRegionState(hri).getRegion().getEncodedName());
     }finally {
@@ -321,7 +321,7 @@ public class TestAssignmentManager extends TestAssignmentManagerBase {
       assertNull("RegionInfo was just instantiated by the test, but "
         + "shouldn't be in AM regionStates yet.", am.getRegionStates().getRegionState(hri));
       assertNull("RegionInfo was never added in META, should had returned null.",
-        am.loadRegionFromMeta(hri.getEncodedName()));
+        am.loadRegionFromCatalog(hri.getEncodedName()));
     }finally {
       this.util.killMiniHBaseCluster();
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java
index 2451f41..61989aa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java
@@ -172,6 +172,8 @@ public abstract class TestAssignmentManagerBase {
 
   protected void setUpMeta() throws Exception {
     rsDispatcher.setMockRsExecutor(new GoodRsExecutor());
+    am.assign(RegionInfoBuilder.ROOT_REGIONINFO);
+    am.wakeRootLoadedEvent();
     am.assign(RegionInfoBuilder.FIRST_META_REGIONINFO);
     am.wakeMetaLoadedEvent();
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java
index 75f73e5..56e1bf6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java
@@ -89,6 +89,11 @@ public class TestCloseRegionWhileRSCrash {
     }
 
     @Override
+    public boolean hasRootTableRegion() {
+      return false;
+    }
+
+    @Override
     public boolean hasMetaTableRegion() {
       return false;
     }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java
index 0545126..791f106 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.master.assignment;
 
-import static org.apache.hadoop.hbase.TestMetaTableAccessor.assertEmptyMetaLocation;
+import static org.apache.hadoop.hbase.TestCatalogAccessor.assertEmptyMetaLocation;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
@@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.CatalogAccessor;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNameTestRule;
@@ -96,7 +96,7 @@ public class TestRegionStateStore {
     final RegionStateStore regionStateStore = UTIL.getHBaseCluster().getMaster().
       getAssignmentManager().getRegionStateStore();
     final AtomicBoolean visitorCalled = new AtomicBoolean(false);
-    regionStateStore.visitMetaForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
+    regionStateStore.visitCatalogForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
       @Override
       public void visitRegionState(Result result, RegionInfo regionInfo, RegionState.State state,
         ServerName regionLocation, ServerName lastHost, long openSeqNum) {
@@ -127,7 +127,7 @@ public class TestRegionStateStore {
     }
 
     final AtomicBoolean visitorCalled = new AtomicBoolean(false);
-    regionStateStore.visitMetaForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
+    regionStateStore.visitCatalogForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
       @Override
       public void visitRegionState(Result result, RegionInfo regionInfo,
                                    RegionState.State state, ServerName regionLocation,
@@ -146,7 +146,7 @@ public class TestRegionStateStore {
     final RegionStateStore regionStateStore = UTIL.getHBaseCluster().getMaster().
       getAssignmentManager().getRegionStateStore();
     final AtomicBoolean visitorCalled = new AtomicBoolean(false);
-    regionStateStore.visitMetaForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
+    regionStateStore.visitCatalogForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
       @Override
       public void visitRegionState(Result result, RegionInfo regionInfo, RegionState.State state,
         ServerName regionLocation, ServerName lastHost, long openSeqNum) {
@@ -173,12 +173,13 @@ public class TestRegionStateStore {
       .setEndKey(HConstants.EMPTY_END_ROW).setSplit(false).setRegionId(regionId + 1).setReplicaId(0)
       .build();
     List<RegionInfo> regionInfos = Lists.newArrayList(parent);
-    MetaTableAccessor.addRegionsToMeta(UTIL.getConnection(), regionInfos, 3);
+    CatalogAccessor.addRegionsToMeta(UTIL.getConnection(), regionInfos, 3);
     final RegionStateStore regionStateStore =
       UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStateStore();
     regionStateStore.splitRegion(parent, splitA, splitB, serverName0,
       TableDescriptorBuilder.newBuilder(tableName).setRegionReplication(3).build());
-    try (Table meta = MetaTableAccessor.getMetaHTable(UTIL.getConnection())) {
+    try (Table meta =
+      CatalogAccessor.getCatalogHTable(UTIL.getConnection(), TableName.META_TABLE_NAME)) {
       assertEmptyMetaLocation(meta, splitA.getRegionName(), 1);
       assertEmptyMetaLocation(meta, splitA.getRegionName(), 2);
       assertEmptyMetaLocation(meta, splitB.getRegionName(), 1);
@@ -202,12 +203,13 @@ public class TestRegionStateStore {
       .setEndKey(HConstants.EMPTY_END_ROW).setSplit(false).setRegionId(regionId + 1).setReplicaId(0)
       .build();
     List<RegionInfo> regionInfos = Lists.newArrayList(parent);
-    MetaTableAccessor.addRegionsToMeta(UTIL.getConnection(), regionInfos, 3);
+    CatalogAccessor.addRegionsToMeta(UTIL.getConnection(), regionInfos, 3);
     final RegionStateStore regionStateStore =
       UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStateStore();
     regionStateStore.splitRegion(parent, splitA, splitB, serverName0,
       TableDescriptorBuilder.newBuilder(tableName).setRegionReplication(3).build());
-    try (Table meta = MetaTableAccessor.getMetaHTable(UTIL.getConnection())) {
+    try (Table meta =
+      CatalogAccessor.getCatalogHTable(UTIL.getConnection(), TableName.META_TABLE_NAME)) {
       Get get1 = new Get(splitA.getRegionName());
       Result resultA = meta.get(get1);
       Cell serverCellA = resultA.getColumnLatestCell(HConstants.CATALOG_FAMILY,
@@ -249,9 +251,10 @@ public class TestRegionStateStore {
     final RegionStateStore regionStateStore =
       UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStateStore();
 
-    try (Table meta = MetaTableAccessor.getMetaHTable(UTIL.getConnection())) {
+    try (Table meta =
+      CatalogAccessor.getCatalogHTable(UTIL.getConnection(), TableName.META_TABLE_NAME)) {
       List<RegionInfo> regionInfos = Lists.newArrayList(parentA, parentB);
-      MetaTableAccessor.addRegionsToMeta(UTIL.getConnection(), regionInfos, 3);
+      CatalogAccessor.addRegionsToMeta(UTIL.getConnection(), regionInfos, 3);
       regionStateStore.mergeRegions(merged, new RegionInfo[] { parentA, parentB }, serverName0,
         TableDescriptorBuilder.newBuilder(tableName).setRegionReplication(3).build());
       assertEmptyMetaLocation(meta, merged.getRegionName(), 1);
@@ -276,9 +279,10 @@ public class TestRegionStateStore {
       .setRegionId(regionId).setReplicaId(0).build();
 
     ServerName sn = ServerName.valueOf("bar", 0, 0);
-    try (Table meta = MetaTableAccessor.getMetaHTable(UTIL.getConnection())) {
+    try (Table meta =
+      CatalogAccessor.getCatalogHTable(UTIL.getConnection(), TableName.META_TABLE_NAME)) {
       List<RegionInfo> regionInfos = Lists.newArrayList(regionInfoA, regionInfoB);
-      MetaTableAccessor.addRegionsToMeta(UTIL.getConnection(), regionInfos, 1);
+      CatalogAccessor.addRegionsToMeta(UTIL.getConnection(), regionInfos, 1);
 
       // write the serverName column with a big current time, but set the masters time as even
       // bigger. When region merge deletes the rows for regionA and regionB, the serverName columns
@@ -287,7 +291,7 @@ public class TestRegionStateStore {
       long masterSystemTime = EnvironmentEdgeManager.currentTime() + 123456789;
 
       // write the serverName columns
-      MetaTableAccessor.updateRegionLocation(UTIL.getConnection(), regionInfoA, sn, 1,
+      CatalogAccessor.updateRegionLocation(UTIL.getConnection(), regionInfoA, sn, 1,
         serverNameTime);
 
       // assert that we have the serverName column with expected ts
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSCP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSCP.java
index 72b1ef8..e4bb98e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSCP.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSCP.java
@@ -71,7 +71,7 @@ public class TestSCP extends TestSCPBase {
       HMaster master = util.getHBaseCluster().getMaster();
       final ProcedureExecutor<MasterProcedureEnv> pExecutor = master.getMasterProcedureExecutor();
       ServerCrashProcedure procB =
-        new ServerCrashProcedure(pExecutor.getEnvironment(), rsToKill, false, false);
+        new ServerCrashProcedure(pExecutor.getEnvironment(), rsToKill, false, false, false);
       AssignmentTestingUtil.killRs(util, rsToKill);
       long procId = getSCPProcId(pExecutor);
       Procedure<?> procA = pExecutor.getProcedure(procId);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerRemoteProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerRemoteProcedure.java
index 14342d4..98f4739 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerRemoteProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerRemoteProcedure.java
@@ -218,6 +218,10 @@ public class TestServerRemoteProcedure {
       return targetServer;
     }
 
+    @Override public boolean hasRootTableRegion() {
+      return false;
+    }
+
     @Override
     public boolean hasMetaTableRegion() {
       return false;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
index 12bfc66..9619b1e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
@@ -951,7 +951,7 @@ public class TestDefaultMemStore {
     edge.setCurrentTimeMillis(1234);
     WALFactory wFactory = new WALFactory(conf, "1234");
     TableDescriptors tds = new FSTableDescriptors(conf);
-    FSTableDescriptors.tryUpdateMetaTableDescriptor(conf);
+    FSTableDescriptors.tryUpdateCatalogTableDescriptor(conf);
     HRegion meta = HRegion.createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, testDir,
         conf, tds.get(TableName.META_TABLE_NAME),
         wFactory.getWAL(RegionInfoBuilder.FIRST_META_REGIONINFO));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
index 79f0291..b9c74f4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.CatalogAccessor;
 import org.apache.hadoop.hbase.TableDescriptors;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
@@ -91,7 +91,7 @@ public class TestGetClosestAtOrBefore {
     Path rootdir = UTIL.getDataTestDirOnTestFS();
     // Up flush size else we bind up when we use default catalog flush of 16k.
     TableDescriptors tds = new FSTableDescriptors(UTIL.getConfiguration());
-    FSTableDescriptors.tryUpdateMetaTableDescriptor(UTIL.getConfiguration());
+    FSTableDescriptors.tryUpdateCatalogTableDescriptor(UTIL.getConfiguration());
     TableDescriptor td = tds.get(TableName.META_TABLE_NAME);
     td = TableDescriptorBuilder.newBuilder(td).setMemStoreFlushSize(64 * 1024 * 1024).build();
     HRegion mr = HBaseTestingUtility.createRegionAndWAL(RegionInfoBuilder.FIRST_META_REGIONINFO,
@@ -108,7 +108,7 @@ public class TestGetClosestAtOrBefore {
             .setEndKey(i == last ? HConstants.EMPTY_BYTE_ARRAY : Bytes.toBytes((byte) i + interval))
             .build();
           Put put =
-            MetaTableAccessor.makePutFromRegionInfo(hri, EnvironmentEdgeManager.currentTime());
+            CatalogAccessor.makePutFromRegionInfo(hri, EnvironmentEdgeManager.currentTime());
           put.setDurability(Durability.SKIP_WAL);
           LOG.info("Put {}", put);
           mr.put(put);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java
index 7d6c55b..f1f103c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java
@@ -71,7 +71,7 @@ public class TestReadAndWriteRegionInfoFile {
     RegionInfo ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
     // Create a region. That'll write the .regioninfo file.
     FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(FS, ROOT_DIR);
-    FSTableDescriptors.tryUpdateAndGetMetaTableDescriptor(CONF, FS, ROOT_DIR, null);
+    FSTableDescriptors.tryUpdateAndGetCatalogTableDescriptor(CONF, FS, ROOT_DIR, null);
     HRegion r = HBaseTestingUtility.createRegionAndWAL(ri, ROOT_DIR, CONF,
       fsTableDescriptors.get(TableName.META_TABLE_NAME));
     // Get modtime on the file.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java
index 1bd8da6..34a3294 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java
@@ -180,7 +180,7 @@ public class TestRegionInfo {
     Path basedir = htu.getDataTestDir();
     // Create a region.  That'll write the .regioninfo file.
     FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(htu.getConfiguration());
-    FSTableDescriptors.tryUpdateMetaTableDescriptor(htu.getConfiguration());
+    FSTableDescriptors.tryUpdateCatalogTableDescriptor(htu.getConfiguration());
     HRegion r = HBaseTestingUtility.createRegionAndWAL(hri, basedir, htu.getConfiguration(),
         fsTableDescriptors.get(TableName.META_TABLE_NAME));
     // Get modtime on the file.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
index 504a140..4de7281 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TestMetaTableAccessor;
+import org.apache.hadoop.hbase.TestCatalogAccessor;
 import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
@@ -146,7 +146,7 @@ public class TestRegionReplicas {
     Table meta = null;
     try {
       meta = HTU.getConnection().getTable(TableName.META_TABLE_NAME);
-      TestMetaTableAccessor.assertMetaLocation(meta, hriPrimary.getRegionName()
+      TestCatalogAccessor.assertMetaLocation(meta, hriPrimary.getRegionName()
         , getRS().getServerName(), -1, 1, false);
     } finally {
       if (meta != null) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithRestartScenarios.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithRestartScenarios.java
index d4e2507..6211db6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithRestartScenarios.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicasWithRestartScenarios.java
@@ -156,7 +156,7 @@ public class TestRegionReplicasWithRestartScenarios {
     assertFalse(res);
     int totalRegions = HTU.getMiniHBaseCluster().getLiveRegionServerThreads().stream().
       mapToInt(l -> l.getRegionServer().getOnlineRegions().size()).sum();
-    assertEquals(61, totalRegions);
+    assertEquals(62, totalRegions);
   }
 
   private boolean checkDuplicates(Collection<HRegion> onlineRegions3) throws Exception {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
index 81a8559..c675d95 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
 import org.apache.hadoop.hbase.util.Threads;
-import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
+import org.apache.hadoop.hbase.zookeeper.RootTableLocator;
 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -112,7 +112,7 @@ public class TestRegionServerNoMaster {
     HRegionServer hrs = HTU.getHBaseCluster()
       .getLiveRegionServerThreads().get(0).getRegionServer();
     ZKWatcher zkw = hrs.getZooKeeper();
-    ServerName sn = MetaTableLocator.getMetaRegionLocation(zkw);
+    ServerName sn = RootTableLocator.getRootRegionLocation(zkw);
     if (sn != null && !masterAddr.equals(sn)) {
       return;
     }
@@ -120,7 +120,7 @@ public class TestRegionServerNoMaster {
     ProtobufUtil.openRegion(null, hrs.getRSRpcServices(),
       hrs.getServerName(), RegionInfoBuilder.FIRST_META_REGIONINFO);
     while (true) {
-      sn = MetaTableLocator.getMetaRegionLocation(zkw);
+      sn = RootTableLocator.getRootRegionLocation(zkw);
       if (sn != null && sn.equals(hrs.getServerName())
           && hrs.getOnlineRegions().containsKey(
             RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName())) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
index ac89c92..15f6e02 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
@@ -93,7 +93,7 @@ public class TestLogRollingNoCluster {
     conf.set(WALFactory.WAL_PROVIDER, "filesystem");
     CommonFSUtils.setRootDir(conf, dir);
     FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(TEST_UTIL.getConfiguration());
-    FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration());
+    FSTableDescriptors.tryUpdateCatalogTableDescriptor(TEST_UTIL.getConfiguration());
     TableDescriptor metaTableDescriptor = fsTableDescriptors.get(TableName.META_TABLE_NAME);
     conf.set("hbase.regionserver.hlog.writer.impl", HighLatencySyncWriter.class.getName());
     final WALFactory wals = new WALFactory(conf, TestLogRollingNoCluster.class.getName());
@@ -159,7 +159,7 @@ public class TestLogRollingNoCluster {
       final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
       try {
         TableDescriptors tds = new FSTableDescriptors(TEST_UTIL.getConfiguration());
-        FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration());
+        FSTableDescriptors.tryUpdateCatalogTableDescriptor(TEST_UTIL.getConfiguration());
         TableDescriptor htd = tds.get(TableName.META_TABLE_NAME);
         for (int i = 0; i < this.count; i++) {
           long now = System.currentTimeMillis();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java
index 0d8d119..9ba0016 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsBasics.java
@@ -87,8 +87,8 @@ public class TestRSGroupsBasics extends TestRSGroupsBase {
     int count = MASTER.getAssignmentManager().getRegionStates().getRegionAssignments().size();
     LOG.info("regions assignments are" +
       MASTER.getAssignmentManager().getRegionStates().getRegionAssignments().toString());
-    // 2 (meta and rsgroup)
-    assertEquals(2, count);
+    // 3 (root, meta and rsgroup)
+    assertEquals(3, count);
   }
 
   @Test
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RootTableLocator.java
similarity index 73%
rename from hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
rename to hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RootTableLocator.java
index bb02af3..80de6e8 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RootTableLocator.java
@@ -22,7 +22,7 @@ import java.util.Collections;
 import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
+import org.apache.hadoop.hbase.NotAllRootRegionsOnlineException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionInfoBuilder;
@@ -50,55 +50,55 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaReg
  * TODO: rewrite using RPC calls to master to find out about hbase:meta.
  */
 @InterfaceAudience.Private
-public final class MetaTableLocator {
-  private static final Logger LOG = LoggerFactory.getLogger(MetaTableLocator.class);
+public final class RootTableLocator {
+  private static final Logger LOG = LoggerFactory.getLogger(RootTableLocator.class);
 
-  private MetaTableLocator() {
+  private RootTableLocator() {
   }
 
   /**
    * @param zkw ZooKeeper watcher to be used
-   * @return meta table regions and their locations.
+   * @return root table regions and their locations.
    */
-  public static List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZKWatcher zkw) {
-    return getMetaRegionsAndLocations(zkw, RegionInfo.DEFAULT_REPLICA_ID);
+  public static List<Pair<RegionInfo, ServerName>> getRootRegionsAndLocations(ZKWatcher zkw) {
+    return getRootRegionsAndLocations(zkw, RegionInfo.DEFAULT_REPLICA_ID);
   }
 
   /**
-   * Gets the meta regions and their locations for the given path and replica ID.
+   * Gets the root regions and their locations for the given path and replica ID.
    *
    * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
    * @param replicaId the ID of the replica
-   * @return meta table regions and their locations.
+   * @return root table regions and their locations.
    */
-  public static List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZKWatcher zkw,
+  public static List<Pair<RegionInfo, ServerName>> getRootRegionsAndLocations(ZKWatcher zkw,
       int replicaId) {
-    ServerName serverName = getMetaRegionLocation(zkw, replicaId);
+    ServerName serverName = getRootRegionLocation(zkw, replicaId);
     List<Pair<RegionInfo, ServerName>> list = new ArrayList<>(1);
     list.add(new Pair<>(RegionReplicaUtil.getRegionInfoForReplica(
-        RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), serverName));
+        RegionInfoBuilder.ROOT_REGIONINFO, replicaId), serverName));
     return list;
   }
 
   /**
-   * Gets the meta regions for the given path with the default replica ID.
+   * Gets the root regions for the given path with the default replica ID.
    *
    * @param zkw ZooKeeper watcher to be used
-   * @return List of meta regions
+   * @return List of root regions
    */
-  public static List<RegionInfo> getMetaRegions(ZKWatcher zkw) {
-    return getMetaRegions(zkw, RegionInfo.DEFAULT_REPLICA_ID);
+  public static List<RegionInfo> getRootRegions(ZKWatcher zkw) {
+    return getRootRegions(zkw, RegionInfo.DEFAULT_REPLICA_ID);
   }
 
   /**
-   * Gets the meta regions for the given path and replica ID.
+   * Gets the root regions for the given path and replica ID.
    * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
    * @param replicaId the ID of the replica
-   * @return List of meta regions
+   * @return List of root regions
    */
-  public static List<RegionInfo> getMetaRegions(ZKWatcher zkw, int replicaId) {
+  public static List<RegionInfo> getRootRegions(ZKWatcher zkw, int replicaId) {
     List<Pair<RegionInfo, ServerName>> result;
-    result = getMetaRegionsAndLocations(zkw, replicaId);
+    result = getRootRegionsAndLocations(zkw, replicaId);
     return getListOfRegionInfos(result);
   }
 
@@ -116,13 +116,13 @@ public final class MetaTableLocator {
   }
 
   /**
-   * Gets the meta region location, if available.  Does not block.
+   * Gets the root region location, if available.  Does not block.
    * @param zkw zookeeper connection to use
    * @return server name or null if we failed to get the data.
    */
-  public static ServerName getMetaRegionLocation(final ZKWatcher zkw) {
+  public static ServerName getRootRegionLocation(final ZKWatcher zkw) {
     try {
-      RegionState state = getMetaRegionState(zkw);
+      RegionState state = getRootRegionState(zkw);
       return state.isOpened() ? state.getServerName() : null;
     } catch (KeeperException ke) {
       return null;
@@ -130,14 +130,14 @@ public final class MetaTableLocator {
   }
 
   /**
-   * Gets the meta region location, if available.  Does not block.
+   * Gets the root region location, if available.  Does not block.
    * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
    * @param replicaId the ID of the replica
    * @return server name
    */
-  public static ServerName getMetaRegionLocation(final ZKWatcher zkw, int replicaId) {
+  public static ServerName getRootRegionLocation(final ZKWatcher zkw, int replicaId) {
     try {
-      RegionState state = getMetaRegionState(zkw, replicaId);
+      RegionState state = getRootRegionState(zkw, replicaId);
       return state.isOpened() ? state.getServerName() : null;
     } catch (KeeperException ke) {
       return null;
@@ -145,35 +145,35 @@ public final class MetaTableLocator {
   }
 
   /**
-   * Gets the meta region location, if available, and waits for up to the specified timeout if not
+   * Gets the root region location, if available, and waits for up to the specified timeout if not
    * immediately available. Given the zookeeper notification could be delayed, we will try to get
    * the latest data.
    * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
    * @param timeout maximum time to wait, in millis
-   * @return server name for server hosting meta region formatted as per {@link ServerName}, or null
+   * @return server name for server hosting root region formatted as per {@link ServerName}, or null
    *         if none available
    * @throws InterruptedException if interrupted while waiting
-   * @throws NotAllMetaRegionsOnlineException if a meta or root region is not online
+   * @throws NotAllRootRegionsOnlineException if a meta or root region is not online
    */
-  public static ServerName waitMetaRegionLocation(ZKWatcher zkw, long timeout)
-      throws InterruptedException, NotAllMetaRegionsOnlineException {
-    return waitMetaRegionLocation(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
+  public static ServerName waitRootRegionLocation(ZKWatcher zkw, long timeout)
+      throws InterruptedException, NotAllRootRegionsOnlineException {
+    return waitRootRegionLocation(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
   }
 
   /**
-   * Gets the meta region location, if available, and waits for up to the specified timeout if not
+   * Gets the root region location, if available, and waits for up to the specified timeout if not
    * immediately available. Given the zookeeper notification could be delayed, we will try to get
    * the latest data.
    * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
    * @param replicaId the ID of the replica
    * @param timeout maximum time to wait, in millis
-   * @return server name for server hosting meta region formatted as per {@link ServerName}, or null
+   * @return server name for server hosting root region formatted as per {@link ServerName}, or null
    *         if none available
    * @throws InterruptedException if waiting for the socket operation fails
-   * @throws NotAllMetaRegionsOnlineException if a meta or root region is not online
+   * @throws NotAllRootRegionsOnlineException if a roo or root region is not online
    */
-  public static ServerName waitMetaRegionLocation(ZKWatcher zkw, int replicaId, long timeout)
-      throws InterruptedException, NotAllMetaRegionsOnlineException {
+  public static ServerName waitRootRegionLocation(ZKWatcher zkw, int replicaId, long timeout)
+      throws InterruptedException, NotAllRootRegionsOnlineException {
     try {
       if (ZKUtil.checkExists(zkw, zkw.getZNodePaths().baseZNode) == -1) {
         String errorMsg = "Check the value configured in 'zookeeper.znode.parent'. " +
@@ -187,27 +187,27 @@ public final class MetaTableLocator {
     ServerName sn = blockUntilAvailable(zkw, replicaId, timeout);
 
     if (sn == null) {
-      throw new NotAllMetaRegionsOnlineException("Timed out; " + timeout + "ms");
+      throw new NotAllRootRegionsOnlineException("Timed out; " + timeout + "ms");
     }
 
     return sn;
   }
 
   /**
-   * Sets the location of <code>hbase:meta</code> in ZooKeeper to the
+   * Sets the location of <code>hbase:root</code> in ZooKeeper to the
    * specified server address.
    * @param zookeeper zookeeper reference
-   * @param serverName The server hosting <code>hbase:meta</code>
+   * @param serverName The server hosting <code>hbase:root</code>
    * @param state The region transition state
    * @throws KeeperException unexpected zookeeper exception
    */
-  public static void setMetaLocation(ZKWatcher zookeeper,
+  public static void setRootLocation(ZKWatcher zookeeper,
       ServerName serverName, RegionState.State state) throws KeeperException {
-    setMetaLocation(zookeeper, serverName, RegionInfo.DEFAULT_REPLICA_ID, state);
+    setRootLocation(zookeeper, serverName, RegionInfo.DEFAULT_REPLICA_ID, state);
   }
 
   /**
-   * Sets the location of <code>hbase:meta</code> in ZooKeeper to the specified server address.
+   * Sets the location of <code>hbase:root</code> in ZooKeeper to the specified server address.
    * @param zookeeper reference to the {@link ZKWatcher} which also contains configuration and
    *                  operation
    * @param serverName the name of the server
@@ -215,15 +215,15 @@ public final class MetaTableLocator {
    * @param state the state of the region
    * @throws KeeperException if a ZooKeeper operation fails
    */
-  public static void setMetaLocation(ZKWatcher zookeeper, ServerName serverName, int replicaId,
+  public static void setRootLocation(ZKWatcher zookeeper, ServerName serverName, int replicaId,
       RegionState.State state) throws KeeperException {
     if (serverName == null) {
-      LOG.warn("Tried to set null ServerName in hbase:meta; skipping -- ServerName required");
+      LOG.warn("Tried to set null ServerName in hbase:root; skipping -- ServerName required");
       return;
     }
-    LOG.info("Setting hbase:meta (replicaId={}) location in ZooKeeper as {}, state={}", replicaId,
+    LOG.info("Setting hbase:root (replicaId={}) location in ZooKeeper as {}, state={}", replicaId,
       serverName, state);
-    // Make the MetaRegionServer pb and then get its bytes and save this as
+    // Make the RootRegionServer pb and then get its bytes and save this as
     // the znode content.
     MetaRegionServer pbrsr = MetaRegionServer.newBuilder()
       .setServer(ProtobufUtil.toServerName(serverName))
@@ -235,9 +235,9 @@ public final class MetaTableLocator {
           zookeeper.getZNodePaths().getZNodeForReplica(replicaId), data);
     } catch(KeeperException.NoNodeException nne) {
       if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
-        LOG.debug("META region location doesn't exist, create it");
+        LOG.debug("ROOT region location doesn't exist, create it");
       } else {
-        LOG.debug("META region location doesn't exist for replicaId=" + replicaId +
+        LOG.debug("ROOT region location doesn't exist for replicaId=" + replicaId +
             ", create it");
       }
       ZKUtil.createAndWatch(zookeeper, zookeeper.getZNodePaths().getZNodeForReplica(replicaId),
@@ -246,26 +246,26 @@ public final class MetaTableLocator {
   }
 
   /**
-   * Load the meta region state from the meta server ZNode.
+   * Load the root region state from the root server ZNode.
    */
-  public static RegionState getMetaRegionState(ZKWatcher zkw) throws KeeperException {
-    return getMetaRegionState(zkw, RegionInfo.DEFAULT_REPLICA_ID);
+  public static RegionState getRootRegionState(ZKWatcher zkw) throws KeeperException {
+    return getRootRegionState(zkw, RegionInfo.DEFAULT_REPLICA_ID);
   }
 
   /**
-   * Load the meta region state from the meta region server ZNode.
+   * Load the root region state from the root region server ZNode.
    *
    * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
    * @param replicaId the ID of the replica
    * @return regionstate
    * @throws KeeperException if a ZooKeeper operation fails
    */
-  public static RegionState getMetaRegionState(ZKWatcher zkw, int replicaId)
+  public static RegionState getRootRegionState(ZKWatcher zkw, int replicaId)
       throws KeeperException {
     RegionState regionState = null;
     try {
       byte[] data = ZKUtil.getData(zkw, zkw.getZNodePaths().getZNodeForReplica(replicaId));
-      regionState = ProtobufUtil.parseMetaRegionStateFrom(data, replicaId);
+      regionState = ProtobufUtil.parseRootRegionStateFrom(data, replicaId);
     } catch (DeserializationException e) {
       throw ZKUtil.convert(e);
     } catch (InterruptedException e) {
@@ -275,21 +275,21 @@ public final class MetaTableLocator {
   }
 
   /**
-   * Deletes the location of <code>hbase:meta</code> in ZooKeeper.
+   * Deletes the location of <code>hbase:root</code> in ZooKeeper.
    * @param zookeeper zookeeper reference
    * @throws KeeperException unexpected zookeeper exception
    */
-  public static void deleteMetaLocation(ZKWatcher zookeeper)
+  public static void deleteRootLocation(ZKWatcher zookeeper)
     throws KeeperException {
-    deleteMetaLocation(zookeeper, RegionInfo.DEFAULT_REPLICA_ID);
+    deleteRootLocation(zookeeper, RegionInfo.DEFAULT_REPLICA_ID);
   }
 
-  public static void deleteMetaLocation(ZKWatcher zookeeper, int replicaId)
+  public static void deleteRootLocation(ZKWatcher zookeeper, int replicaId)
     throws KeeperException {
     if (replicaId == RegionInfo.DEFAULT_REPLICA_ID) {
-      LOG.info("Deleting hbase:meta region location in ZooKeeper");
+      LOG.info("Deleting hbase:root region location in ZooKeeper");
     } else {
-      LOG.info("Deleting hbase:meta for {} region location in ZooKeeper", replicaId);
+      LOG.info("Deleting hbase:root for {} region location in ZooKeeper", replicaId);
     }
     try {
       // Just delete the node.  Don't need any watches.
@@ -299,7 +299,7 @@ public final class MetaTableLocator {
     }
   }
   /**
-   * Wait until the primary meta region is available. Get the secondary locations as well but don't
+   * Wait until the primary root region is available. Get the secondary locations as well but don't
    * block for those.
    *
    * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
@@ -324,14 +324,14 @@ public final class MetaTableLocator {
     servers.add(server);
 
     try {
-      List<String> metaReplicaNodes = zkw.getMetaReplicaNodes();
-      numReplicasConfigured = metaReplicaNodes.size();
+      List<String> rootReplicaNodes = zkw.getRootReplicaNodes();
+      numReplicasConfigured = rootReplicaNodes.size();
     } catch (KeeperException e) {
       LOG.warn("Got ZK exception {}", e);
     }
     for (int replicaId = 1; replicaId < numReplicasConfigured; replicaId++) {
-      // return all replica locations for the meta
-      servers.add(getMetaRegionLocation(zkw, replicaId));
+      // return all replica locations for the root
+      servers.add(getRootRegionLocation(zkw, replicaId));
     }
     return servers;
   }
@@ -369,7 +369,7 @@ public final class MetaTableLocator {
     long startTime = System.currentTimeMillis();
     ServerName sn = null;
     while (true) {
-      sn = getMetaRegionLocation(zkw, replicaId);
+      sn = getRootRegionLocation(zkw, replicaId);
       if (sn != null ||
         (System.currentTimeMillis() - startTime) > timeout - HConstants.SOCKET_RETRY_WAIT_MS) {
         break;
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
index 19d11d0..4ef4371 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -1860,13 +1860,13 @@ public final class ZKUtil {
         }
       }
       sb.append("\nRegion server holding hbase:meta: "
-        + MetaTableLocator.getMetaRegionLocation(zkw));
+        + RootTableLocator.getRootRegionLocation(zkw));
       Configuration conf = HBaseConfiguration.create();
       int numMetaReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
                HConstants.DEFAULT_META_REPLICA_NUM);
       for (int i = 1; i < numMetaReplicas; i++) {
         sb.append("\nRegion server holding hbase:meta, replicaId " + i + " "
-                    + MetaTableLocator.getMetaRegionLocation(zkw, i));
+                    + RootTableLocator.getRootRegionLocation(zkw, i));
       }
       sb.append("\nRegion servers:");
       final List<String> rsChildrenNoWatchList =
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
index 136134d..ca7d97a 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKWatcher.java
@@ -400,37 +400,37 @@ public class ZKWatcher implements Watcher, Abortable, Closeable {
    * @return list of znodes
    * @throws KeeperException if a ZooKeeper operation fails
    */
-  public List<String> getMetaReplicaNodes() throws KeeperException {
+  public List<String> getRootReplicaNodes() throws KeeperException {
     List<String> childrenOfBaseNode = ZKUtil.listChildrenNoWatch(this, znodePaths.baseZNode);
-    return filterMetaReplicaNodes(childrenOfBaseNode);
+    return filterRootReplicaNodes(childrenOfBaseNode);
   }
 
   /**
-   * Same as {@link #getMetaReplicaNodes()} except that this also registers a watcher on base znode
+   * Same as {@link #getRootReplicaNodes()} except that this also registers a watcher on base znode
    * for subsequent CREATE/DELETE operations on child nodes.
    */
-  public List<String> getMetaReplicaNodesAndWatchChildren() throws KeeperException {
+  public List<String> getRootReplicaNodesAndWatchChildren() throws KeeperException {
     List<String> childrenOfBaseNode =
         ZKUtil.listChildrenAndWatchForNewChildren(this, znodePaths.baseZNode);
-    return filterMetaReplicaNodes(childrenOfBaseNode);
+    return filterRootReplicaNodes(childrenOfBaseNode);
   }
 
   /**
    * @param nodes Input list of znodes
    * @return Filtered list of znodes from nodes that belong to meta replica(s).
    */
-  private List<String> filterMetaReplicaNodes(List<String> nodes) {
+  private List<String> filterRootReplicaNodes(List<String> nodes) {
     if (nodes == null || nodes.isEmpty()) {
       return new ArrayList<>();
     }
-    List<String> metaReplicaNodes = new ArrayList<>(2);
+    List<String> rootReplicaNodes = new ArrayList<>(2);
     String pattern = conf.get(ZNodePaths.META_ZNODE_PREFIX_CONF_KEY, ZNodePaths.META_ZNODE_PREFIX);
     for (String child : nodes) {
       if (child.startsWith(pattern)) {
-        metaReplicaNodes.add(child);
+        rootReplicaNodes.add(child);
       }
     }
-    return metaReplicaNodes;
+    return rootReplicaNodes;
   }
 
   /**