|
19 | 19 | package org.apache.bookkeeper.proto; |
20 | 20 |
|
21 | 21 | import static org.junit.Assert.assertEquals; |
| 22 | +import static org.junit.Assert.assertFalse; |
22 | 23 | import static org.junit.Assert.assertTrue; |
23 | 24 | import static org.mockito.ArgumentMatchers.any; |
24 | 25 | import static org.mockito.ArgumentMatchers.anyLong; |
25 | 26 | import static org.mockito.Mockito.doAnswer; |
26 | 27 | import static org.mockito.Mockito.mock; |
| 28 | +import static org.mockito.Mockito.never; |
27 | 29 | import static org.mockito.Mockito.times; |
28 | 30 | import static org.mockito.Mockito.verify; |
29 | 31 | import static org.mockito.Mockito.when; |
|
38 | 40 | import java.util.concurrent.CountDownLatch; |
39 | 41 | import java.util.concurrent.ExecutorService; |
40 | 42 | import java.util.concurrent.Executors; |
| 43 | +import java.util.concurrent.Semaphore; |
41 | 44 | import java.util.concurrent.atomic.AtomicReference; |
42 | 45 | import org.apache.bookkeeper.bookie.Bookie; |
43 | 46 | import org.apache.bookkeeper.bookie.BookieException; |
44 | 47 | import org.apache.bookkeeper.common.concurrent.FutureUtils; |
| 48 | +import org.apache.bookkeeper.conf.ServerConfiguration; |
45 | 49 | import org.apache.bookkeeper.proto.BookieProtocol.ReadRequest; |
46 | 50 | import org.apache.bookkeeper.proto.BookieProtocol.Response; |
47 | 51 | import org.apache.bookkeeper.stats.NullStatsLogger; |
@@ -195,4 +199,178 @@ public void testNonFenceRequest() throws Exception { |
195 | 199 | assertEquals(BookieProtocol.READENTRY, response.getOpCode()); |
196 | 200 | assertEquals(BookieProtocol.EOK, response.getErrorCode()); |
197 | 201 | } |
| 202 | + |
| 203 | + /** |
| 204 | + * Test that when throttleReadResponses=true and the caller is not in the Netty event loop, |
| 205 | + * the read thread is not blocked by the write. onReadRequestFinish() should only be called |
| 206 | + * after the write future completes, preserving throttling without blocking the thread. |
| 207 | + */ |
| 208 | + @Test |
| 209 | + public void testThrottledReadNonBlockingOnSuccess() throws Exception { |
| 210 | + // Setup event loop to simulate read worker thread (not event loop thread) |
| 211 | + EventLoop eventLoop = mock(EventLoop.class); |
| 212 | + when(eventLoop.inEventLoop()).thenReturn(false); |
| 213 | + doAnswer(inv -> { |
| 214 | + ((Runnable) inv.getArgument(0)).run(); |
| 215 | + return null; |
| 216 | + }).when(eventLoop).execute(any(Runnable.class)); |
| 217 | + when(channel.eventLoop()).thenReturn(eventLoop); |
| 218 | + |
| 219 | + // Use a controllable promise so we can verify deferred behavior |
| 220 | + DefaultChannelPromise writeFuture = new DefaultChannelPromise(channel); |
| 221 | + doAnswer(inv -> writeFuture).when(channel).writeAndFlush(any(Response.class)); |
| 222 | + |
| 223 | + long ledgerId = System.currentTimeMillis(); |
| 224 | + ReadRequest request = ReadRequest.create( |
| 225 | + BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, 1, (short) 0, new byte[]{}); |
| 226 | + ReadEntryProcessor processor = ReadEntryProcessor.create( |
| 227 | + request, requestHandler, requestProcessor, null, true /* throttle */); |
| 228 | + |
| 229 | + // run() should return immediately without blocking on the write |
| 230 | + processor.run(); |
| 231 | + |
| 232 | + // Write should have been issued |
| 233 | + verify(channel, times(1)).writeAndFlush(any(Response.class)); |
| 234 | + // But onReadRequestFinish should NOT have been called yet — write not completed |
| 235 | + verify(requestProcessor, never()).onReadRequestFinish(); |
| 236 | + |
| 237 | + // Complete the write |
| 238 | + writeFuture.setSuccess(); |
| 239 | + |
| 240 | + // Now onReadRequestFinish should have been called |
| 241 | + verify(requestProcessor, times(1)).onReadRequestFinish(); |
| 242 | + } |
| 243 | + |
| 244 | + /** |
| 245 | + * Test that onReadRequestFinish() is still called even when the write fails, |
| 246 | + * so the read semaphore is always released. |
| 247 | + */ |
| 248 | + @Test |
| 249 | + public void testThrottledReadNonBlockingOnWriteFailure() throws Exception { |
| 250 | + EventLoop eventLoop = mock(EventLoop.class); |
| 251 | + when(eventLoop.inEventLoop()).thenReturn(false); |
| 252 | + doAnswer(inv -> { |
| 253 | + ((Runnable) inv.getArgument(0)).run(); |
| 254 | + return null; |
| 255 | + }).when(eventLoop).execute(any(Runnable.class)); |
| 256 | + when(channel.eventLoop()).thenReturn(eventLoop); |
| 257 | + |
| 258 | + DefaultChannelPromise writeFuture = new DefaultChannelPromise(channel); |
| 259 | + doAnswer(inv -> writeFuture).when(channel).writeAndFlush(any(Response.class)); |
| 260 | + |
| 261 | + long ledgerId = System.currentTimeMillis(); |
| 262 | + ReadRequest request = ReadRequest.create( |
| 263 | + BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, 1, (short) 0, new byte[]{}); |
| 264 | + ReadEntryProcessor processor = ReadEntryProcessor.create( |
| 265 | + request, requestHandler, requestProcessor, null, true /* throttle */); |
| 266 | + |
| 267 | + processor.run(); |
| 268 | + |
| 269 | + verify(channel, times(1)).writeAndFlush(any(Response.class)); |
| 270 | + verify(requestProcessor, never()).onReadRequestFinish(); |
| 271 | + |
| 272 | + // Fail the write |
| 273 | + writeFuture.setFailure(new IOException("channel write failed")); |
| 274 | + |
| 275 | + // onReadRequestFinish must still be called to release the read semaphore |
| 276 | + verify(requestProcessor, times(1)).onReadRequestFinish(); |
| 277 | + } |
| 278 | + |
| 279 | + /** |
| 280 | + * Test that when throttleReadResponses=false, onReadRequestFinish() is called |
| 281 | + * synchronously before run() returns. |
| 282 | + */ |
| 283 | + @Test |
| 284 | + public void testNonThrottledReadCallsOnFinishSynchronously() throws Exception { |
| 285 | + // sendResponse (non-throttle path) uses channel.isActive() and two-arg writeAndFlush |
| 286 | + when(channel.isActive()).thenReturn(true); |
| 287 | + when(channel.writeAndFlush(any(), any(ChannelPromise.class))).thenReturn(mock(ChannelPromise.class)); |
| 288 | + |
| 289 | + long ledgerId = System.currentTimeMillis(); |
| 290 | + ReadRequest request = ReadRequest.create( |
| 291 | + BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, 1, (short) 0, new byte[]{}); |
| 292 | + ReadEntryProcessor processor = ReadEntryProcessor.create( |
| 293 | + request, requestHandler, requestProcessor, null, false /* no throttle */); |
| 294 | + |
| 295 | + processor.run(); |
| 296 | + |
| 297 | + verify(channel, times(1)).writeAndFlush(any(), any(ChannelPromise.class)); |
| 298 | + // onReadRequestFinish should have been called synchronously |
| 299 | + verify(requestProcessor, times(1)).onReadRequestFinish(); |
| 300 | + } |
| 301 | + |
| 302 | + /** |
| 303 | + * Verify that maxReadsInProgressLimit defaults to 10000 (enabled), |
| 304 | + * ensuring non-blocking read response writes are bounded by default. |
| 305 | + */ |
| 306 | + @Test |
| 307 | + public void testDefaultMaxReadsInProgressLimitIsEnabled() { |
| 308 | + ServerConfiguration conf = new ServerConfiguration(); |
| 309 | + assertEquals("maxReadsInProgressLimit should default to 10000", |
| 310 | + 10000, conf.getMaxReadsInProgressLimit()); |
| 311 | + } |
| 312 | + |
| 313 | + /** |
| 314 | + * Test that the read semaphore is held from request creation until the write future completes, |
| 315 | + * not released when the read thread returns. This ensures that maxReadsInProgressLimit correctly |
| 316 | + * bounds the number of read responses buffered in memory, even though the read thread is |
| 317 | + * non-blocking. |
| 318 | + */ |
| 319 | + @Test |
| 320 | + public void testThrottledReadHoldsSemaphoreUntilWriteCompletes() throws Exception { |
| 321 | + // Simulate maxReadsInProgressLimit=1 with a real semaphore |
| 322 | + Semaphore readsSemaphore = new Semaphore(1); |
| 323 | + |
| 324 | + doAnswer(inv -> { |
| 325 | + readsSemaphore.acquireUninterruptibly(); |
| 326 | + return null; |
| 327 | + }).when(requestProcessor).onReadRequestStart(any(Channel.class)); |
| 328 | + doAnswer(inv -> { |
| 329 | + readsSemaphore.release(); |
| 330 | + return null; |
| 331 | + }).when(requestProcessor).onReadRequestFinish(); |
| 332 | + |
| 333 | + // Setup non-event-loop thread |
| 334 | + EventLoop eventLoop = mock(EventLoop.class); |
| 335 | + when(eventLoop.inEventLoop()).thenReturn(false); |
| 336 | + doAnswer(inv -> { |
| 337 | + ((Runnable) inv.getArgument(0)).run(); |
| 338 | + return null; |
| 339 | + }).when(eventLoop).execute(any(Runnable.class)); |
| 340 | + when(channel.eventLoop()).thenReturn(eventLoop); |
| 341 | + |
| 342 | + // Controllable write future |
| 343 | + DefaultChannelPromise writeFuture = new DefaultChannelPromise(channel); |
| 344 | + doAnswer(inv -> writeFuture).when(channel).writeAndFlush(any(Response.class)); |
| 345 | + |
| 346 | + long ledgerId = System.currentTimeMillis(); |
| 347 | + ReadRequest request = ReadRequest.create( |
| 348 | + BookieProtocol.CURRENT_PROTOCOL_VERSION, ledgerId, 1, (short) 0, new byte[]{}); |
| 349 | + |
| 350 | + // create() calls onReadRequestStart → semaphore acquired |
| 351 | + ReadEntryProcessor processor = ReadEntryProcessor.create( |
| 352 | + request, requestHandler, requestProcessor, null, true /* throttle */); |
| 353 | + |
| 354 | + // Semaphore should be acquired (1 permit used) |
| 355 | + assertEquals("semaphore should have 0 permits after read started", |
| 356 | + 0, readsSemaphore.availablePermits()); |
| 357 | + |
| 358 | + // Run the processor — thread returns immediately (non-blocking) |
| 359 | + processor.run(); |
| 360 | + |
| 361 | + // Semaphore should STILL be held (write not completed) |
| 362 | + assertEquals("semaphore should still have 0 permits while write is in progress", |
| 363 | + 0, readsSemaphore.availablePermits()); |
| 364 | + |
| 365 | + // A second read would be unable to acquire the semaphore |
| 366 | + assertFalse("second read should not be able to acquire semaphore", |
| 367 | + readsSemaphore.tryAcquire()); |
| 368 | + |
| 369 | + // Complete the write |
| 370 | + writeFuture.setSuccess(); |
| 371 | + |
| 372 | + // Now semaphore should be released — a new read can proceed |
| 373 | + assertEquals("semaphore should have 1 permit after write completes", |
| 374 | + 1, readsSemaphore.availablePermits()); |
| 375 | + } |
198 | 376 | } |
0 commit comments