apurtell commented on code in PR #7864:
URL: https://github.com/apache/hbase/pull/7864#discussion_r3326474077


##########
hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestSimpleRpcScheduler.java:
##########
@@ -808,4 +838,161 @@ 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);
+        call.setStatus(new MonitoredRPCHandlerImpl("test"));
+        doAnswer(invocation -> {
+          completedCalls.add(callId);
+          Thread.sleep(100); // Slow processing to allow queue to build up
+          return null;
+        }).when(call).run();
+        scheduler.dispatch(call);
+        // No delay between dispatches - rapidly fill the queue
+      }
+
+      // Wait for some calls to complete
+      await().atMost(2, TimeUnit.SECONDS).until(() -> completedCalls.size() >= 
3);
+
+      // Check that we had LIFO switches
+      long lifoSwitches = scheduler.getNumLifoModeSwitches();
+      assertTrue(lifoSwitches > 0,
+        "Should have switched to LIFO mode at least once, but got: " + 
lifoSwitches);
+
+      // Verify LIFO behavior: Among first completed calls, we should see 
higher call IDs
+      // (indicating later dispatched calls completed first)
+      int maxCallIdCompleted = -1;
+      for (int i = 0; i < completedCalls.size(); i++) {
+        maxCallIdCompleted = Math.max(maxCallIdCompleted, 
completedCalls.get(i));
+      }
+      // At least one of the early completed calls should have a high ID (>20)
+      // indicating LIFO processing
+      assertTrue(maxCallIdCompleted > maxCallQueueLength * coDelLifoThreshold,
+        "Expected LIFO behavior: early completed calls should include call 
arrived after threshold "
+          + "maxCallIdCompleted: " + maxCallIdCompleted);
+
+    } finally {
+      scheduler.stop();
+    }
+  }
+
+  /**
+   * Test that CoDel queue returns to FIFO mode after draining below threshold.
+   */
+  @Test
+  public void testCoDelQueueDrainAndFifoReturn() throws Exception {

Review Comment:
   The new test runs 5 calls (IDs 100–104) on 2 handlers with 50ms processing 
each, then asserts that completion order has violations <= 1. With two threads 
pulling concurrently from a single queue FIFO take() can produce an order like 
100, 102, 101, 103, 104 (two violations) when handler A picks 100 before 
handler B picks 101 but B finishes first while A was paused by the OS 
scheduler. On a busy CI runner this will eventually flake. 
   
   Replace the order-violation count with a check that the first completion has 
the smallest id (completedCalls.get(0) == 100). This is what FIFO actually 
guarantees for a single producer issuing sequential dispatch calls, and is 
robust to handler interleaving.



-- 
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]

Reply via email to