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

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


The following commit(s) were added to refs/heads/branch-2 by this push:
     new eb73492eac9 HBASE-20499 Replication/Priority executors can use 
specific max queue length as default value instead of general maxQueueLength
eb73492eac9 is described below

commit eb73492eac975169581a6753de276a04f6fa6bde
Author: Nihal Jain <[email protected]>
AuthorDate: Fri Apr 27 14:13:57 2018 +0530

    HBASE-20499 Replication/Priority executors can use specific max queue 
length as default value instead of general maxQueueLength
    
    Signed-off-by: tedyu <[email protected]>
    (cherry picked from commit 6d080762ef795adf02dd0ab236c4b3eb73e19a91)
---
 .../src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java    | 8 ++++++--
 .../main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java | 6 +++++-
 2 files changed, 11 insertions(+), 3 deletions(-)

diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
index ef163640e51..e00ca6a991c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java
@@ -438,8 +438,12 @@ public abstract class RpcExecutor {
    */
   public void resizeQueues(Configuration conf) {
     String configKey = RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH;
-    if (name != null && name.toLowerCase(Locale.ROOT).contains("priority")) {
-      configKey = RpcScheduler.IPC_SERVER_PRIORITY_MAX_CALLQUEUE_LENGTH;
+    if (name != null) {
+      if (name.toLowerCase(Locale.ROOT).contains("priority")) {
+        configKey = RpcScheduler.IPC_SERVER_PRIORITY_MAX_CALLQUEUE_LENGTH;
+      } else if (name.toLowerCase(Locale.ROOT).contains("replication")) {
+        configKey = RpcScheduler.IPC_SERVER_REPLICATION_MAX_CALLQUEUE_LENGTH;
+      }
     }
     final int queueLimit = currentQueueLimit;
     currentQueueLimit = conf.getInt(configKey, queueLimit);
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 82ddcaaf5ea..ace61015ca6 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
@@ -71,6 +71,9 @@ public class SimpleRpcScheduler extends RpcScheduler 
implements ConfigurationObs
       handlerCount * RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER);
     int maxPriorityQueueLength = 
conf.getInt(RpcScheduler.IPC_SERVER_PRIORITY_MAX_CALLQUEUE_LENGTH,
       priorityHandlerCount * 
RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER);
+    int maxReplicationQueueLength =
+      conf.getInt(RpcScheduler.IPC_SERVER_REPLICATION_MAX_CALLQUEUE_LENGTH,
+        replicationHandlerCount * 
RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER);
     int maxBulkLoadQueueLength = 
conf.getInt(RpcScheduler.IPC_SERVER_BULKLOAD_MAX_CALLQUEUE_LENGTH,
       bulkLoadHandlerCount * 
RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER);
 
@@ -116,7 +119,8 @@ 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)
+        RpcExecutor.CALL_QUEUE_TYPE_FIFO_CONF_VALUE, 
maxReplicationQueueLength, priority, conf,
+        abortable)
       : null;
 
     this.metaTransitionExecutor = metaTransitionHandler > 0

Reply via email to