Skip to content

Conversation

@TakaHiR07
Copy link
Contributor

@TakaHiR07 TakaHiR07 commented Nov 10, 2023

Fixes #21551

Motivation

Fix topic compaction is failed after compactedLedger's all quorum is being recover, which is described in issue.

Modifications

When CompactedTopicImpl try to open compactedLedger, use asyncOpenLedgerNoRecovery instead of asyncOpenLedger. Then compactedLedger can watch the zk node change and update the ledger metadata in broker.

Does this pull request potentially affect one of the following parts:

If the box was checked, please highlight the changes

  • Dependencies (add or upgrade a dependency)
  • The public API
  • The schema
  • The default values of configurations
  • The threading model
  • The binary protocol
  • The REST endpoints
  • The admin CLI options
  • The metrics
  • Anything that affects deployment

Documentation

  • doc
  • doc-required
  • doc-not-needed
  • doc-complete

Matching PR in forked repository

PR in forked repository: TakaHiR07#19

@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label Nov 10, 2023
Copy link
Contributor

@codelipenghui codelipenghui left a comment

Choose a reason for hiding this comment

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

@TakaHiR07 But the new solution will introduce a data inconsistent issue?

Copy link
Member

@zymap zymap left a comment

Choose a reason for hiding this comment

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

As you described in the issue, it seems the ledger doesn't replicate successfully to the new ensemble. The metadata is still the old one:

Read of ledger entry failed: L1705537 E0-E0, Sent to [bookie-1, bookie-2, bookie-3], Heard from [] : bitset = {}, Error = 'Bookie handle is not available'. First unread entry is (-1, rc = null)

It is still trying to read the old ensembles.

asyncOpenLedgerNoRecovery is not recommended because it could receive different responses on the different bookies.
For example, when you write 3 bookies and you need 2 bookie ack but only receive one ack. Using this API can read that entry from one bookie but can not read it from the another the failed write bookie. The write client know it's a failed write. But the read client can read it.

@TakaHiR07
Copy link
Contributor Author

As you described in the issue, it seems the ledger doesn't replicate successfully to the new ensemble. The metadata is still the old one:

Read of ledger entry failed: L1705537 E0-E0, Sent to [bookie-1, bookie-2, bookie-3], Heard from [] : bitset = {}, Error = 'Bookie handle is not available'. First unread entry is (-1, rc = null)

It is still trying to read the old ensembles.

asyncOpenLedgerNoRecovery is not recommended because it could receive different responses on the different bookies. It violates the bookie's write consistent policy. For example, when you write 3 bookies and you need 2 bookie ack but only receive one ack. Using this API can read that entry from one bookie but can not read it from the another the failed write bookie. The write client know it's a failed write. But the read client can read it.

Replicate of auto-recovery is success, metadata on zk is the new one, but metadata on CompactTopicContext#ReadLedgerHandle is the old one. And I can read ledger directly from bookie-shell.

I guess compactedLedger is closed ledger, it should not be write and read at the same time. Therefore the read client should not be able to read the failed write entry ??

And I also have a question here. Why asyncOpenLedger not register the zk listener to update metadata change?

@zymap
Copy link
Member

zymap commented Nov 24, 2023

openLedger is a frequent operation, If every openLedger operation registers the listener, that would be a huge number.
If one broker can not communicate with zookeeper and doesn't close the compacted ledger, then a new broker opens the ledger. That would take some unexpected things.

@Technoboy- Technoboy- added this to the 3.3.0 milestone Dec 22, 2023
@yebai1105
Copy link

I have the same problem, is there any new progress?

@coderzc coderzc modified the milestones: 3.3.0, 3.4.0 May 8, 2024
@lhotari lhotari modified the milestones: 4.0.0, 4.1.0 Oct 11, 2024
@TakaHiR07
Copy link
Contributor Author

TakaHiR07 commented Feb 6, 2025

This problem has not been fixed and I hope to continue the review of this pr.

Firstly, I think the data inconsistent issue of asyncOpenLedgerNoRecovery is not exist.

  • Because we can read the compactedLedger until ledger become Closed. There is no write client and read client exist at the same time. Just as the following code show, we close the ledger success and then do reader.acknowledgeCumulativeAsync, then COMPACTED_TOPIC_LEDGER_PROPERTY would be changed to the specific ledgerId, and then we asyncOpenLedgerNoRecovery the compactedLedger and the compactedTopicContext would be created.
  • If the write client not finish, we can not create compactedTopicContext and can not create the read client.

private CompletableFuture<Long> phaseTwoSeekThenLoop(RawReader reader, MessageId from,
MessageId to,
MessageId lastReadId, Map<String, MessageId> latestForKey, BookKeeper bk,
LedgerHandle ledger) {
CompletableFuture<Long> promise = new CompletableFuture<>();
reader.seekAsync(from).thenCompose((v) -> {
Semaphore outstanding = new Semaphore(MAX_OUTSTANDING);
CompletableFuture<Void> loopPromise = new CompletableFuture<>();
phaseTwoLoop(reader, to, latestForKey, ledger, outstanding, loopPromise, MessageId.earliest);
return loopPromise;
}).thenCompose((v) -> closeLedger(ledger))
.thenCompose((v) -> reader.acknowledgeCumulativeAsync(lastReadId,
Map.of(COMPACTED_TOPIC_LEDGER_PROPERTY, ledger.getId())))
.whenComplete((res, exception) -> {
if (exception != null) {
deleteLedger(bk, ledger).whenComplete((res2, exception2) -> {
if (exception2 != null) {
log.warn("Cleanup of ledger {} for failed", ledger, exception2);
}
// complete with original exception
promise.completeExceptionally(exception);
});
} else {
promise.complete(ledger.getId());
}
});
return promise;
}

Secondly, we only register listener for compactedLedger, the listener number is not huge.

@codelipenghui @zymap @codelipenghui @yebai1105 @coderzc

@lhotari
Copy link
Member

lhotari commented Aug 27, 2025

possible related: apache/bookkeeper#4613

@lhotari lhotari requested a review from poorbarcode August 27, 2025 08:44
@nodece nodece self-requested a review August 29, 2025 03:13
@coderzc coderzc modified the milestones: 4.1.0, 4.2.0 Sep 1, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

doc-not-needed Your PR changes do not impact docs triage/lhotari/important lhotari's triaging label for important issues or PRs

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[Bug][broker] topic compaction is failed after compactedLedger 3 quorum all being recover in bookie autoRecovery

7 participants