Fix read thread blocking in sendResponseAndWait causing READ_ENTRY_REQUEST p99 latency spike#4730
Open
hangc0276 wants to merge 2 commits intoapache:masterfrom
Open
Fix read thread blocking in sendResponseAndWait causing READ_ENTRY_REQUEST p99 latency spike#4730hangc0276 wants to merge 2 commits intoapache:masterfrom
hangc0276 wants to merge 2 commits intoapache:masterfrom
Conversation
…QUEST p99 latency spike
merlimat
approved these changes
Mar 13, 2026
wenbingshen
approved these changes
Mar 16, 2026
Member
wenbingshen
left a comment
There was a problem hiding this comment.
LGTM. The change to maxReadsInProgressLimit needs to be added to the release documentation in the future.
lhotari
approved these changes
Mar 16, 2026
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
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:
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:
One slow consumer can block a thread pool thread, reducing capacity for all other channels — a classic head-of-line blocking problem.
Changes
1. Non-blocking sendResponseAndWait (PacketProcessorBase.java)
Replace the blocking future.get() in sendResponseAndWait() with a non-blocking ChannelFutureListener. The key design decisions:
read concurrency limit still gates on write completion, without wasting a thread to enforce it.
ExecutionException/InterruptedException, sendResponseAndWait returned early without recording the metric, though onReadRequestFinish was still called by the caller.)
2. Enable maxReadsInProgressLimit by default (ServerConfiguration.java)
Changed the default of maxReadsInProgressLimit from 0 (unlimited) to 10000.
With the old blocking sendResponseAndWait, the read thread pool (default 8 threads) acted as an implicit concurrency bound — each thread blocked on one write at a time, so at most 8 responses could be
buffered. With the new non-blocking approach, threads are freed immediately and keep processing reads. Without a bound, a slow consumer could cause unbounded response buffer growth in Netty's write
queue, leading to OOM.
The maxReadsInProgressLimit semaphore now serves as the explicit bound. Since onReadRequestFinish() is called in the ChannelFutureListener (after the write completes), the semaphore precisely counts
reads whose response data is in memory but not yet flushed. The default of 10,000 provides good throughput (~40MB at 4KB entries) while preventing unbounded memory growth. Users with larger entries
should tune this lower (memoryBudget / avgEntrySize).
Test Plan
Added 5 new unit tests to ReadEntryProcessorTest:
held through non-blocking run() return, blocks a second read while write is in progress, and released only when the write future completes