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 2021/07/22 15:48:40 UTC

[hbase] 05/09: 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

commit cf662042d1f6bf4354dddcb45ec56e8e58e65b3a
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>
---
 .../AsyncScanSingleRegionRpcRetryingCaller.java    | 47 +++++++++--
 .../hadoop/hbase/client/ConnectionUtils.java       | 26 ------
 .../hbase/client/TableRegionLocationCache.java     |  2 +-
 .../hbase/master/assignment/AssignmentManager.java |  3 +-
 .../hbase/master/assignment/RegionStateStore.java  | 52 +++++++-----
 .../master/procedure/ModifyTableProcedure.java     | 16 +++-
 .../DelimitedKeyPrefixRegionSplitPolicy.java       |  2 +-
 .../apache/hadoop/hbase/regionserver/HRegion.java  |  5 --
 .../apache/hadoop/hbase/regionserver/HStore.java   |  2 -
 .../hbase/regionserver/MetaRegionSplitPolicy.java  | 40 +++++++++
 .../hbase/regionserver/RegionSplitPolicy.java      |  3 +-
 .../hadoop/hbase/util/FSTableDescriptors.java      |  8 +-
 .../apache/hadoop/hbase/TestMetaSplitBySize.java   | 62 ++++++++++++++
 .../hbase/TestMetaUpdatesGoToPriorityQueue.java    |  2 +-
 .../apache/hadoop/hbase/TestSimpleMetaSplit.java   | 98 ++++++++++++++++++++++
 .../client/TestShutdownOfMetaReplicaHolder.java    | 22 ++++-
 .../master/assignment/TestRegionStateStore.java    | 28 +++----
 17 files changed, 323 insertions(+), 95 deletions(-)

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 1433c55..4697153 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
@@ -280,32 +280,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 ed7cb90..4d0be07 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
@@ -110,7 +110,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 1786afe..a625fb3 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
@@ -1911,9 +1911,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
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 cce7a81..c290c5e 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
@@ -271,30 +271,38 @@ public class RegionStateStore {
    */
   private void multiMutate(RegionInfo ri, List<Mutation> mutations) throws IOException {
     debugLogMutations(mutations);
-    byte[] row =
-      Bytes.toBytes(RegionReplicaUtil.getRegionInfoForDefaultReplica(ri).getRegionNameAsString() +
-        HConstants.DELIMITER);
-    MutateRowsRequest.Builder builder = MutateRowsRequest.newBuilder();
-    for (Mutation mutation : mutations) {
-      if (mutation instanceof Put) {
-        builder.addMutationRequest(
-          ProtobufUtil.toMutation(ClientProtos.MutationProto.MutationType.PUT, mutation));
-      } else if (mutation instanceof Delete) {
-        builder.addMutationRequest(
-          ProtobufUtil.toMutation(ClientProtos.MutationProto.MutationType.DELETE, mutation));
-      } else {
-        throw new DoNotRetryIOException(
-          "multi in MetaEditor doesn't support " + mutation.getClass().getName());
+    if (ri.isMetaRegion()) {
+      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[] row =
+        Bytes.toBytes(RegionReplicaUtil.getRegionInfoForDefaultReplica(ri).getRegionNameAsString() +
+          HConstants.DELIMITER);
+      MutateRowsRequest.Builder builder = MutateRowsRequest.newBuilder();
+      for (Mutation mutation : mutations) {
+        if (mutation instanceof Put) {
+          builder.addMutationRequest(
+            ProtobufUtil.toMutation(ClientProtos.MutationProto.MutationType.PUT, mutation));
+        } else if (mutation instanceof Delete) {
+          builder.addMutationRequest(
+            ProtobufUtil.toMutation(ClientProtos.MutationProto.MutationType.DELETE, mutation));
+        } else {
+          throw new DoNotRetryIOException(
+            "multi in MetaEditor doesn't support " + mutation.getClass().getName());
+        }
       }
+      MutateRowsRequest request = builder.build();
+      AsyncTable<?> table =
+        master.getConnection().toAsyncConnection().getTable(TableName.META_TABLE_NAME);
+      CompletableFuture<MutateRowsResponse> future =
+        table.<MultiRowMutationService, MutateRowsResponse> coprocessorService(
+          MultiRowMutationService::newStub,
+          (stub, controller, done) -> stub.mutateRows(controller, request, done), row);
+      FutureUtils.get(future);
     }
-    MutateRowsRequest request = builder.build();
-    AsyncTable<?> table =
-      master.getConnection().toAsyncConnection().getTable(TableName.META_TABLE_NAME);
-    CompletableFuture<MutateRowsResponse> future =
-      table.<MultiRowMutationService, MutateRowsResponse> coprocessorService(
-        MultiRowMutationService::newStub,
-        (stub, controller, done) -> stub.mutateRows(controller, request, done), row);
-    FutureUtils.get(future);
   }
 
   private Table getMetaTable() throws IOException {
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 247dd9c..23aa028 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
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
 import org.apache.hadoop.hbase.master.zksyncer.MetaLocationSyncer;
 import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.regionserver.MetaRegionSplitPolicy;
 import org.apache.hadoop.hbase.replication.ReplicationException;
 import org.apache.hadoop.hbase.rsgroup.RSGroupInfo;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -101,17 +102,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 241c062..a1fd3f4 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
@@ -49,7 +49,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
   public String toString() {
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 8c4660c..ef1e7ae 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
@@ -7912,11 +7912,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 9b482b6..5cc1844 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
@@ -2010,8 +2010,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 91de8b0..f64aaf5 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;
@@ -194,9 +195,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());
   }
 
   protected boolean isUsecache() {
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..8775247
--- /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 HBaseTestingUtil UTIL = new HBaseTestingUtil();
+
+  @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/TestMetaUpdatesGoToPriorityQueue.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaUpdatesGoToPriorityQueue.java
index cd04157..8396e5a 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
@@ -132,4 +132,4 @@ public class TestMetaUpdatesGoToPriorityQueue {
 
     assertTrue(prevCalls < scheduler.numPriorityCalls);
   }
-}
\ No newline at end of file
+}
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..6d50f71
--- /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 HBaseTestingUtil UTIL = new HBaseTestingUtil();
+
+  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 83e5431..1d2b346 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
@@ -64,7 +64,6 @@ import org.junit.experimental.categories.Category;
 
 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
 
-
 @Category({ MasterTests.class, MediumTests.class })
 public class TestRegionStateStore {
 
@@ -89,12 +88,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
@@ -109,18 +108,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);
@@ -129,9 +128,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);
@@ -143,8 +141,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