You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by al...@apache.org on 2019/01/23 13:42:20 UTC

[hbase] branch branch-2.0 updated: HBASE-21754 ReportRegionStateTransitionRequest should be executed in priority executor

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

allan163 pushed a commit to branch branch-2.0
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.0 by this push:
     new bdc0c82  HBASE-21754 ReportRegionStateTransitionRequest should be executed in priority executor
bdc0c82 is described below

commit bdc0c828376bc2231372b7d3e573053152aae39b
Author: Allan Yang <al...@apache.org>
AuthorDate: Wed Jan 23 21:41:40 2019 +0800

    HBASE-21754 ReportRegionStateTransitionRequest should be executed in priority executor
---
 .../java/org/apache/hadoop/hbase/HConstants.java   |   9 +-
 .../hadoop/hbase/ipc/MetricsHBaseServerSource.java |   2 +
 .../hbase/ipc/MetricsHBaseServerWrapper.java       |   2 +
 .../hbase/ipc/MetricsHBaseServerSourceImpl.java    |   2 +
 .../apache/hadoop/hbase/ipc/FifoRpcScheduler.java  |   5 +
 .../hbase/ipc/MetricsHBaseServerWrapperImpl.java   |   9 +-
 .../org/apache/hadoop/hbase/ipc/RpcScheduler.java  |   3 +
 .../hadoop/hbase/ipc/SimpleRpcScheduler.java       |  78 ++++++++++++----
 .../apache/hadoop/hbase/ipc/SimpleRpcServer.java   |   3 +-
 .../MasterAnnotationReadingPriorityFunction.java   |   9 +-
 .../regionserver/SimpleRpcSchedulerFactory.java    |   2 +
 .../hadoop/hbase/ipc/DelegatingRpcScheduler.java   |   5 +
 .../hbase/ipc/MetricsHBaseServerWrapperStub.java   |   6 ++
 .../hadoop/hbase/ipc/TestRpcHandlerException.java  |   2 +-
 .../TestMasterHandlerFullWhenTransitRegion.java    | 101 +++++++++++++++++++++
 .../hadoop/hbase/master/TestMasterQosFunction.java |   4 +-
 16 files changed, 214 insertions(+), 28 deletions(-)

diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index cbff7d8..c62bbdf 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -1035,6 +1035,11 @@ public final class HConstants {
   public static final String REGION_SERVER_REPLICATION_HANDLER_COUNT =
       "hbase.regionserver.replication.handler.count";
   public static final int DEFAULT_REGION_SERVER_REPLICATION_HANDLER_COUNT = 3;
+  // Meta Transition handlers to deal with meta ReportRegionStateTransitionRequest. Meta transition
+  // should be dealt with in a separate handler in case blocking other region's transition.
+  public static final String MASTER_META_TRANSITION_HANDLER_COUNT =
+      "hbase.master.meta.transition.handler.count";
+  public static final int MASTER__META_TRANSITION_HANDLER_COUNT_DEFAULT = 1;
 
   @Deprecated // unused. see HBASE-10569. remove this in 3.0
   public static final String MASTER_HANDLER_COUNT = "hbase.master.handler.count";
@@ -1102,7 +1107,7 @@ public final class HConstants {
    * by different set of handlers. For example, HIGH_QOS tagged methods are
    * handled by high priority handlers.
    */
-  // normal_QOS < replication_QOS < replay_QOS < QOS_threshold < admin_QOS < high_QOS
+  // normal_QOS < replication_QOS < replay_QOS < QOS_threshold < admin_QOS < high_QOS < meta_QOS
   public static final int PRIORITY_UNSET = -1;
   public static final int NORMAL_QOS = 0;
   public static final int REPLICATION_QOS = 5;
@@ -1111,6 +1116,8 @@ public final class HConstants {
   public static final int ADMIN_QOS = 100;
   public static final int HIGH_QOS = 200;
   public static final int SYSTEMTABLE_QOS = HIGH_QOS;
+  public static final int META_QOS = 300;
+
 
   /** Directory under /hbase where archived hfiles are stored */
   public static final String HFILE_ARCHIVE_DIRECTORY = "archive";
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
index 0833751..b87f3cc 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSource.java
@@ -60,10 +60,12 @@ public interface MetricsHBaseServerSource extends ExceptionTrackingSource {
   String GENERAL_QUEUE_DESC = "Number of calls in the general call queue; " +
     "parsed requests waiting in scheduler to be executed";
   String PRIORITY_QUEUE_NAME = "numCallsInPriorityQueue";
+  String METAPRIORITY_QUEUE_NAME = "numCallsInMetaPriorityQueue";
   String REPLICATION_QUEUE_NAME = "numCallsInReplicationQueue";
   String REPLICATION_QUEUE_DESC =
       "Number of calls in the replication call queue waiting to be run";
   String PRIORITY_QUEUE_DESC = "Number of calls in the priority call queue waiting to be run";
+  String METAPRIORITY_QUEUE_DESC = "Number of calls in the priority call queue waiting to be run";
   String WRITE_QUEUE_NAME = "numCallsInWriteQueue";
   String WRITE_QUEUE_DESC = "Number of calls in the write call queue; " +
     "parsed requests waiting in scheduler to be executed";
diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapper.java
index c80d1a9..a9e7b35 100644
--- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapper.java
+++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapper.java
@@ -31,6 +31,8 @@ public interface MetricsHBaseServerWrapper {
 
   int getPriorityQueueLength();
 
+  int getMetaPriorityQueueLength();
+
   int getNumOpenConnections();
 
   int getActiveRpcHandlerCount();
diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
index a63f287..e2e2445 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerSourceImpl.java
@@ -152,6 +152,8 @@ public class MetricsHBaseServerSourceImpl extends ExceptionTrackingSourceImpl
               REPLICATION_QUEUE_DESC), wrapper.getReplicationQueueLength())
           .addGauge(Interns.info(PRIORITY_QUEUE_NAME, PRIORITY_QUEUE_DESC),
               wrapper.getPriorityQueueLength())
+          .addGauge(Interns.info(METAPRIORITY_QUEUE_NAME, METAPRIORITY_QUEUE_DESC),
+              wrapper.getMetaPriorityQueueLength())
           .addGauge(Interns.info(NUM_OPEN_CONNECTIONS_NAME,
               NUM_OPEN_CONNECTIONS_DESC), wrapper.getNumOpenConnections())
           .addGauge(Interns.info(NUM_ACTIVE_HANDLER_NAME,
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java
index ce0f86d..5cad012 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java
@@ -174,6 +174,11 @@ public class FifoRpcScheduler extends RpcScheduler {
   }
 
   @Override
+  public int getMetaPriorityQueueLength() {
+    return 0;
+  }
+
+  @Override
   public CallQueueInfo getCallQueueInfo() {
     String queueName = "Fifo Queue";
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperImpl.java
index 8753ebb..eec8b40 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperImpl.java
@@ -67,6 +67,14 @@ public class MetricsHBaseServerWrapperImpl implements MetricsHBaseServerWrapper
   }
 
   @Override
+  public int getMetaPriorityQueueLength() {
+    if (!isServerStarted() || this.server.getScheduler() == null) {
+      return 0;
+    }
+    return server.getScheduler().getMetaPriorityQueueLength();
+  }
+
+  @Override
   public int getNumOpenConnections() {
     if (!isServerStarted()) {
       return 0;
@@ -81,7 +89,6 @@ public class MetricsHBaseServerWrapperImpl implements MetricsHBaseServerWrapper
     }
     return server.getScheduler().getActiveRpcHandlerCount();
   }
-
   @Override
   public long getNumGeneralCallsDropped() {
     if (!isServerStarted() || this.server.getScheduler() == null) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcScheduler.java
index bd088a7..7f04108 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcScheduler.java
@@ -74,6 +74,9 @@ public abstract class RpcScheduler {
   /** Retrieves length of the priority queue for metrics. */
   public abstract int getPriorityQueueLength();
 
+  /** Retrieves length of the meta priority queue for metrics. */
+  public abstract int getMetaPriorityQueueLength();
+
   /** Retrieves length of the replication queue for metrics. */
   public abstract int getReplicationQueueLength();
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
index 817163c..2335ed9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java
@@ -41,6 +41,11 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
   private final RpcExecutor priorityExecutor;
   private final RpcExecutor replicationExecutor;
 
+  /**
+   * This executor is only for meta transition
+   */
+  private final RpcExecutor metaTransitionExecutor;
+
   /** What level a high priority call is at. */
   private final int highPriorityLevel;
 
@@ -59,6 +64,7 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
       int handlerCount,
       int priorityHandlerCount,
       int replicationHandlerCount,
+      int metaTransitionHandler,
       PriorityFunction priority,
       Abortable server,
       int highPriorityLevel) {
@@ -97,18 +103,18 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
     this.replicationExecutor = replicationHandlerCount > 0 ? new FastPathBalancedQueueRpcExecutor(
         "replication.FPBQ", replicationHandlerCount, RpcExecutor.CALL_QUEUE_TYPE_FIFO_CONF_VALUE,
         maxQueueLength, priority, conf, abortable) : null;
-  }
 
+    this.metaTransitionExecutor = metaTransitionHandler > 0 ?
+        new FastPathBalancedQueueRpcExecutor("metaPriority.FPBQ", metaTransitionHandler,
+            RpcExecutor.CALL_QUEUE_TYPE_FIFO_CONF_VALUE, maxPriorityQueueLength, priority, conf,
+            abortable) :
+        null;
+  }
 
-  public SimpleRpcScheduler(
-	      Configuration conf,
-	      int handlerCount,
-	      int priorityHandlerCount,
-	      int replicationHandlerCount,
-	      PriorityFunction priority,
-	      int highPriorityLevel) {
-	  this(conf, handlerCount, priorityHandlerCount, replicationHandlerCount, priority,
-	    null, highPriorityLevel);
+  public SimpleRpcScheduler(Configuration conf, int handlerCount, int priorityHandlerCount,
+      int replicationHandlerCount, PriorityFunction priority, int highPriorityLevel) {
+    this(conf, handlerCount, priorityHandlerCount, replicationHandlerCount, 0, priority, null,
+        highPriorityLevel);
   }
 
   /**
@@ -124,6 +130,9 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
     if (replicationExecutor != null) {
       replicationExecutor.resizeQueues(conf);
     }
+    if (metaTransitionExecutor != null) {
+      metaTransitionExecutor.resizeQueues(conf);
+    }
 
     String callQueueType = conf.get(RpcExecutor.CALL_QUEUE_TYPE_CONF_KEY,
       RpcExecutor.CALL_QUEUE_TYPE_CONF_DEFAULT);
@@ -140,15 +149,31 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
   @Override
   public void start() {
     callExecutor.start(port);
-    if (priorityExecutor != null) priorityExecutor.start(port);
-    if (replicationExecutor != null) replicationExecutor.start(port);
+    if (priorityExecutor != null) {
+      priorityExecutor.start(port);
+    }
+    if (replicationExecutor != null) {
+      replicationExecutor.start(port);
+    }
+    if (metaTransitionExecutor != null) {
+      metaTransitionExecutor.start(port);
+    }
+
   }
 
   @Override
   public void stop() {
     callExecutor.stop();
-    if (priorityExecutor != null) priorityExecutor.stop();
-    if (replicationExecutor != null) replicationExecutor.stop();
+    if (priorityExecutor != null) {
+      priorityExecutor.stop();
+    }
+    if (replicationExecutor != null) {
+      replicationExecutor.stop();
+    }
+    if (metaTransitionExecutor != null) {
+      metaTransitionExecutor.stop();
+    }
+
   }
 
   @Override
@@ -159,7 +184,9 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
     if (level == HConstants.PRIORITY_UNSET) {
       level = HConstants.NORMAL_QOS;
     }
-    if (priorityExecutor != null && level > highPriorityLevel) {
+    if (metaTransitionExecutor != null && level == HConstants.META_QOS) {
+      return metaTransitionExecutor.dispatch(callTask);
+    } else if (priorityExecutor != null && level > highPriorityLevel) {
       return priorityExecutor.dispatch(callTask);
     } else if (replicationExecutor != null && level == HConstants.REPLICATION_QOS) {
       return replicationExecutor.dispatch(callTask);
@@ -169,6 +196,11 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
   }
 
   @Override
+  public int getMetaPriorityQueueLength() {
+    return metaTransitionExecutor == null ? 0 : metaTransitionExecutor.getQueueLength();
+  }
+
+  @Override
   public int getGeneralQueueLength() {
     return callExecutor.getQueueLength();
   }
@@ -187,7 +219,8 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
   public int getActiveRpcHandlerCount() {
     return callExecutor.getActiveHandlerCount() +
            (priorityExecutor == null ? 0 : priorityExecutor.getActiveHandlerCount()) +
-           (replicationExecutor == null ? 0 : replicationExecutor.getActiveHandlerCount());
+           (replicationExecutor == null ? 0 : replicationExecutor.getActiveHandlerCount()) +
+           (metaTransitionExecutor == null ? 0 : metaTransitionExecutor.getActiveHandlerCount());
   }
 
   @Override
@@ -236,24 +269,31 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
 
     CallQueueInfo callQueueInfo = new CallQueueInfo();
 
-    if(null!=callExecutor) {
+    if (null != callExecutor) {
       queueName = "Call Queue";
       callQueueInfo.setCallMethodCount(queueName, callExecutor.getCallQueueCountsSummary());
       callQueueInfo.setCallMethodSize(queueName, callExecutor.getCallQueueSizeSummary());
     }
 
-    if(null!=priorityExecutor) {
+    if (null != priorityExecutor) {
       queueName = "Priority Queue";
       callQueueInfo.setCallMethodCount(queueName, priorityExecutor.getCallQueueCountsSummary());
       callQueueInfo.setCallMethodSize(queueName, priorityExecutor.getCallQueueSizeSummary());
     }
 
-    if(null!=replicationExecutor) {
+    if (null != replicationExecutor) {
       queueName = "Replication Queue";
       callQueueInfo.setCallMethodCount(queueName, replicationExecutor.getCallQueueCountsSummary());
       callQueueInfo.setCallMethodSize(queueName, replicationExecutor.getCallQueueSizeSummary());
     }
 
+    if (null != metaTransitionExecutor) {
+      queueName = "Meta Transition Queue";
+      callQueueInfo.setCallMethodCount(queueName,
+          metaTransitionExecutor.getCallQueueCountsSummary());
+      callQueueInfo.setCallMethodSize(queueName, metaTransitionExecutor.getCallQueueSizeSummary());
+    }
+
     return callQueueInfo;
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServer.java
index 13a3cf7..2a8cfbe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServer.java
@@ -613,7 +613,8 @@ public class SimpleRpcServer extends RpcServer {
             "; connections=" + size() +
             ", queued calls size (bytes)=" + callQueueSizeInBytes.sum() +
             ", general queued calls=" + scheduler.getGeneralQueueLength() +
-            ", priority queued calls=" + scheduler.getPriorityQueueLength());
+            ", priority queued calls=" + scheduler.getPriorityQueueLength() +
+            ", meta priority queued calls=" + scheduler.getMetaPriorityQueueLength());
       }
       return connection;
     }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterAnnotationReadingPriorityFunction.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterAnnotationReadingPriorityFunction.java
index 61eb28f..e99b7fb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterAnnotationReadingPriorityFunction.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterAnnotationReadingPriorityFunction.java
@@ -43,6 +43,9 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
  * processing of the request to online meta. To accomplish this this priority function makes sure
  * that all requests to transition meta are handled in different threads from other report region
  * in transition calls.
+ * After HBASE-21754, ReportRegionStateTransitionRequest for meta region will be assigned a META_QOS
+ * , a separate executor called metaTransitionExecutor will execute it. Other transition request
+ * will be executed in priorityExecutor to prevent being mixed with normal requests
  */
 @InterfaceAudience.Private
 public class MasterAnnotationReadingPriorityFunction extends AnnotationReadingPriorityFunction {
@@ -78,13 +81,13 @@ public class MasterAnnotationReadingPriorityFunction extends AnnotationReadingPr
         if (rst.getRegionInfoList() != null) {
           for (HBaseProtos.RegionInfo info : rst.getRegionInfoList()) {
             TableName tn = ProtobufUtil.toTableName(info.getTableName());
-            if (tn.isSystemTable()) {
-              return HConstants.SYSTEMTABLE_QOS;
+            if (TableName.META_TABLE_NAME.equals(tn)) {
+              return HConstants.META_QOS;
             }
           }
         }
       }
-      return HConstants.NORMAL_QOS;
+      return HConstants.HIGH_QOS;
     }
 
     // Handle the rest of the different reasons to change priority.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SimpleRpcSchedulerFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SimpleRpcSchedulerFactory.java
index 673e785..67aad26 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SimpleRpcSchedulerFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SimpleRpcSchedulerFactory.java
@@ -49,6 +49,8 @@ public class SimpleRpcSchedulerFactory implements RpcSchedulerFactory {
         HConstants.DEFAULT_REGION_SERVER_HIGH_PRIORITY_HANDLER_COUNT),
       conf.getInt(HConstants.REGION_SERVER_REPLICATION_HANDLER_COUNT,
           HConstants.DEFAULT_REGION_SERVER_REPLICATION_HANDLER_COUNT),
+        conf.getInt(HConstants.MASTER_META_TRANSITION_HANDLER_COUNT,
+            HConstants.MASTER__META_TRANSITION_HANDLER_COUNT_DEFAULT),
       priority,
       server,
       HConstants.QOS_THRESHOLD);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java
index 3dfae82..b48a26f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/DelegatingRpcScheduler.java
@@ -65,6 +65,11 @@ public class DelegatingRpcScheduler extends RpcScheduler {
   }
 
   @Override
+  public int getMetaPriorityQueueLength() {
+    return delegate.getMetaPriorityQueueLength();
+  }
+
+  @Override
   public long getNumGeneralCallsDropped() {
     return delegate.getNumGeneralCallsDropped();
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperStub.java
index 3b787a9..8166ab9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperStub.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/MetricsHBaseServerWrapperStub.java
@@ -88,4 +88,10 @@ public class MetricsHBaseServerWrapperStub implements MetricsHBaseServerWrapper{
   public int getActiveScanRpcHandlerCount() {
     return 6;
   }
+
+  @Override
+  public int getMetaPriorityQueueLength() {
+    return 1;
+  }
+
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java
index cddb647..88f2434 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcHandlerException.java
@@ -95,7 +95,7 @@ public class TestRpcHandlerException {
     PriorityFunction qosFunction = mock(PriorityFunction.class);
     Abortable abortable = new AbortServer();
     CONF.set(RpcServerFactory.CUSTOM_RPC_SERVER_IMPL_CONF_KEY, rpcServerImpl);
-    RpcScheduler scheduler = new SimpleRpcScheduler(CONF, 2, 0, 0, qosFunction, abortable, 0);
+    RpcScheduler scheduler = new SimpleRpcScheduler(CONF, 2, 0, 0, 0, qosFunction, abortable, 0);
     RpcServer rpcServer = RpcServerFactory.createRpcServer(null, "testRpcServer",
         Lists.newArrayList(new BlockingServiceAndInterface((BlockingService) SERVICE, null)),
         new InetSocketAddress("localhost", 0), CONF, scheduler);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterHandlerFullWhenTransitRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterHandlerFullWhenTransitRegion.java
new file mode 100644
index 0000000..049c560
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterHandlerFullWhenTransitRegion.java
@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master;
+
+import java.io.IOException;
+import java.util.Optional;
+
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ MasterTests.class, LargeTests.class })
+public class TestMasterHandlerFullWhenTransitRegion {
+
+  private static Logger LOG = LoggerFactory
+      .getLogger(TestMasterHandlerFullWhenTransitRegion.class.getName());
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+      HBaseClassTestRule.forClass(TestMasterHandlerFullWhenTransitRegion.class);
+
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static final String TABLENAME = "table";
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    UTIL.getConfiguration().setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
+        DelayOpenCP.class.getName());
+    //set handler number to 1.
+    UTIL.getConfiguration().setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 1);
+    UTIL.startMiniCluster(2);
+    UTIL.createTable(TableName.valueOf(TABLENAME), "fa");
+  }
+
+  @Test(timeout = 30000)
+  public void test() throws Exception {
+    RegionInfo regionInfo = UTIL.getAdmin().getRegions(TableName.valueOf(TABLENAME)).get(0);
+    //See HBASE-21754
+    //There is Only one handler, if ReportRegionStateTransitionRequest executes in the same kind
+    // of thread with moveRegion, it will lock each other. Making the move operation can not finish.
+    UTIL.getAdmin().move(regionInfo.getEncodedNameAsBytes(), null);
+    LOG.info("Region move complete");
+  }
+
+
+  /**
+   * Make open region very slow
+   */
+  public static class DelayOpenCP implements RegionCoprocessor, RegionObserver {
+
+    @Override
+    public void preOpen(ObserverContext<RegionCoprocessorEnvironment> c) throws IOException {
+      try {
+        if (!c.getEnvironment().getRegion().getRegionInfo().getTable().isSystemTable()) {
+          LOG.info("begin to sleep");
+          Thread.sleep(10000);
+          LOG.info("finish sleep");
+        }
+      } catch (Throwable t) {
+
+      }
+    }
+
+    @Override
+    public Optional<RegionObserver> getRegionObserver() {
+      return Optional.of(this);
+    }
+  }
+
+}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterQosFunction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterQosFunction.java
index 5280ed6..f82ece1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterQosFunction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterQosFunction.java
@@ -96,9 +96,9 @@ public class TestMasterQosFunction extends QosTestHelper {
             .addTransition(normalTransition).build();
 
     final String reportFuncName = "ReportRegionStateTransition";
-    checkMethod(conf, reportFuncName, HConstants.SYSTEMTABLE_QOS, qosFunction,
+    checkMethod(conf, reportFuncName, HConstants.META_QOS, qosFunction,
         metaTransitionRequest);
-    checkMethod(conf, reportFuncName, HConstants.NORMAL_QOS, qosFunction, normalTransitionRequest);
+    checkMethod(conf, reportFuncName, HConstants.HIGH_QOS, qosFunction, normalTransitionRequest);
   }
 
   @Test