diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java index 298a9fc3aeb2..6d1dd42d15bc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RWQueueRpcExecutor.java @@ -76,7 +76,9 @@ public RWQueueRpcExecutor(final String name, final int handlerCount, final int m numWriteQueues = calcNumWriters(this.numCallQueues, callqReadShare); writeHandlersCount = Math.max(numWriteQueues, calcNumWriters(handlerCount, callqReadShare)); - int readQueues = calcNumReaders(this.numCallQueues, callqReadShare); + int readQueues = callqReadShare > 0 ? + Math.max(1, calcNumReaders(this.numCallQueues, callqReadShare)) : + calcNumReaders(this.numCallQueues, callqReadShare); int readHandlers = Math.max(readQueues, calcNumReaders(handlerCount, callqReadShare)); int scanHandlers = Math.max(0, (int) Math.floor(readHandlers * callqScanShare)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRpcSchedulerFactory.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRpcSchedulerFactory.java index 17f0dc3a7839..4698a0acfd15 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRpcSchedulerFactory.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRpcSchedulerFactory.java @@ -85,4 +85,16 @@ public void testFifo() { RpcScheduler rpcScheduler = factory.create(this.conf, null, null); assertTrue(rpcScheduler.getClass().equals(FifoRpcScheduler.class)); } + + @Test + public void testRWQWithSmallCallQueueFactor() { + // Set some configs just to see how it changes the scheduler. Can't assert the settings had + // an effect. Just eyeball the log. + this.conf.setDouble(RWQueueRpcExecutor.CALL_QUEUE_READ_SHARE_CONF_KEY, 0.5); + this.conf.setDouble(RpcExecutor.CALL_QUEUE_HANDLER_FACTOR_CONF_KEY, 0.000001); + this.conf.setDouble(RWQueueRpcExecutor.CALL_QUEUE_SCAN_SHARE_CONF_KEY, 0.5); + RpcSchedulerFactory factory = new SimpleRpcSchedulerFactory(); + RpcScheduler rpcScheduler = factory.create(this.conf, null, null); + assertTrue(rpcScheduler.getClass().equals(SimpleRpcScheduler.class)); + } }