Skip to content
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

[fix] Write stuck due to pending add callback by multiple threads #4557

Open
wants to merge 2 commits into
base: master
Choose a base branch
from

Conversation

poorbarcode
Copy link
Contributor

@poorbarcode poorbarcode commented Feb 20, 2025

Motivation

Background: the normal steps of adding an entry

  • Gains BKC's IO thread
  • Triggers the callback after gaining the IO thread
    • Write data to servers
  • Triggers PendingAddOp.writeComplete after receiving the response from BK servers.
  • Triggers all succeed callbacks in the pending queue

Background: the steps of disconnection

  • Gains BKC's IO thread
  • Triggers a failed callback after gaining the IO thread
    • Triggers a failed PendingAddOp.writeComplete. You can reproduce this flow by the new test testAddEntriesCallbackWithBKClientThread
    • (Highlight) If the writing is already complete, this process will also trigger all successful callbacks in the pending queue even if the current writing is failed[code-1]

Issue: write stuck due to pending add callback by multiple threads

  • Settings
    • Ensembles: 3
    • WriteQuoram: 3
    • AckQuoram: 2
steps write entries client->BK1 client->BK2 client-> BK3
1 write BKs
2 start writing start writing start writing
3 write success
ack: 1/3
write success
ack: 2/3
4 mark the writing as complete since ack quorum is 2/3
5 connected and disconnected
6 Triggers a failed PendingAddOp.writeComplete
8 thread: bookkeeper workers thread: client-server io
7 Trigger all succeed callbacks in the pending queue Trigger all succeed callbacks in the pending queue[code-1]

Since there are multiple threads that will trigger all successful callbacks in the pending queue, it may cause the following race condition[code-2]

  • BTW, the thread-1 and thread-2 may be triggered by different PendingAddOps
steps thread-1 thread-2
1 peek pending addOp from the queue peek pending addOp from the queue
2 check it is the first item: success check it is the first item: success
3 call queue.pop call queue.pop
4 Issue: the second OP will never get a callback triggering

[1] code link: https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java#L307

// PendingAddOp.writeComplete
public synchronized void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) {
        if (completed) {
            sendAddSuccessCallbacks();
            maybeRecycle();
            return;
}

[2] code-link: https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L2092-L2124

    // LedgerHandle.sendAddSuccessCallbacks
    void sendAddSuccessCallbacks() {
        while ((pendingAddOp = pendingAddOps.peek()) != null && !changingEnsemble) {
            if (!pendingAddOp.completed) {
                return;
            }
            pendingAddOps.remove();
            explicitLacFlushPolicy.updatePiggyBackedLac(lastAddConfirmed);
            pendingAddsSequenceHead = pendingAddOp.entryId;
            pendingAddOp.submitCallback(BKException.Code.OK);
        }
    }

The issue we encountered

A pulsar topic is stuck at ClosingLedger state

pulsar topic stats

{
  "entriesAddedCounter" : 12485917,
  "numberOfEntries" : 126383,
  "totalSize" : 53668291,
  "currentLedgerEntries" : 137418,
  "currentLedgerSize" : 58374388,
  "lastLedgerCreatedTimestamp" : "2025-02-06T09:13:45.371Z",
  "waitingCursorsCount" : 1,
  "pendingAddEntriesCount" : 8342123,
  "lastConfirmedEntry" : "41901:126416",
  "state" : "ClosingLedger",
  "ledgers" : [ {
    "ledgerId" : 41901,
    "entries" : 0,
    "size" : 0,
    "offloaded" : false,
    "underReplicated" : false
  } ],
  "cursors" : {
...

logs

2025-02-06T09:14:06,421+0000 [BookKeeperClientWorker-OrderedExecutor-2-0] WARN  org.apache.bookkeeper.client.PendingAddOp - Failed to write entry (41901, 126417) to bookie (2, bookie-0:3181): Bookie handle is not available"
2025-02-06T09:14:06,421+0000 [BookKeeperClientWorker-OrderedExecutor-2-0] WARN  org.apache.bookkeeper.client.PendingAddOp - Failed to write entry (41901, 126412) to bookie (2, bookie-0:3181): Bookie handle is not available"
2025-02-06T09:14:06,421+0000 [pulsar-io-3-5] INFO  org.apache.bookkeeper.proto.PerChannelBookieClient - Successfully connected to bookie: bookie-0:3181 [id: 0xbc663b9b, L:/10.247.4.112:59578 - R:bookie-0/10.247.4.38:3181]"
2025-02-06T09:14:06,421+0000 [pulsar-io-3-5] INFO  org.apache.bookkeeper.proto.PerChannelBookieClient - connection [id: 0xbc663b9b, L:/10.247.4.112:59578 - R:bookie-0/10.247.4.38:3181] authenticated as BookKeeperPrincipal{ANONYMOUS}"
2025-02-06T09:14:06,423+0000 [BookKeeperClientWorker-OrderedExecutor-2-0] INFO  org.apache.bookkeeper.client.LedgerHandle - New Ensemble: [bookie-3:3181, bookie-2:3181, bookie-1:3181] for ledger: 41901"
2025-02-06T09:14:06,423+0000 [pulsar-io-3-5] WARN  org.apache.bookkeeper.proto.PerChannelBookieClient - Exception caught on:[id: 0xbc663b9b, L:/10.247.4.112:59578 - R:bookie-0/10.247.4.38:3181] cause: recvAddress(..) failed: Connection reset by peer"
2025-02-06T09:14:06,423+0000 [pulsar-io-3-5] INFO  org.apache.bookkeeper.proto.PerChannelBookieClient - Disconnected from bookie channel [id: 0xbc663b9b, L:/10.247.4.112:59578 ! R:bookie-0/10.247.4.38:3181]"
2025-02-06T09:14:06,423+0000 [pulsar-io-3-6] WARN  org.apache.bookkeeper.mledger.impl.OpAddEntry - [{topic}] The add op is terminal legacy callback for entry 41901-126416 adding."
2025-02-06T09:14:06,423+0000 [pulsar-io-3-6] WARN  org.apache.bookkeeper.client.PendingAddOp - Failed to write entry (41901, 126426) to bookie (2, bookie-1:3181): Bookie handle is not available"
2025-02-06T09:14:06,423+0000 [BookKeeperClientWorker-OrderedExecutor-3-0] WARN  org.apache.bookkeeper.client.BookieWatcherImpl - replaceBookie for bookie: bookie-1:3181 in ensemble: [bookie-3:3181, bookie-2:3181, bookie-1:3181] is not adhering to placement policy and chose bookie-0:3181. excludedBookies [bookie-1:3181] and quarantinedBookies [bookie-2:3181]"

Changes

Switch the thread to Bookkeeper works if the connection is broken.

@lhotari
Copy link
Member

lhotari commented Feb 20, 2025

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.

@poorbarcode
Copy link
Contributor Author

rerun failure checks

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.

2 participants