HBASE-15971 Regression: Random Read/WorkloadC slower in 1.x than 0.98 Restore FIFO as the default in place of 'deadline' that sorted on request priority. The sort costs us throughput. Enable it if you want it rather than have it as default.
Name threads to include the scheduler type.
This commit is contained in:
parent
b69c77ac1a
commit
fa50d456a8
|
@ -52,12 +52,16 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
|
|||
public static final String CALL_QUEUE_HANDLER_FACTOR_CONF_KEY =
|
||||
"hbase.ipc.server.callqueue.handler.factor";
|
||||
|
||||
/** If set to 'deadline', the default, uses a priority queue and deprioritizes long-running scans
|
||||
/**
|
||||
* The default, 'fifo', has the least friction but is dumb.
|
||||
* If set to 'deadline', uses a priority queue and deprioritizes long-running scans. Sorting by
|
||||
* priority comes at a cost, reduced throughput.
|
||||
*/
|
||||
public static final String CALL_QUEUE_TYPE_CONF_KEY = "hbase.ipc.server.callqueue.type";
|
||||
public static final String CALL_QUEUE_TYPE_CODEL_CONF_VALUE = "codel";
|
||||
public static final String CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE = "deadline";
|
||||
public static final String CALL_QUEUE_TYPE_FIFO_CONF_VALUE = "fifo";
|
||||
public static final String CALL_QUEUE_TYPE_CONF_KEY = "hbase.ipc.server.callqueue.type";
|
||||
public static final String CALL_QUEUE_TYPE_CONF_DEFAULT = CALL_QUEUE_TYPE_FIFO_CONF_VALUE;
|
||||
|
||||
/** max delay in msec used to bound the deprioritized requests */
|
||||
public static final String QUEUE_MAX_CALL_DELAY_CONF_KEY
|
||||
|
@ -181,7 +185,7 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
|
|||
this.abortable = server;
|
||||
|
||||
String callQueueType = conf.get(CALL_QUEUE_TYPE_CONF_KEY,
|
||||
CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE);
|
||||
CALL_QUEUE_TYPE_FIFO_CONF_VALUE);
|
||||
float callqReadShare = conf.getFloat(CALL_QUEUE_READ_SHARE_CONF_KEY, 0);
|
||||
float callqScanShare = conf.getFloat(CALL_QUEUE_SCAN_SHARE_CONF_KEY, 0);
|
||||
|
||||
|
@ -200,18 +204,18 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
|
|||
// multiple read/write queues
|
||||
if (isDeadlineQueueType(callQueueType)) {
|
||||
CallPriorityComparator callPriority = new CallPriorityComparator(conf, this.priority);
|
||||
callExecutor = new RWQueueRpcExecutor("RWQ.default", handlerCount, numCallQueues,
|
||||
callExecutor = new RWQueueRpcExecutor("RW.deadline.Q", handlerCount, numCallQueues,
|
||||
callqReadShare, callqScanShare, maxQueueLength, conf, abortable,
|
||||
BoundedPriorityBlockingQueue.class, callPriority);
|
||||
} else if (callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE)) {
|
||||
Object[] callQueueInitArgs = {maxQueueLength, codelTargetDelay, codelInterval,
|
||||
codelLifoThreshold, numGeneralCallsDropped, numLifoModeSwitches};
|
||||
callExecutor = new RWQueueRpcExecutor("RWQ.default", handlerCount,
|
||||
callExecutor = new RWQueueRpcExecutor("RW.codel.Q", handlerCount,
|
||||
numCallQueues, callqReadShare, callqScanShare,
|
||||
AdaptiveLifoCoDelCallQueue.class, callQueueInitArgs,
|
||||
AdaptiveLifoCoDelCallQueue.class, callQueueInitArgs);
|
||||
} else {
|
||||
callExecutor = new RWQueueRpcExecutor("RWQ.default", handlerCount, numCallQueues,
|
||||
callExecutor = new RWQueueRpcExecutor("RW.fifo.Q", handlerCount, numCallQueues,
|
||||
callqReadShare, callqScanShare, maxQueueLength, conf, abortable);
|
||||
}
|
||||
} else {
|
||||
|
@ -219,26 +223,26 @@ public class SimpleRpcScheduler extends RpcScheduler implements ConfigurationObs
|
|||
if (isDeadlineQueueType(callQueueType)) {
|
||||
CallPriorityComparator callPriority = new CallPriorityComparator(conf, this.priority);
|
||||
callExecutor =
|
||||
new BalancedQueueRpcExecutor("BalancedQ.default", handlerCount, numCallQueues,
|
||||
new BalancedQueueRpcExecutor("B.deadline.Q", handlerCount, numCallQueues,
|
||||
conf, abortable, BoundedPriorityBlockingQueue.class, maxQueueLength, callPriority);
|
||||
} else if (callQueueType.equals(CALL_QUEUE_TYPE_CODEL_CONF_VALUE)) {
|
||||
callExecutor =
|
||||
new BalancedQueueRpcExecutor("BalancedQ.default", handlerCount, numCallQueues,
|
||||
new BalancedQueueRpcExecutor("B.codel.Q", handlerCount, numCallQueues,
|
||||
conf, abortable, AdaptiveLifoCoDelCallQueue.class, maxQueueLength,
|
||||
codelTargetDelay, codelInterval, codelLifoThreshold,
|
||||
numGeneralCallsDropped, numLifoModeSwitches);
|
||||
} else {
|
||||
callExecutor = new BalancedQueueRpcExecutor("BalancedQ.default", handlerCount,
|
||||
callExecutor = new BalancedQueueRpcExecutor("B.fifo.Q", handlerCount,
|
||||
numCallQueues, maxQueueLength, conf, abortable);
|
||||
}
|
||||
}
|
||||
// Create 2 queues to help priorityExecutor be more scalable.
|
||||
this.priorityExecutor = priorityHandlerCount > 0 ?
|
||||
new BalancedQueueRpcExecutor("BalancedQ.priority", priorityHandlerCount, 2,
|
||||
new BalancedQueueRpcExecutor("B.priority.fifo.Q", priorityHandlerCount, 2,
|
||||
maxPriorityQueueLength):
|
||||
null;
|
||||
this.replicationExecutor =
|
||||
replicationHandlerCount > 0 ? new BalancedQueueRpcExecutor("BalancedQ.replication",
|
||||
replicationHandlerCount > 0 ? new BalancedQueueRpcExecutor("B.replication.fifo.Q",
|
||||
replicationHandlerCount, 1, maxQueueLength, conf, abortable) : null;
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue