-
Notifications
You must be signed in to change notification settings - Fork 3.4k
HBASE-29912 Codel lifoThreshold should be applied on soft queue limit… #7864
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,6 +17,7 @@ | |
| */ | ||
| package org.apache.hadoop.hbase.ipc; | ||
|
|
||
| import static org.awaitility.Awaitility.await; | ||
| import static org.junit.Assert.assertEquals; | ||
| import static org.junit.Assert.assertFalse; | ||
| import static org.junit.Assert.assertNotEquals; | ||
|
|
@@ -34,13 +35,15 @@ | |
| import java.lang.reflect.Field; | ||
| import java.net.InetSocketAddress; | ||
| import java.util.ArrayList; | ||
| import java.util.Collections; | ||
| import java.util.HashSet; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Set; | ||
| import java.util.concurrent.BlockingQueue; | ||
| import java.util.concurrent.CountDownLatch; | ||
| import java.util.concurrent.LinkedBlockingQueue; | ||
| import java.util.concurrent.TimeUnit; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.hbase.Abortable; | ||
| import org.apache.hadoop.hbase.HBaseClassTestRule; | ||
|
|
@@ -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); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 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 |
||
| 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(1, TimeUnit.SECONDS).until(() -> completedCalls.size() >= 2); | ||
|
|
||
| // Check that we had LIFO switches | ||
| long lifoSwitches = scheduler.getNumLifoModeSwitches(); | ||
| assertTrue("Should have switched to LIFO mode at least once, but got: " + lifoSwitches, | ||
| lifoSwitches > 0); | ||
|
|
||
| // 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 < 5 && 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( | ||
| "Expected LIFO behavior: early completed calls should include call arrived after threshold " | ||
| + "maxCallIdCompleted: " + maxCallIdCompleted, | ||
| maxCallIdCompleted > maxCallQueueLength * codelLifoThreshold); | ||
|
|
||
| } finally { | ||
| scheduler.stop(); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Test that CoDel queue returns to FIFO mode after draining below threshold. | ||
| */ | ||
| @Test | ||
| public void testCoDelQueueDrainAndFifoReturn() throws Exception { | ||
| Configuration testConf = HBaseConfiguration.create(); | ||
| testConf.set(RpcExecutor.CALL_QUEUE_TYPE_CONF_KEY, | ||
| RpcExecutor.CALL_QUEUE_TYPE_CODEL_CONF_VALUE); | ||
| testConf.setInt(RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH, 50); | ||
| testConf.setDouble(RpcExecutor.CALL_QUEUE_CODEL_LIFO_THRESHOLD, 0.8); | ||
| testConf.setInt(HConstants.REGION_SERVER_HANDLER_COUNT, 2); | ||
|
|
||
| PriorityFunction priority = mock(PriorityFunction.class); | ||
| when(priority.getPriority(any(), any(), any())).thenReturn(HConstants.NORMAL_QOS); | ||
| SimpleRpcScheduler scheduler = | ||
| new SimpleRpcScheduler(testConf, 2, 0, 0, priority, HConstants.QOS_THRESHOLD); | ||
|
|
||
| try { | ||
| scheduler.init(CONTEXT); | ||
| scheduler.start(); | ||
|
|
||
| final List<Integer> completedCalls = Collections.synchronizedList(new ArrayList<>()); | ||
|
|
||
| // Phase 1: Fill queue rapidly to trigger LIFO (>40 calls for 80% of 50) | ||
| for (int i = 0; i < 48; i++) { | ||
| final int callId = i; | ||
| CallRunner call = createMockTask(HConstants.NORMAL_QOS); | ||
| call.setStatus(new MonitoredRPCHandlerImpl("test")); | ||
| doAnswer(invocation -> { | ||
| completedCalls.add(callId); | ||
| Thread.sleep(80); | ||
| return null; | ||
| }).when(call).run(); | ||
| scheduler.dispatch(call); | ||
| } | ||
|
|
||
| // Wait for calls to complete | ||
| await().atMost(1, TimeUnit.SECONDS).until(() -> completedCalls.size() >= 2); | ||
| long lifoSwitchesPhase1 = scheduler.getNumLifoModeSwitches(); | ||
| assertTrue("Should have entered LIFO mode", lifoSwitchesPhase1 > 0); | ||
|
|
||
| // Phase 2: Let queue drain | ||
| await().atMost(2, TimeUnit.SECONDS).until(() -> scheduler.getGeneralQueueLength() <= 0); | ||
| int queueLength = scheduler.getGeneralQueueLength(); | ||
| assertTrue("Queue should have drained significantly, but got: " + queueLength, | ||
| queueLength < 25); | ||
|
|
||
| // Phase 3: Send new calls - should process in FIFO order | ||
| completedCalls.clear(); | ||
| for (int i = 100; i < 105; i++) { | ||
| final int callId = i; | ||
| CallRunner call = createMockTask(HConstants.NORMAL_QOS); | ||
| call.setStatus(new MonitoredRPCHandlerImpl("test")); | ||
| doAnswer(invocation -> { | ||
| completedCalls.add(callId); | ||
| Thread.sleep(80); | ||
| return null; | ||
| }).when(call).run(); | ||
| scheduler.dispatch(call); | ||
| } | ||
|
|
||
| // Wait for these calls to complete | ||
| await().atMost(2, TimeUnit.SECONDS).until(() -> completedCalls.size() >= 2); | ||
|
|
||
| // Verify FIFO behavior: calls should complete in order (100, 101, 102, 103, 104) | ||
| assertTrue( | ||
| "Should have completed test calls, but count of completed calls: " + completedCalls.size(), | ||
| completedCalls.size() >= 2); | ||
| // Check that calls completed roughly in order (allowing some variance due to threading with 2 | ||
| // handlers) | ||
| // With 2 handlers, we expect general FIFO order but some interleaving is possible | ||
| // Just verify the general trend is increasing | ||
| int violations = 0; | ||
| for (int i = 0; i < completedCalls.size() - 1; i++) { | ||
| int current = completedCalls.get(i); | ||
| int next = completedCalls.get(i + 1); | ||
| if (next < current) { | ||
| violations++; | ||
| } | ||
| } | ||
| // Allow at most 1 violation due to concurrent execution by 2 handlers | ||
| assertTrue("Calls should complete in approximate FIFO order, violations: " + violations | ||
| + ", order: " + completedCalls, violations <= 1); | ||
|
|
||
| } finally { | ||
| scheduler.stop(); | ||
| } | ||
| } | ||
|
|
||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we still need this guard?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
no we don't need it now.