Skip to content

Fix NPE in PendingAddOp.maybeTimeout() when clientCtx is null after recycling#4760

Open
eolivelli wants to merge 1 commit intoapache:masterfrom
eolivelli:fix/pending-add-op-maybe-timeout-null-clientctx
Open

Fix NPE in PendingAddOp.maybeTimeout() when clientCtx is null after recycling#4760
eolivelli wants to merge 1 commit intoapache:masterfrom
eolivelli:fix/pending-add-op-maybe-timeout-null-clientctx

Conversation

@eolivelli
Copy link
Copy Markdown
Contributor

@eolivelli eolivelli commented Apr 25, 2026

Motivation

monitorPendingAddOps() iterates over the pendingAddOps queue and calls
maybeTimeout() on each op. Concurrently, sendAddSuccessCallbacks() can
remove a completed op from the queue, call submitCallback() on it, and
ultimately trigger recyclePendAddOpObject() which sets clientCtx = null.

If the scheduler thread still holds a live iterator reference to the same op
and then calls maybeTimeout(), dereferencing clientCtx causes a
NullPointerException:

java.lang.NullPointerException
    at org.apache.bookkeeper.client.PendingAddOp.maybeTimeout(PendingAddOp.java:157)
    at org.apache.bookkeeper.client.LedgerHandle.monitorPendingAddOps(LedgerHandle.java:2063)

The race is only triggered when addEntryQuorumTimeoutNanos > 0 (i.e. the
quorum-timeout monitor is actually scheduled). It became reliably observable
with Netty 4.1.130, which changed Recycler thread-scheduling behavior
and narrowed the window enough to expose the pre-existing race.

Closes #4759.

Changes

PendingAddOp.java

  • clientCtxvolatile ClientContext clientCtx
    recyclePendAddOpObject() is synchronized but maybeTimeout() is not.
    Without volatile the JMM does not guarantee that the null write in
    recyclePendAddOpObject() is visible to the unsynchronized read in
    maybeTimeout(). Making the field volatile (consistent with the existing
    volatile long requestTimeNanos) closes the visibility gap.
  • Null guard at the top of maybeTimeout()
    If clientCtx is null the op has already been recycled and its
    add-entry completed; there is nothing to time out, so the method
    returns false immediately.

PendingAddOpTest.java

Three new unit tests:

Test What it verifies
testMaybeTimeoutReturnsFalseWhenClientCtxIsNull The exact race: clientCtx = null must not NPE
testMaybeTimeoutReturnsFalseWhenWithinQuorumTimeout Normal path — not yet timed out
testMaybeTimeoutReturnsTrueWhenQuorumTimeoutExpired Normal path — timed out, true returned

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

boolean maybeTimeout() {
if (clientCtx == null) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

NullPointerException in PendingAddOp.maybeTimeout when clientCtx is null after recycling

2 participants