apurtell commented on code in PR #7864:
URL: https://github.com/apache/hbase/pull/7864#discussion_r2893029495
##########
hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java:
##########
@@ -814,4 +817,166 @@ public void drop() {
}
};
}
+
+ /**
+ * Test LIFO switching behavior through actual RPC calls. This test verifies
that when the queue
+ * fills beyond the LIFO threshold, newer calls are processed before older
calls (LIFO mode).
+ */
+ @Test
+ public void testCoDelLifoWithRpcCalls() throws Exception {
+ Configuration testConf = HBaseConfiguration.create();
+ testConf.set(RpcExecutor.CALL_QUEUE_TYPE_CONF_KEY,
+ RpcExecutor.CALL_QUEUE_TYPE_CODEL_CONF_VALUE);
+ int maxCallQueueLength = 50;
+ double codelLifoThreshold = 0.8;
+ testConf.setInt(RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH,
maxCallQueueLength);
+ testConf.setDouble(RpcExecutor.CALL_QUEUE_CODEL_LIFO_THRESHOLD,
codelLifoThreshold);
+ testConf.setInt(RpcExecutor.CALL_QUEUE_CODEL_TARGET_DELAY, 100);
+ testConf.setInt(RpcExecutor.CALL_QUEUE_CODEL_INTERVAL, 100);
+ testConf.setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 1); // Single
handler to control
+ // processing
+
+ PriorityFunction priority = mock(PriorityFunction.class);
+ when(priority.getPriority(any(), any(),
any())).thenReturn(HConstants.NORMAL_QOS);
+ SimpleRpcScheduler scheduler =
+ new SimpleRpcScheduler(testConf, 1, 0, 0, priority,
HConstants.QOS_THRESHOLD);
+
+ try {
+ scheduler.init(CONTEXT);
+ scheduler.start();
+
+ // Track completion order
+ final List<Integer> completedCalls = Collections.synchronizedList(new
ArrayList<>());
+
+ // Dispatch many slow calls rapidly to fill the queue beyond 80%
threshold
+ // With queue limit of 50, we need > 40 calls to cross 80%
+ int numCalls = 48;
+ for (int i = 0; i < numCalls; i++) {
+ final int callId = i;
+ CallRunner call = createMockTask(HConstants.NORMAL_QOS);
Review Comment:
Here and in two more cases createMockTask will mock a ServerCall without
setting getReceiveTime(). It will default to 0. Doesn't this mess up the call
delay calculation that coDel will do? It will be something like `callDelay =
System.currentTimeMillis() - 0` . I guess your tests are passing but CoDel will
immediately detect overload. The LIFO switch counter will still increment but
other aspects are probably wrong.
You should look at this.
##########
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/AdaptiveLifoCoDelCallQueue.java:
##########
@@ -44,7 +44,7 @@ public class AdaptiveLifoCoDelCallQueue implements
BlockingQueue<CallRunner> {
private LinkedBlockingDeque<CallRunner> queue;
// so we can calculate actual threshold to switch to LIFO under load
- private int maxCapacity;
+ private int currentQueueLimit;
Review Comment:
Because this can be updated with `onConfigurationChange` should this be
`volatile` ? Just a nit.
##########
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/AdaptiveLifoCoDelCallQueue.java:
##########
@@ -44,7 +44,7 @@ public class AdaptiveLifoCoDelCallQueue implements
BlockingQueue<CallRunner> {
private LinkedBlockingDeque<CallRunner> queue;
// so we can calculate actual threshold to switch to LIFO under load
- private int maxCapacity;
+ private int currentQueueLimit;
Review Comment:
This is also confusing in that it seems to shadow a variable of the same
name that *is* declared volatile?
Please don't shadow. Why do we need this separately?
##########
hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcExecutor.java:
##########
@@ -231,20 +234,7 @@ public Map<String, Long> getCallQueueSizeSummary() {
.collect(Collectors.groupingBy(Pair::getFirst,
Collectors.summingLong(Pair::getSecond)));
}
- // This method can only be called ONCE per executor instance.
- // Before calling: queueInitArgs[0] contains the soft limit (desired queue
capacity)
- // After calling: queueInitArgs[0] is set to hard limit and
currentQueueLimit stores the original
- // soft limit.
- // Multiple calls would incorrectly use the hard limit as the soft limit.
- // As all the queues has same initArgs and queueClass, there should be no
need to call this again.
protected void initializeQueues(final int numQueues) {
- if (!queues.isEmpty()) {
Review Comment:
Do we still need this guard?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]