[fix]Fix entry loss due to incorrect lock of LedgerHandle#4701
[fix]Fix entry loss due to incorrect lock of LedgerHandle#4701zymap merged 3 commits intoapache:masterfrom
Conversation
| } | ||
| } | ||
| handle = LedgerDescriptor.create(masterKey, ledgerId, ledgerStorage); | ||
| ledgers.putIfAbsent(ledgerId, handle); |
There was a problem hiding this comment.
Looks like we can avoid the synchronization by handling the result of the putIfAbsent? I understand the main issue is that here we create two LedgerDescriptor, then used by different threads. But the putIfAbsent will reduce one. So we can handle the result to ensure the same handle is returned?
There was a problem hiding this comment.
Improved, please review again
There was a problem hiding this comment.
Pull request overview
This PR fixes a race condition in HandleFactoryImpl.getHandle() that could lead to entry loss when multiple threads simultaneously create a LedgerDescriptor for the same ledger. The issue occurs when both the write thread and the recovery thread attempt to create a new descriptor concurrently, potentially breaking the synchronization lock that prevents fencing races.
Changes:
- Implemented double-checked locking pattern in
getHandle()to ensure only oneLedgerDescriptoris created per ledger ID - Synchronized on the
ledgersmap during descriptor creation to prevent race conditions
💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/HandleFactoryImpl.java
Outdated
Show resolved
Hide resolved
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/HandleFactoryImpl.java
Outdated
Show resolved
Hide resolved
bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/HandleFactoryImpl.java
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
Pull request overview
Copilot reviewed 1 out of 1 changed files in this pull request and generated no new comments.
💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.
|
rerun failure checks |
1 similar comment
|
rerun failure checks |
* [fix]Fix entry loss due to incorrect lock of LedgerHandle (cherry picked from commit 44607a0)
Motivation
Background
HandleFactoryImpl, which maintains the ledger state.fencedBookieHighPriorityThreadBookieWriteThreadPoolTo avoid the following multi-threading competition issue,BK uses a lock[1]
(BookieWriteThreadPool)Add entry(BookieHighPriorityThread)Opening ledger with recoveryLedgerDescriptorfromHandleFactoryImplnon-fencedLedgerDescriptorfromHandleFactoryImpl0-1synchronized(LedgerDescriptor.this)Issue: there is a race condition breaks the lock above
https://github.com/apache/bookkeeper/blob/release-4.17.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/HandleFactoryImpl.java#L54-L68
(BookieWriteThreadPool)Add entry(BookieHighPriorityThread)Opening ledger with recoveryLedgerDescriptorfromHandleFactoryImplLedgerDescriptorfromHandleFactoryImplfalsefalseChanges
Fix the bug