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

[hbase] branch HBASE-24950 updated: HBASE-24391 Implement meta split (#2010)

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

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


The following commit(s) were added to refs/heads/HBASE-24950 by this push:
     new e5cf12e  HBASE-24391 Implement meta split (#2010)
e5cf12e is described below

commit e5cf12e94a04a880f4b7eefc66f01aee837769c6
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Mon Sep 7 10:59:43 2020 +0800

    HBASE-24391 Implement meta split (#2010)
    
    Signed-off-by: Guanghao Zhang <zg...@apache.org>
---
 .../org/apache/hadoop/hbase/MetaTableAccessor.java |  74 ++-----------
 .../AsyncScanSingleRegionRpcRetryingCaller.java    |  47 ++++++--
 .../hadoop/hbase/client/ConnectionUtils.java       |  26 -----
 .../hbase/client/TableRegionLocationCache.java     |   2 +-
 .../hbase/master/assignment/AssignmentManager.java |   7 +-
 .../hbase/master/assignment/RegionStateStore.java  |  62 ++++++++++-
 .../master/procedure/ModifyTableProcedure.java     |  16 ++-
 .../DelimitedKeyPrefixRegionSplitPolicy.java       |   2 +-
 .../apache/hadoop/hbase/regionserver/HRegion.java  |   5 -
 .../apache/hadoop/hbase/regionserver/HStore.java   |   9 +-
 .../hbase/regionserver/MetaRegionSplitPolicy.java  |  40 +++++++
 .../hbase/regionserver/RegionSplitPolicy.java      |   3 +-
 .../hadoop/hbase/util/FSTableDescriptors.java      |   8 +-
 .../apache/hadoop/hbase/TestMetaSplitBySize.java   |  62 +++++++++++
 .../apache/hadoop/hbase/TestMetaTableAccessor.java | 121 ---------------------
 .../hbase/TestMetaUpdatesGoToPriorityQueue.java    | 100 +++++++++++++++++
 .../apache/hadoop/hbase/TestSimpleMetaSplit.java   |  98 +++++++++++++++++
 .../client/TestShutdownOfMetaReplicaHolder.java    |  22 +++-
 .../master/assignment/TestRegionStateStore.java    | 119 +++++++++++++++++---
 19 files changed, 553 insertions(+), 270 deletions(-)

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 6144de9..c424513 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
@@ -44,7 +44,6 @@ 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;
@@ -751,7 +750,7 @@ public final class MetaTableAccessor {
   /**
    * Adds split daughters to the Put
    */
-  private static Put addDaughtersToPut(Put put, RegionInfo splitA, RegionInfo splitB)
+  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())
@@ -865,7 +864,7 @@ public final class MetaTableAccessor {
     }
   }
 
-  private static Put addRegionStateToPut(Put put, RegionState.State state) throws IOException {
+  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()))
@@ -885,9 +884,7 @@ public final class MetaTableAccessor {
   /**
    * 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.
+   * as the parent.
    * @param connection connection we're using
    * @param regionInfo RegionInfo of parent region
    * @param splitA first split daughter of the parent regionInfo
@@ -906,11 +903,7 @@ public final class MetaTableAccessor {
   }
 
   /**
-   * 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.
+   * Adds a (single) hbase:meta row for the specified new region and its daughters.
    * @param connection connection we're using
    * @param regionInfo region information
    * @throws IOException if problem connecting or updating meta
@@ -1038,56 +1031,6 @@ public final class MetaTableAccessor {
   }
 
   /**
-   * Splits the region into two in an atomic operation. Offlines the parent region with the
-   * information that it is split into two, and also adds the daughter regions. Does not add the
-   * location information to the daughter regions since they are not open yet.
-   * @param connection connection we're using
-   * @param parent the parent region which is split
-   * @param parentOpenSeqNum the next open sequence id for parent region, used by serial
-   *          replication. -1 if not necessary.
-   * @param splitA Split daughter region A
-   * @param splitB Split daughter region B
-   * @param sn the location of the region
-   */
-  public static void splitRegion(Connection connection, RegionInfo parent, long parentOpenSeqNum,
-    RegionInfo splitA, RegionInfo splitB, ServerName sn, int regionReplication) throws IOException {
-    long time = EnvironmentEdgeManager.currentTime();
-    // Put for parent
-    Put putParent = makePutFromRegionInfo(
-      RegionInfoBuilder.newBuilder(parent).setOffline(true).setSplit(true).build(), time);
-    addDaughtersToPut(putParent, splitA, splitB);
-
-    // Puts for daughters
-    Put putA = makePutFromRegionInfo(splitA, time);
-    Put putB = makePutFromRegionInfo(splitB, time);
-    if (parentOpenSeqNum > 0) {
-      addReplicationBarrier(putParent, parentOpenSeqNum);
-      addReplicationParent(putA, Collections.singletonList(parent));
-      addReplicationParent(putB, Collections.singletonList(parent));
-    }
-    // Set initial state to CLOSED
-    // NOTE: If initial state is not set to CLOSED then daughter regions get added with the
-    // 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}
-    addRegionStateToPut(putA, RegionState.State.CLOSED);
-    addRegionStateToPut(putB, RegionState.State.CLOSED);
-
-    addSequenceNum(putA, 1, splitA.getReplicaId()); // new regions, openSeqNum = 1 is fine.
-    addSequenceNum(putB, 1, splitB.getReplicaId());
-
-    // Add empty locations for region replicas of daughters 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(putA, i);
-      addEmptyLocation(putB, i);
-    }
-
-    byte[] tableRow = Bytes.toBytes(parent.getRegionNameAsString() + HConstants.DELIMITER);
-    multiMutate(connection, tableRow, Arrays.asList(putParent, putA, putB));
-  }
-
-  /**
    * Update state of the table in meta.
    * @param connection what we use for update
    * @param state new state
@@ -1121,12 +1064,13 @@ public final class MetaTableAccessor {
     deleteFromMetaTable(connection, delete);
     LOG.info("Deleted table " + table + " state from META");
   }
+
   /**
    * Performs an atomic multi-mutate operation against the given table. Used by the likes of merge
    * and split as these want to make atomic mutations across multiple rows.
    * @throws IOException even if we encounter a RuntimeException, we'll still wrap it in an IOE.
    */
-  private static void multiMutate(Connection conn, byte[] row, List<Mutation> mutations)
+  public static void multiMutate(Connection conn, byte[] row, List<Mutation> mutations)
     throws IOException {
     debugLogMutations(mutations);
     MutateRowsRequest.Builder builder = MutateRowsRequest.newBuilder();
@@ -1364,7 +1308,7 @@ public final class MetaTableAccessor {
     return parents;
   }
 
-  private static void addReplicationParent(Put put, List<RegionInfo> parents) throws IOException {
+  public static void addReplicationParent(Put put, List<RegionInfo> parents) throws IOException {
     byte[] value = getParentsBytes(parents);
     put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
       .setFamily(HConstants.REPLICATION_BARRIER_FAMILY).setQualifier(REPLICATION_PARENT_QUALIFIER)
@@ -1388,7 +1332,7 @@ public final class MetaTableAccessor {
       .build());
   }
 
-  private static Put addEmptyLocation(Put p, int replicaId) throws IOException {
+  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)
@@ -1541,7 +1485,7 @@ public final class MetaTableAccessor {
     METALOG.debug("{} {}", p.getClass().getSimpleName(), p.toJSON());
   }
 
-  private static Put addSequenceNum(Put p, long openSeqNum, int replicaId) throws IOException {
+  public static Put addSequenceNum(Put p, long openSeqNum, int replicaId) throws IOException {
     return p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(p.getRow())
       .setFamily(HConstants.CATALOG_FAMILY)
       .setQualifier(CatalogFamilyFormat.getSeqNumColumn(replicaId)).setTimestamp(p.getTimestamp())
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
index 1fa3c81..3c41df1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncScanSingleRegionRpcRetryingCaller.java
@@ -21,8 +21,8 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.SLEEP_DELTA_NS;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.getPauseTime;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.incRPCCallsMetrics;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.incRPCRetriesMetrics;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForReverseScan;
-import static org.apache.hadoop.hbase.client.ConnectionUtils.noMoreResultsForScan;
+import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStartRow;
+import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.resetController;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.translateException;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.updateResultsMetrics;
@@ -30,6 +30,7 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.updateServerSideMet
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Comparator;
 import java.util.List;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
@@ -37,6 +38,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.hbase.CallQueueTooBigException;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MetaCellComparator;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.UnknownScannerException;
 import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer.ScanResumer;
@@ -45,6 +47,7 @@ import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
 import org.apache.hadoop.hbase.exceptions.ScannerResetException;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -114,6 +117,8 @@ class AsyncScanSingleRegionRpcRetryingCaller {
 
   private final HBaseRpcController controller;
 
+  private final Comparator<byte[]> comparator;
+
   private byte[] nextStartRowWhenError;
 
   private boolean includeNextStartRowWhenError;
@@ -304,11 +309,11 @@ class AsyncScanSingleRegionRpcRetryingCaller {
   }
 
   public AsyncScanSingleRegionRpcRetryingCaller(Timer retryTimer, AsyncConnectionImpl conn,
-      Scan scan, ScanMetrics scanMetrics, long scannerId, ScanResultCache resultCache,
-      AdvancedScanResultConsumer consumer, Interface stub, HRegionLocation loc,
-      boolean isRegionServerRemote, int priority, long scannerLeaseTimeoutPeriodNs, long pauseNs,
-      long pauseForCQTBENs, int maxAttempts, long scanTimeoutNs, long rpcTimeoutNs,
-      int startLogErrorsCnt) {
+    Scan scan, ScanMetrics scanMetrics, long scannerId, ScanResultCache resultCache,
+    AdvancedScanResultConsumer consumer, Interface stub, HRegionLocation loc,
+    boolean isRegionServerRemote, int priority, long scannerLeaseTimeoutPeriodNs, long pauseNs,
+    long pauseForCQTBENs, int maxAttempts, long scanTimeoutNs, long rpcTimeoutNs,
+    int startLogErrorsCnt) {
     this.retryTimer = retryTimer;
     this.scan = scan;
     this.scanMetrics = scanMetrics;
@@ -335,6 +340,8 @@ class AsyncScanSingleRegionRpcRetryingCaller {
     this.controller = conn.rpcControllerFactory.newController();
     this.controller.setPriority(priority);
     this.exceptions = new ArrayList<>();
+    this.comparator =
+      loc.getRegion().isMetaRegion() ? MetaCellComparator.ROW_COMPARATOR : Bytes.BYTES_COMPARATOR;
   }
 
   private long elapsedMs() {
@@ -442,6 +449,32 @@ class AsyncScanSingleRegionRpcRetryingCaller {
     includeNextStartRowWhenError = result.mayHaveMoreCellsInRow();
   }
 
+  private boolean noMoreResultsForScan(Scan scan, RegionInfo info) {
+    if (isEmptyStopRow(info.getEndKey())) {
+      return true;
+    }
+    if (isEmptyStopRow(scan.getStopRow())) {
+      return false;
+    }
+    int c = comparator.compare(info.getEndKey(), scan.getStopRow());
+    // 1. if our stop row is less than the endKey of the region
+    // 2. if our stop row is equal to the endKey of the region and we do not include the stop row
+    // for scan.
+    return c > 0 || (c == 0 && !scan.includeStopRow());
+  }
+
+  private boolean noMoreResultsForReverseScan(Scan scan, RegionInfo info) {
+    if (isEmptyStartRow(info.getStartKey())) {
+      return true;
+    }
+    if (isEmptyStopRow(scan.getStopRow())) {
+      return false;
+    }
+    // no need to test the inclusive of the stop row as the start key of a region is included in
+    // the region.
+    return comparator.compare(info.getStartKey(), scan.getStopRow()) <= 0;
+  }
+
   private void completeWhenNoMoreResultsInRegion() {
     if (noMoreResultsForScan(scan, loc.getRegion())) {
       completeNoMoreResults();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
index 86d6442..8f298e6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionUtils.java
@@ -296,32 +296,6 @@ public final class ConnectionUtils {
     }
   }
 
-  static boolean noMoreResultsForScan(Scan scan, RegionInfo info) {
-    if (isEmptyStopRow(info.getEndKey())) {
-      return true;
-    }
-    if (isEmptyStopRow(scan.getStopRow())) {
-      return false;
-    }
-    int c = Bytes.compareTo(info.getEndKey(), scan.getStopRow());
-    // 1. if our stop row is less than the endKey of the region
-    // 2. if our stop row is equal to the endKey of the region and we do not include the stop row
-    // for scan.
-    return c > 0 || (c == 0 && !scan.includeStopRow());
-  }
-
-  static boolean noMoreResultsForReverseScan(Scan scan, RegionInfo info) {
-    if (isEmptyStartRow(info.getStartKey())) {
-      return true;
-    }
-    if (isEmptyStopRow(scan.getStopRow())) {
-      return false;
-    }
-    // no need to test the inclusive of the stop row as the start key of a region is included in
-    // the region.
-    return Bytes.compareTo(info.getStartKey(), scan.getStopRow()) <= 0;
-  }
-
   static <T> CompletableFuture<List<T>> allOf(List<CompletableFuture<T>> futures) {
     return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]))
       .thenApply(v -> futures.stream().map(f -> f.getNow(null)).collect(toList()));
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableRegionLocationCache.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableRegionLocationCache.java
index 41f203e..ed25b5d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableRegionLocationCache.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/TableRegionLocationCache.java
@@ -111,7 +111,7 @@ class TableRegionLocationCache {
         // the region is different, here we trust the one we fetched. This maybe wrong but finally
         // the upper layer can detect this and trigger removal of the wrong locations
         if (LOG.isDebugEnabled()) {
-          LOG.debug("The newnly fetch region {} is different from the old one {} for row '{}'," +
+          LOG.debug("The newly fetch region {} is different from the old one {} for row '{}'," +
             " try replaing the old one...", region, oldRegion, Bytes.toStringBinary(startKey));
         }
         if (cache.replace(startKey, oldLocs, locs)) {
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 c089927..3db221e 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
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionStatesCount;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
 import org.apache.hadoop.hbase.favored.FavoredNodesManager;
@@ -1820,9 +1821,8 @@ public class AssignmentManager {
   // ============================================================================================
   // The above methods can only be called in TransitRegionStateProcedure(and related procedures)
   // ============================================================================================
-
   public void markRegionAsSplit(final RegionInfo parent, final ServerName serverName,
-      final RegionInfo daughterA, final RegionInfo daughterB) throws IOException {
+    final RegionInfo daughterA, final RegionInfo daughterB) throws IOException {
     // Update hbase:meta. Parent will be marked offline and split up in hbase:meta.
     // The parent stays in regionStates until cleared when removed by CatalogJanitor.
     // Update its state in regionStates to it shows as offline and split when read
@@ -1835,7 +1835,8 @@ public class AssignmentManager {
     final RegionStateNode nodeB = regionStates.getOrCreateRegionStateNode(daughterB);
     nodeB.setState(State.SPLITTING_NEW);
 
-    regionStateStore.splitRegion(parent, daughterA, daughterB, serverName);
+    TableDescriptor td = master.getTableDescriptors().get(parent.getTable());
+    regionStateStore.splitRegion(parent, daughterA, daughterB, serverName, td);
     if (shouldAssignFavoredNodes(parent)) {
       List<ServerName> onlineServers = this.master.getServerManager().getOnlineServersList();
       getFavoredNodePromoter().generateFavoredNodesForDaughter(onlineServers, parent, daughterA,
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 5a73f9e..7c36b50 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
@@ -18,10 +18,12 @@
 package org.apache.hadoop.hbase.master.assignment;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.stream.Collectors;
 import org.apache.hadoop.hbase.CatalogFamilyFormat;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellBuilderFactory;
@@ -33,13 +35,16 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.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.Result;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.MasterFileSystem;
 import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.master.RegionState.State;
 import org.apache.hadoop.hbase.master.region.MasterRegion;
 import org.apache.hadoop.hbase.procedure2.Procedure;
@@ -250,15 +255,62 @@ public class RegionStateStore {
   // ============================================================================================
   //  Update Region Splitting State helpers
   // ============================================================================================
-  public void splitRegion(RegionInfo parent, RegionInfo hriA, RegionInfo hriB,
-      ServerName serverName) throws IOException {
-    TableDescriptor htd = getDescriptor(parent.getTable());
+  /**
+   * Splits the region into two in an atomic operation. Offlines the parent region with the
+   * information that it is split into two, and also adds the daughter regions. Does not add the
+   * location information to the daughter regions since they are not open yet.
+   */
+  public void splitRegion(RegionInfo parent, RegionInfo splitA, RegionInfo splitB,
+    ServerName serverName, TableDescriptor htd) throws IOException {
     long parentOpenSeqNum = HConstants.NO_SEQNUM;
     if (htd.hasGlobalReplicationScope()) {
       parentOpenSeqNum = getOpenSeqNumForParentRegion(parent);
     }
-    MetaTableAccessor.splitRegion(master.getConnection(), parent, parentOpenSeqNum, hriA, hriB,
-      serverName, getRegionReplication(htd));
+    long time = EnvironmentEdgeManager.currentTime();
+    // Put for parent
+    Put putParent = MetaTableAccessor.makePutFromRegionInfo(
+      RegionInfoBuilder.newBuilder(parent).setOffline(true).setSplit(true).build(), time);
+    MetaTableAccessor.addDaughtersToPut(putParent, splitA, splitB);
+
+    // Puts for daughters
+    Put putA = MetaTableAccessor.makePutFromRegionInfo(splitA, time);
+    Put putB = MetaTableAccessor.makePutFromRegionInfo(splitB, time);
+    if (parentOpenSeqNum > 0) {
+      MetaTableAccessor.addReplicationBarrier(putParent, parentOpenSeqNum);
+      MetaTableAccessor.addReplicationParent(putA, Collections.singletonList(parent));
+      MetaTableAccessor.addReplicationParent(putB, Collections.singletonList(parent));
+    }
+    // Set initial state to CLOSED
+    // NOTE: If initial state is not set to CLOSED then daughter regions get added with the
+    // 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);
+
+    // new regions, openSeqNum = 1 is fine.
+    MetaTableAccessor.addSequenceNum(putA, 1, splitA.getReplicaId());
+    MetaTableAccessor.addSequenceNum(putB, 1, splitB.getReplicaId());
+
+    // Add empty locations for region replicas of daughters 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(putA, i);
+      MetaTableAccessor.addEmptyLocation(putB, i);
+    }
+
+    List<Mutation> mutations = Arrays.asList(putParent, putA, putB);
+    if (htd.isMetaTable()) {
+      masterRegion.update(region -> {
+        List<byte[]> rowsToLock =
+          mutations.stream().map(Mutation::getRow).collect(Collectors.toList());
+        region.mutateRowsWithLocks(mutations, rowsToLock, HConstants.NO_NONCE, HConstants.NO_NONCE);
+      });
+    } else {
+      byte[] tableRow = Bytes.toBytes(parent.getRegionNameAsString() + HConstants.DELIMITER);
+      MetaTableAccessor.multiMutate(master.getConnection(), tableRow, mutations);
+    }
   }
 
   // ============================================================================================
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
index 64f4bf6..4ca33b2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableState;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.regionserver.MetaRegionSplitPolicy;
 import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
@@ -105,17 +106,24 @@ public class ModifyTableProcedure
     if (this.modifiedTableDescriptor.isMetaTable()) {
       // If we are modifying the hbase:meta table, make sure we are not deleting critical
       // column families else we'll damage the cluster.
-      Set<byte []> cfs = this.modifiedTableDescriptor.getColumnFamilyNames();
+      Set<byte[]> cfs = this.modifiedTableDescriptor.getColumnFamilyNames();
       for (byte[] family : UNDELETABLE_META_COLUMNFAMILIES) {
         if (!cfs.contains(family)) {
-          throw new HBaseIOException("Delete of hbase:meta column family " +
-            Bytes.toString(family));
+          throw new HBaseIOException(
+            "Delete of hbase:meta column family " + Bytes.toString(family));
         }
       }
+      // also check if we want to change the split policy, which is not allowed
+      if (!MetaRegionSplitPolicy.class.getName()
+        .equals(this.modifiedTableDescriptor.getRegionSplitPolicyClassName())) {
+        throw new HBaseIOException("Can not change split policy for hbase:meta to " +
+          this.modifiedTableDescriptor.getRegionSplitPolicyClassName());
+      }
     }
   }
 
-  private void initialize(final TableDescriptor unmodifiedTableDescriptor,
+  private void initialize(
+    final TableDescriptor unmodifiedTableDescriptor,
       final boolean shouldCheckDescriptor) {
     this.unmodifiedTableDescriptor = unmodifiedTableDescriptor;
     this.shouldCheckDescriptor = shouldCheckDescriptor;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java
index 8bafaf5..7ff1a4d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.java
@@ -45,7 +45,7 @@ public class DelimitedKeyPrefixRegionSplitPolicy extends IncreasingToUpperBoundR
       .getLogger(DelimitedKeyPrefixRegionSplitPolicy.class);
   public static final String DELIMITER_KEY = "DelimitedKeyPrefixRegionSplitPolicy.delimiter";
 
-  private byte[] delimiter = null;
+  protected byte[] delimiter = null;
 
   @Override
   protected void configureForRegion(HRegion region) {
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..652901e 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
@@ -8589,11 +8589,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
    * Return the split point. An empty result indicates the region isn't splittable.
    */
   public Optional<byte[]> checkSplit(boolean force) {
-    // Can't split META
-    if (this.getRegionInfo().isMetaRegion()) {
-      return Optional.empty();
-    }
-
     // Can't split a region that is closing.
     if (this.isClosing()) {
       return Optional.empty();
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..c421002 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
@@ -53,7 +53,6 @@ import java.util.function.Predicate;
 import java.util.function.ToLongFunction;
 import java.util.stream.Collectors;
 import java.util.stream.LongStream;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -96,8 +95,6 @@ import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.hadoop.hbase.util.ClassSize;
@@ -110,6 +107,7 @@ import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
 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 org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableCollection;
@@ -119,6 +117,9 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
 import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
 import org.apache.hbase.thirdparty.org.apache.commons.collections4.IterableUtils;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
 /**
  * A Store holds a column family in a Region.  Its a memstore and a set of zero
  * or more StoreFiles, which stretch backwards over time.
@@ -2115,8 +2116,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
   public Optional<byte[]> getSplitPoint() {
     this.lock.readLock().lock();
     try {
-      // Should already be enforced by the split policy!
-      assert !this.getRegionInfo().isMetaRegion();
       // 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/MetaRegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetaRegionSplitPolicy.java
new file mode 100644
index 0000000..b0b13fa
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetaRegionSplitPolicy.java
@@ -0,0 +1,40 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * The split policy for meta.
+ * <p/>
+ * Now we just use {@link DelimitedKeyPrefixRegionSplitPolicy} with
+ * {@value org.apache.hadoop.hbase.HConstants#DELIMITER}, which means all the records for a table
+ * will be in the same region, so the multi-mutate operation when splitting/merging is still valid.
+ */
+@InterfaceAudience.Private
+public class MetaRegionSplitPolicy extends DelimitedKeyPrefixRegionSplitPolicy {
+
+  @Override
+  protected void configureForRegion(HRegion region) {
+    // TODO: it will issue an error of can not find the delimiter
+    super.configureForRegion(region);
+    delimiter = Bytes.toBytes(HConstants.DELIMITER);
+  }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
index 4a13030..3d94c37 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java
@@ -74,8 +74,7 @@ public abstract class RegionSplitPolicy extends Configured {
    * @return {@code true} if the specified region can be split.
    */
   protected boolean canSplit() {
-    return !region.getRegionInfo().isMetaRegion() && region.isAvailable() &&
-      region.getStores().stream().allMatch(HStore::canSplit);
+    return region.isAvailable() && region.getStores().stream().allMatch(HStore::canSplit);
   }
 
   /**
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..3054c2c 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
@@ -47,6 +47,7 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.apache.hadoop.hbase.regionserver.MetaRegionSplitPolicy;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -192,9 +193,10 @@ public class FSTableDescriptors implements TableDescriptors {
         .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());
+      .setCoprocessor(
+        CoprocessorDescriptorBuilder.newBuilder(MultiRowMutationEndpoint.class.getName())
+          .setPriority(Coprocessor.PRIORITY_SYSTEM).build())
+      .setRegionSplitPolicyClassName(MetaRegionSplitPolicy.class.getName());
   }
 
   @VisibleForTesting
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaSplitBySize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaSplitBySize.java
new file mode 100644
index 0000000..dd2d734
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaSplitBySize.java
@@ -0,0 +1,62 @@
+/**
+ * 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.io.IOException;
+import org.apache.hadoop.hbase.regionserver.FlushLifeCycleTracker;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MiscTests.class, MediumTests.class })
+public class TestMetaSplitBySize {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestMetaSplitBySize.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.getConfiguration().setLong("hbase.increasing.policy.initial.size", 1024);
+    UTIL.startMiniCluster(3);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws IOException {
+    byte[] family = Bytes.toBytes("family");
+    for (int i = 0; i < 10; i++) {
+      UTIL.createTable(TableName.valueOf("table_" + i), family);
+    }
+    HRegion region = UTIL.getMiniHBaseCluster().getRegions(TableName.META_TABLE_NAME).get(0);
+    region.requestFlush(FlushLifeCycleTracker.DUMMY);
+    UTIL.waitFor(30000, () -> UTIL.getAdmin().getRegions(TableName.META_TABLE_NAME).size() > 1);
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
index 372d4db..cc018dd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
@@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.ipc.RpcScheduler;
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
-import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.regionserver.SimpleRpcSchedulerFactory;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
@@ -500,34 +499,6 @@ public class TestMetaTableAccessor {
   }
 
   @Test
-  public void testMetaLocationForRegionReplicasIsAddedAtRegionSplit() throws IOException {
-    long regionId = System.currentTimeMillis();
-    ServerName serverName0 = ServerName.valueOf("foo", 60010, random.nextLong());
-    RegionInfo parent = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
-      .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(HConstants.EMPTY_END_ROW).setSplit(false)
-      .setRegionId(regionId).setReplicaId(0).build();
-
-    RegionInfo splitA = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
-      .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(Bytes.toBytes("a")).setSplit(false)
-      .setRegionId(regionId + 1).setReplicaId(0).build();
-    RegionInfo splitB = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
-      .setStartKey(Bytes.toBytes("a")).setEndKey(HConstants.EMPTY_END_ROW).setSplit(false)
-      .setRegionId(regionId + 1).setReplicaId(0).build();
-
-    try (Table meta = MetaTableAccessor.getMetaHTable(connection)) {
-      List<RegionInfo> regionInfos = Lists.newArrayList(parent);
-      MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
-
-      MetaTableAccessor.splitRegion(connection, parent, -1L, splitA, splitB, serverName0, 3);
-
-      assertEmptyMetaLocation(meta, splitA.getRegionName(), 1);
-      assertEmptyMetaLocation(meta, splitA.getRegionName(), 2);
-      assertEmptyMetaLocation(meta, splitB.getRegionName(), 1);
-      assertEmptyMetaLocation(meta, splitB.getRegionName(), 2);
-    }
-  }
-
-  @Test
   public void testMetaLocationForRegionReplicasIsAddedAtRegionMerge() throws IOException {
     long regionId = System.currentTimeMillis();
     ServerName serverName0 = ServerName.valueOf("foo", 60010, random.nextLong());
@@ -731,98 +702,6 @@ public class TestMetaTableAccessor {
   }
 
   @Test
-  public void testMetaUpdatesGoToPriorityQueue() throws Exception {
-    // This test has to be end-to-end, and do the verification from the server side
-    Configuration c = UTIL.getConfiguration();
-
-    c.set(RSRpcServices.REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS,
-      SpyingRpcSchedulerFactory.class.getName());
-
-    // restart so that new config takes place
-    afterClass();
-    beforeClass();
-
-    final TableName tableName = TableName.valueOf(name.getMethodName());
-    try (Admin admin = connection.getAdmin();
-      RegionLocator rl = connection.getRegionLocator(tableName)) {
-
-      // create a table and prepare for a manual split
-      UTIL.createTable(tableName, "cf1");
-
-      HRegionLocation loc = rl.getAllRegionLocations().get(0);
-      RegionInfo parent = loc.getRegion();
-      long rid = 1000;
-      byte[] splitKey = Bytes.toBytes("a");
-      RegionInfo splitA =
-        RegionInfoBuilder.newBuilder(parent.getTable()).setStartKey(parent.getStartKey())
-          .setEndKey(splitKey).setSplit(false).setRegionId(rid).build();
-      RegionInfo splitB = RegionInfoBuilder.newBuilder(parent.getTable()).setStartKey(splitKey)
-        .setEndKey(parent.getEndKey()).setSplit(false).setRegionId(rid).build();
-
-      // find the meta server
-      MiniHBaseCluster cluster = UTIL.getMiniHBaseCluster();
-      int rsIndex = cluster.getServerWithMeta();
-      HRegionServer rs;
-      if (rsIndex >= 0) {
-        rs = cluster.getRegionServer(rsIndex);
-      } else {
-        // it is in master
-        rs = cluster.getMaster();
-      }
-      SpyingRpcScheduler scheduler = (SpyingRpcScheduler) rs.getRpcServer().getScheduler();
-      long prevCalls = scheduler.numPriorityCalls;
-      MetaTableAccessor.splitRegion(connection, parent, -1L, splitA, splitB, loc.getServerName(),
-        1);
-
-      assertTrue(prevCalls < scheduler.numPriorityCalls);
-    }
-  }
-
-  @Test
-  public void testEmptyMetaDaughterLocationDuringSplit() throws IOException {
-    long regionId = System.currentTimeMillis();
-    ServerName serverName0 = ServerName.valueOf("foo", 60010, random.nextLong());
-    RegionInfo parent = RegionInfoBuilder.newBuilder(TableName.valueOf("table_foo"))
-      .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(HConstants.EMPTY_END_ROW).setSplit(false)
-      .setRegionId(regionId).setReplicaId(0).build();
-    RegionInfo splitA = RegionInfoBuilder.newBuilder(TableName.valueOf("table_foo"))
-      .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(Bytes.toBytes("a")).setSplit(false)
-      .setRegionId(regionId + 1).setReplicaId(0).build();
-    RegionInfo splitB = RegionInfoBuilder.newBuilder(TableName.valueOf("table_foo"))
-      .setStartKey(Bytes.toBytes("a")).setEndKey(HConstants.EMPTY_END_ROW).setSplit(false)
-      .setRegionId(regionId + 1).setReplicaId(0).build();
-
-    Table meta = MetaTableAccessor.getMetaHTable(connection);
-    try {
-      List<RegionInfo> regionInfos = Lists.newArrayList(parent);
-      MetaTableAccessor.addRegionsToMeta(connection, regionInfos, 3);
-
-      MetaTableAccessor.splitRegion(connection, parent, -1L, splitA, splitB, serverName0, 3);
-      Get get1 = new Get(splitA.getRegionName());
-      Result resultA = meta.get(get1);
-      Cell serverCellA = resultA.getColumnLatestCell(HConstants.CATALOG_FAMILY,
-        CatalogFamilyFormat.getServerColumn(splitA.getReplicaId()));
-      Cell startCodeCellA = resultA.getColumnLatestCell(HConstants.CATALOG_FAMILY,
-        CatalogFamilyFormat.getStartCodeColumn(splitA.getReplicaId()));
-      assertNull(serverCellA);
-      assertNull(startCodeCellA);
-
-      Get get2 = new Get(splitA.getRegionName());
-      Result resultB = meta.get(get2);
-      Cell serverCellB = resultB.getColumnLatestCell(HConstants.CATALOG_FAMILY,
-        CatalogFamilyFormat.getServerColumn(splitB.getReplicaId()));
-      Cell startCodeCellB = resultB.getColumnLatestCell(HConstants.CATALOG_FAMILY,
-        CatalogFamilyFormat.getStartCodeColumn(splitB.getReplicaId()));
-      assertNull(serverCellB);
-      assertNull(startCodeCellB);
-    } finally {
-      if (meta != null) {
-        meta.close();
-      }
-    }
-  }
-
-  @Test
   public void testScanByRegionEncodedNameExistingRegion() throws Exception {
     final TableName tableName = TableName.valueOf("testScanByRegionEncodedNameExistingRegion");
     UTIL.createTable(tableName, "cf");
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
new file mode 100644
index 0000000..db5d161
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java
@@ -0,0 +1,100 @@
+/**
+ * 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 static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Arrays;
+import org.apache.hadoop.hbase.TestMetaTableAccessor.SpyingRpcScheduler;
+import org.apache.hadoop.hbase.TestMetaTableAccessor.SpyingRpcSchedulerFactory;
+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.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.RSRpcServices;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MiscTests.class, MediumTests.class })
+public class TestMetaUpdatesGoToPriorityQueue {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestMetaUpdatesGoToPriorityQueue.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    // This test has to be end-to-end, and do the verification from the server side
+    UTIL.getConfiguration().set(RSRpcServices.REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS,
+      SpyingRpcSchedulerFactory.class.getName());
+    UTIL.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws IOException, InterruptedException {
+    TableName tableName = TableName.valueOf(getClass().getSimpleName());
+    // create a table and prepare for a manual split
+    UTIL.createTable(tableName, "cf1");
+    UTIL.waitTableAvailable(tableName);
+    RegionInfo parent = UTIL.getAdmin().getRegions(tableName).get(0);
+    long rid = 1000;
+    byte[] splitKey = Bytes.toBytes("a");
+    RegionInfo splitA =
+      RegionInfoBuilder.newBuilder(parent.getTable()).setStartKey(parent.getStartKey())
+        .setEndKey(splitKey).setSplit(false).setRegionId(rid).build();
+    RegionInfo splitB = RegionInfoBuilder.newBuilder(parent.getTable()).setStartKey(splitKey)
+      .setEndKey(parent.getEndKey()).setSplit(false).setRegionId(rid).build();
+
+    // find the meta server
+    MiniHBaseCluster cluster = UTIL.getMiniHBaseCluster();
+    int rsIndex = cluster.getServerWithMeta();
+    HRegionServer rs;
+    if (rsIndex >= 0) {
+      rs = cluster.getRegionServer(rsIndex);
+    } else {
+      // it is in master
+      rs = cluster.getMaster();
+    }
+    SpyingRpcScheduler scheduler = (SpyingRpcScheduler) rs.getRpcServer().getScheduler();
+    long prevCalls = scheduler.numPriorityCalls;
+    long time = System.currentTimeMillis();
+    Put putParent = MetaTableAccessor.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);
+    MetaTableAccessor.multiMutate(UTIL.getConnection(), putParent.getRow(),
+      Arrays.asList(putParent, putA, putB));
+
+    assertTrue(prevCalls < scheduler.numPriorityCalls);
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSimpleMetaSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSimpleMetaSplit.java
new file mode 100644
index 0000000..110309c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestSimpleMetaSplit.java
@@ -0,0 +1,98 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+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.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.MiscTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ MiscTests.class, MediumTests.class })
+public class TestSimpleMetaSplit {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestSimpleMetaSplit.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static byte[] CF = Bytes.toBytes("cf");
+
+  private static byte[] CQ = Bytes.toBytes("cq");
+
+  private static TableDescriptor TD1 = TableDescriptorBuilder.newBuilder(TableName.valueOf("a"))
+    .setColumnFamily(ColumnFamilyDescriptorBuilder.of(CF)).build();
+
+  private static TableDescriptor TD2 = TableDescriptorBuilder.newBuilder(TableName.valueOf("b"))
+    .setColumnFamily(ColumnFamilyDescriptorBuilder.of(CF)).build();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.startMiniCluster(3);
+    UTIL.getAdmin().createTable(TD1);
+    UTIL.getAdmin().createTable(TD2);
+    UTIL.waitTableAvailable(TD1.getTableName());
+    UTIL.waitTableAvailable(TD2.getTableName());
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    UTIL.shutdownMiniCluster();
+  }
+
+  @Test
+  public void test() throws IOException {
+    try (Table table = UTIL.getConnection().getTable(TD1.getTableName())) {
+      table.put(new Put(Bytes.toBytes("row1")).addColumn(CF, CQ, Bytes.toBytes("row1")));
+    }
+    try (Table table = UTIL.getConnection().getTable(TD2.getTableName())) {
+      table.put(new Put(Bytes.toBytes("row2")).addColumn(CF, CQ, Bytes.toBytes("row2")));
+    }
+    // split meta
+    UTIL.getAdmin().split(TableName.META_TABLE_NAME, Bytes.toBytes("b"));
+    // do not count it from client as it will reset the location cache for meta table
+    assertEquals(2, UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager().getRegionStates()
+      .getRegionsOfTable(TableName.META_TABLE_NAME).size());
+    // clear the cache for table 'b'
+    try (RegionLocator locator = UTIL.getConnection().getRegionLocator(TD2.getTableName())) {
+      locator.clearRegionLocationCache();
+    }
+    // make sure that we could get the location of the TD2 from the second meta region
+    try (Table table = UTIL.getConnection().getTable(TD2.getTableName())) {
+      Result result = table.get(new Get(Bytes.toBytes("row2")));
+      assertEquals("row2", Bytes.toString(result.getValue(CF, CQ)));
+    }
+    // assert from client side
+    assertEquals(2, UTIL.getAdmin().getRegions(TableName.META_TABLE_NAME).size());
+  }
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java
index 11e5404..1a3d42b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestShutdownOfMetaReplicaHolder.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import java.io.IOException;
+import java.util.List;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
@@ -45,23 +47,35 @@ public class TestShutdownOfMetaReplicaHolder extends MetaWithReplicasTestBase {
     startCluster();
   }
 
+  private HRegionLocation getLoc(RegionLocator locator, int replica)
+    throws IOException, InterruptedException {
+    // we have backup master in this test so we may get stale meta replicas since the cache is
+    // refreshed asynchronously, so add retries here.
+    for (;;) {
+      List<HRegionLocation> locs = locator.getRegionLocations(HConstants.EMPTY_START_ROW, true);
+      if (locs.size() > replica) {
+        return locs.get(1);
+      }
+      Thread.sleep(1000);
+    }
+  }
+
   @Test
   public void testShutdownOfReplicaHolder() throws Exception {
     // checks that the when the server holding meta replica is shut down, the meta replica
     // can be recovered
     try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
       RegionLocator locator = conn.getRegionLocator(TableName.META_TABLE_NAME)) {
-      HRegionLocation hrl = locator.getRegionLocations(HConstants.EMPTY_START_ROW, true).get(1);
+      HRegionLocation hrl = getLoc(locator, 1);
       ServerName oldServer = hrl.getServerName();
       TEST_UTIL.getHBaseClusterInterface().killRegionServer(oldServer);
       LOG.debug("Waiting for the replica {} to come up", hrl.getRegion());
       TEST_UTIL.waitFor(30000, () -> {
-        HRegionLocation loc = locator.getRegionLocations(HConstants.EMPTY_START_ROW, true).get(1);
+        HRegionLocation loc = getLoc(locator, 1);
         return loc != null && !loc.getServerName().equals(oldServer);
       });
       LOG.debug("Replica {} is online on {}, old server is {}", hrl.getRegion(),
-        locator.getRegionLocations(HConstants.EMPTY_START_ROW, true).get(1).getServerName(),
-        oldServer);
+        getLoc(locator, 1).getServerName(), oldServer);
     }
   }
 }
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 fea362f..078d96b 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,21 +17,32 @@
  */
 package org.apache.hadoop.hbase.master.assignment;
 
+import static org.apache.hadoop.hbase.TestMetaTableAccessor.assertEmptyMetaLocation;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
 import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.hadoop.hbase.CatalogFamilyFormat;
+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.ServerName;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNameTestRule;
+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.Result;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
@@ -41,11 +52,11 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
 @Category({ MasterTests.class, MediumTests.class })
 public class TestRegionStateStore {
@@ -54,10 +65,11 @@ public class TestRegionStateStore {
   public static final HBaseClassTestRule CLASS_RULE =
     HBaseClassTestRule.forClass(TestRegionStateStore.class);
 
-  private static final Logger LOG = LoggerFactory.getLogger(TestRegionStateStore.class);
-
   private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
 
+  @Rule
+  public final TableNameTestRule name = new TableNameTestRule();
+
   @BeforeClass
   public static void beforeClass() throws Exception {
     UTIL.startMiniCluster();
@@ -70,12 +82,12 @@ public class TestRegionStateStore {
 
   @Test
   public void testVisitMetaForRegionExistingRegion() throws Exception {
-    final TableName tableName = TableName.valueOf("testVisitMetaForRegion");
+    final TableName tableName = name.getTableName();
     UTIL.createTable(tableName, "cf");
     final List<HRegion> regions = UTIL.getHBaseCluster().getRegions(tableName);
     final String encodedName = regions.get(0).getRegionInfo().getEncodedName();
-    final RegionStateStore regionStateStore = UTIL.getHBaseCluster().getMaster().
-      getAssignmentManager().getRegionStateStore();
+    final RegionStateStore regionStateStore =
+      UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStateStore();
     final AtomicBoolean visitorCalled = new AtomicBoolean(false);
     regionStateStore.visitMetaForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
       @Override
@@ -90,18 +102,18 @@ public class TestRegionStateStore {
 
   @Test
   public void testVisitMetaForBadRegionState() throws Exception {
-    final TableName tableName = TableName.valueOf("testVisitMetaForBadRegionState");
+    final TableName tableName = name.getTableName();
     UTIL.createTable(tableName, "cf");
     final List<HRegion> regions = UTIL.getHBaseCluster().getRegions(tableName);
     final String encodedName = regions.get(0).getRegionInfo().getEncodedName();
-    final RegionStateStore regionStateStore = UTIL.getHBaseCluster().getMaster().
-        getAssignmentManager().getRegionStateStore();
+    final RegionStateStore regionStateStore =
+      UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStateStore();
 
     // add the BAD_STATE which does not exist in enum RegionState.State
-    Put put = new Put(regions.get(0).getRegionInfo().getRegionName(),
-        EnvironmentEdgeManager.currentTime());
+    Put put =
+      new Put(regions.get(0).getRegionInfo().getRegionName(), EnvironmentEdgeManager.currentTime());
     put.addColumn(HConstants.CATALOG_FAMILY, HConstants.STATE_QUALIFIER,
-        Bytes.toBytes("BAD_STATE"));
+      Bytes.toBytes("BAD_STATE"));
 
     try (Table table = UTIL.getConnection().getTable(TableName.META_TABLE_NAME)) {
       table.put(put);
@@ -110,9 +122,8 @@ public class TestRegionStateStore {
     final AtomicBoolean visitorCalled = new AtomicBoolean(false);
     regionStateStore.visitMetaForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
       @Override
-      public void visitRegionState(Result result, RegionInfo regionInfo,
-                                   RegionState.State state, ServerName regionLocation,
-                                   ServerName lastHost, long openSeqNum) {
+      public void visitRegionState(Result result, RegionInfo regionInfo, RegionState.State state,
+        ServerName regionLocation, ServerName lastHost, long openSeqNum) {
         assertEquals(encodedName, regionInfo.getEncodedName());
         assertNull(state);
         visitorCalled.set(true);
@@ -124,8 +135,8 @@ public class TestRegionStateStore {
   @Test
   public void testVisitMetaForRegionNonExistingRegion() throws Exception {
     final String encodedName = "fakeencodedregionname";
-    final RegionStateStore regionStateStore = UTIL.getHBaseCluster().getMaster().
-      getAssignmentManager().getRegionStateStore();
+    final RegionStateStore regionStateStore =
+      UTIL.getHBaseCluster().getMaster().getAssignmentManager().getRegionStateStore();
     final AtomicBoolean visitorCalled = new AtomicBoolean(false);
     regionStateStore.visitMetaForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
       @Override
@@ -136,4 +147,76 @@ public class TestRegionStateStore {
     });
     assertFalse("Visitor has been called, but it shouldn't.", visitorCalled.get());
   }
+
+  @Test
+  public void testMetaLocationForRegionReplicasIsAddedAtRegionSplit() throws IOException {
+    long regionId = System.currentTimeMillis();
+    ServerName serverName0 =
+      ServerName.valueOf("foo", 60010, ThreadLocalRandom.current().nextLong());
+    TableName tableName = name.getTableName();
+    RegionInfo parent = RegionInfoBuilder.newBuilder(tableName)
+      .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(HConstants.EMPTY_END_ROW).setSplit(false)
+      .setRegionId(regionId).setReplicaId(0).build();
+
+    RegionInfo splitA = RegionInfoBuilder.newBuilder(tableName)
+      .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(Bytes.toBytes("a")).setSplit(false)
+      .setRegionId(regionId + 1).setReplicaId(0).build();
+    RegionInfo splitB = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("a"))
+      .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);
+    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())) {
+      assertEmptyMetaLocation(meta, splitA.getRegionName(), 1);
+      assertEmptyMetaLocation(meta, splitA.getRegionName(), 2);
+      assertEmptyMetaLocation(meta, splitB.getRegionName(), 1);
+      assertEmptyMetaLocation(meta, splitB.getRegionName(), 2);
+    }
+  }
+
+  @Test
+  public void testEmptyMetaDaughterLocationDuringSplit() throws IOException {
+    TableName tableName = name.getTableName();
+    long regionId = System.currentTimeMillis();
+    ServerName serverName0 =
+      ServerName.valueOf("foo", 60010, ThreadLocalRandom.current().nextLong());
+    RegionInfo parent = RegionInfoBuilder.newBuilder(tableName)
+      .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(HConstants.EMPTY_END_ROW).setSplit(false)
+      .setRegionId(regionId).setReplicaId(0).build();
+    RegionInfo splitA = RegionInfoBuilder.newBuilder(tableName)
+      .setStartKey(HConstants.EMPTY_START_ROW).setEndKey(Bytes.toBytes("a")).setSplit(false)
+      .setRegionId(regionId + 1).setReplicaId(0).build();
+    RegionInfo splitB = RegionInfoBuilder.newBuilder(tableName).setStartKey(Bytes.toBytes("a"))
+      .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);
+    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())) {
+      Get get1 = new Get(splitA.getRegionName());
+      Result resultA = meta.get(get1);
+      Cell serverCellA = resultA.getColumnLatestCell(HConstants.CATALOG_FAMILY,
+        CatalogFamilyFormat.getServerColumn(splitA.getReplicaId()));
+      Cell startCodeCellA = resultA.getColumnLatestCell(HConstants.CATALOG_FAMILY,
+        CatalogFamilyFormat.getStartCodeColumn(splitA.getReplicaId()));
+      assertNull(serverCellA);
+      assertNull(startCodeCellA);
+
+      Get get2 = new Get(splitA.getRegionName());
+      Result resultB = meta.get(get2);
+      Cell serverCellB = resultB.getColumnLatestCell(HConstants.CATALOG_FAMILY,
+        CatalogFamilyFormat.getServerColumn(splitB.getReplicaId()));
+      Cell startCodeCellB = resultB.getColumnLatestCell(HConstants.CATALOG_FAMILY,
+        CatalogFamilyFormat.getStartCodeColumn(splitB.getReplicaId()));
+      assertNull(serverCellB);
+      assertNull(startCodeCellB);
+    }
+  }
 }