-
Notifications
You must be signed in to change notification settings - Fork 953
[fix] Write stuck due to pending add callback by multiple threads #4557
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Conversation
Great analysis @poorbarcode. IIRC, there might be some other pending PRs in this area where the review isn't complete. I'll check if they are related. |
There are a few open Issues/PRs that haven't been processed and which could be related:
@poorbarcode would you be able to check if they are related and relevant? |
rerun failure checks |
@poorbarcode please take a look at the tests - there failures/timeouts |
reopen's reason: rerun failure checks |
![]() https://github.com/apache/bookkeeper/actions/runs/14370207115/job/40291989272?pr=4557 |
Motivation
Background: the normal steps of adding an entry
PendingAddOp.writeComplete
after receiving the response from BK servers.Background: the steps of disconnection
PendingAddOp.writeComplete
. You can reproduce this flow by the new testtestAddEntriesCallbackWithBKClientThread
Issue-1: write stuck due to pending add callback by multiple threads
3
3
2
client->BK1
client->BK2
client-> BK3
ack
:1/3
ack
:2/3
complete
since ack quorum is2/3
PendingAddOp.writeComplete
thread
:bookkeeper workers
thread
:client-server io
Since there are multiple threads that will trigger all successful callbacks in the pending queue, it may cause the following race condition[code-2]
thread-1
andthread-2
may be triggered by differentPendingAddOps
thread-1
thread-2
success
success
queue.pop
queue.pop
[1] code link: https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java#L307
[2] code-link: https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L2092-L2124
Issue-2: ledger will be closed with a incorrect length
Since the task that triggers all successful callbacks in the pending queue may be run in
IO
thread, the task "triggers all successful callbacks in the pending queue" and closing ledger may concurrectly executeworker-thread
io-thread
success
ledger.length
which was popped out from the queue[code-3]queue.pop
and pop nothingledger.LAC
The variables
ledger.LAC
andledger.length
do not match[3] code-link: https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L2076-L2084
The issue we encountered
A pulsar topic is stuck at
ClosingLedger
statepulsar topic stats
logs
Changes
Switch the thread to
Bookkeeper works
if the connection is broken.