hangc0276 opened a new pull request, #4730:
URL: https://github.com/apache/bookkeeper/pull/4730

   
   
   ### Motivation
   
     During performance testing, we observed that bookkeeper_server_READ_ENTRY 
p99 latency is less than 100ms, but bookkeeper_server_READ_ENTRY_REQUEST p99 
latency spikes to 3 seconds. The two metrics
     differ in scope:
   
     - READ_ENTRY measures the storage read time only (from before readData() 
to after it completes)
     - READ_ENTRY_REQUEST measures from request enqueue time (enqueueNanos) 
through response write
   
     The root cause is in PacketProcessorBase.sendResponseAndWait(). When 
readWorkerThreadsThrottlingEnabled=true (the default), read responses go 
through sendResponseAndWait, which calls future.get() to
     block the read thread pool thread until the Netty channel write completes.
   
     Under heavy read load, this causes a cascading failure:
   
     1. future.get() blocks each read thread waiting for Netty write completion
     2. If writes are slow (client backpressure, network congestion, event loop 
busy), threads pile up blocked
     3. The read thread pool becomes saturated — all threads blocked waiting 
for writes
     4. New read requests queue up, and their scheduling delay grows
     5. READ_ENTRY_REQUEST (measured from enqueueNanos) spikes, even for 
requests to healthy channels
   
     One slow consumer can block a thread pool thread, reducing capacity for 
all other channels — a classic head-of-line blocking problem.
   
   ### Changes
   Replace the blocking future.get() in sendResponseAndWait() with a 
non-blocking ChannelFutureListener. The key design decisions:
   
     - Read thread freed immediately: After writeAndFlush, the thread returns 
to the pool and can process other requests instead of blocking on future.get()
     - Throttling semantics preserved: onReadRequestFinish() (which releases 
the read semaphore) is moved into the ChannelFutureListener, so it is only 
called after the write completes. This ensures the
     read concurrency limit still gates on write completion, without wasting a 
thread to enforce it.
     - Captured local variables: Since the processor may be recycled after 
sendResponseAndWait returns, enqueueNanos and requestProcessor are captured as 
local variables before the async callback.
     - Semaphore always released: Both the success and failure paths in the 
listener call onReadRequestFinish(), preventing semaphore leaks on write 
failure. (The previous code had a subtle issue: on
     ExecutionException/InterruptedException, sendResponseAndWait returned 
early without recording the metric, though onReadRequestFinish was still called 
by the caller.)
   
   ### Test Plan
     Added 3 new unit tests to ReadEntryProcessorTest:
   
     - testThrottledReadNonBlockingOnSuccess — verifies that with 
throttle=true, run() returns immediately and onReadRequestFinish() is deferred 
until the write future completes
     - testThrottledReadNonBlockingOnWriteFailure — verifies 
onReadRequestFinish() is still called when the write fails, ensuring the read 
semaphore is always released
     - testNonThrottledReadCallsOnFinishSynchronously — verifies that with 
throttle=false, onReadRequestFinish() is called synchronously
   
   


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