Fix NPE in PendingAddOp.maybeTimeout() when clientCtx is null after recycling#4760
Open
eolivelli wants to merge 1 commit intoapache:masterfrom
Open
Fix NPE in PendingAddOp.maybeTimeout() when clientCtx is null after recycling#4760eolivelli wants to merge 1 commit intoapache:masterfrom
eolivelli wants to merge 1 commit intoapache:masterfrom
Conversation
…ecycling monitorPendingAddOps() iterates the pendingAddOps queue and calls maybeTimeout() on each op. Concurrently, sendAddSuccessCallbacks() can remove a completed op from the queue and, via submitCallback(), lead to recyclePendAddOpObject() which sets clientCtx = null. If the scheduler thread still holds an iterator reference to that op and then calls maybeTimeout(), the dereference of clientCtx causes an NPE. The race is triggered in practice when addEntryQuorumTimeoutNanos > 0 (i.e. the quorum-timeout monitor is enabled). It became visible with Netty 4.1.130, which changed Recycler thread-scheduling behavior and made the narrow window between the iterator snapshot and the null assignment observable. Fix: - Make clientCtx volatile so that the null write in the synchronized recyclePendAddOpObject() is immediately visible to the unsynchronized maybeTimeout() reader. - Guard the top of maybeTimeout() with an explicit null check: if clientCtx is null the op has already completed and recycled, so there is nothing to time out and the method returns false. Closes: apache#4759
merlimat
reviewed
Apr 26, 2026
| } | ||
|
|
||
| boolean maybeTimeout() { | ||
| if (clientCtx == null) { |
Contributor
There was a problem hiding this comment.
I don't think this would solve the race condition completely, since it could be set to null before the next read on line 163.
Better approach would be (even without the volatile) to assign to local variable and the the null there. Still wouldn't address whether it's safe to call timeoutQuorumWait() if it was already recycled.
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
monitorPendingAddOps()iterates over thependingAddOpsqueue and callsmaybeTimeout()on each op. Concurrently,sendAddSuccessCallbacks()canremove a completed op from the queue, call
submitCallback()on it, andultimately trigger
recyclePendAddOpObject()which setsclientCtx = null.If the scheduler thread still holds a live iterator reference to the same op
and then calls
maybeTimeout(), dereferencingclientCtxcauses aNullPointerException:The race is only triggered when
addEntryQuorumTimeoutNanos > 0(i.e. thequorum-timeout monitor is actually scheduled). It became reliably observable
with Netty 4.1.130, which changed
Recyclerthread-scheduling behaviorand narrowed the window enough to expose the pre-existing race.
Closes #4759.
Changes
PendingAddOp.javaclientCtx→volatile ClientContext clientCtxrecyclePendAddOpObject()issynchronizedbutmaybeTimeout()is not.Without
volatilethe JMM does not guarantee that thenullwrite inrecyclePendAddOpObject()is visible to the unsynchronized read inmaybeTimeout(). Making the fieldvolatile(consistent with the existingvolatile long requestTimeNanos) closes the visibility gap.maybeTimeout()If
clientCtxisnullthe op has already been recycled and itsadd-entry completed; there is nothing to time out, so the method
returns
falseimmediately.PendingAddOpTest.javaThree new unit tests:
testMaybeTimeoutReturnsFalseWhenClientCtxIsNullclientCtx = nullmust not NPEtestMaybeTimeoutReturnsFalseWhenWithinQuorumTimeouttestMaybeTimeoutReturnsTrueWhenQuorumTimeoutExpiredtruereturned