-
Notifications
You must be signed in to change notification settings - Fork 3.7k
[fix] Fix ManagedLedgerImpl's pendingAddEntries leak. #25240
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?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -1652,6 +1652,8 @@ public synchronized void asyncClose(final CloseCallback callback, final Object c | |
|
|
||
| factory.close(this); | ||
| STATE_UPDATER.set(this, State.Closed); | ||
| executor.execute(() -> | ||
| clearNotInitiatedPendingAddEntries(new ManagedLedgerAlreadyClosedException("Managed ledger is closed"))); | ||
| cancelScheduledTasks(); | ||
|
|
||
| LedgerHandle lh = currentLedger; | ||
|
|
@@ -1751,20 +1753,32 @@ public void operationComplete(Void v, Stat stat) { | |
| log.debug("[{}] Updating of ledgers list after create complete. version={}", name, stat); | ||
| } | ||
| ledgersStat = stat; | ||
| synchronized (ManagedLedgerImpl.this) { | ||
| LedgerHandle originalCurrentLedger = currentLedger; | ||
| ledgers.put(lh.getId(), newLedger); | ||
| currentLedger = lh; | ||
| currentLedgerTimeoutTriggered = new AtomicBoolean(); | ||
| currentLedgerEntries = 0; | ||
| currentLedgerSize = 0; | ||
| updateLedgersIdsComplete(originalCurrentLedger); | ||
| mbean.addLedgerSwitchLatencySample(System.currentTimeMillis() | ||
| - lastLedgerCreationInitiationTimestamp, TimeUnit.MILLISECONDS); | ||
| // May need to update the cursor position | ||
| maybeUpdateCursorBeforeTrimmingConsumedLedger(); | ||
| // make sure that pendingAddEntries' operations are executed in the same thread | ||
| // to avoid potential concurrent issues | ||
| State state = STATE_UPDATER.get(ManagedLedgerImpl.this); | ||
| if (state == State.Closed || state.isFenced()) { | ||
| if (log.isDebugEnabled()) { | ||
| log.debug("[{}] Skipping ledger update after create complete because ledger is " | ||
| + "closed or fenced", name); | ||
| } | ||
| } else { | ||
| executor.execute(() -> { | ||
| synchronized (ManagedLedgerImpl.this) { | ||
| LedgerHandle originalCurrentLedger = currentLedger; | ||
| ledgers.put(lh.getId(), newLedger); | ||
| currentLedger = lh; | ||
| currentLedgerTimeoutTriggered = new AtomicBoolean(); | ||
| currentLedgerEntries = 0; | ||
| currentLedgerSize = 0; | ||
| updateLedgersIdsComplete(originalCurrentLedger); | ||
| mbean.addLedgerSwitchLatencySample(System.currentTimeMillis() | ||
| - lastLedgerCreationInitiationTimestamp, TimeUnit.MILLISECONDS); | ||
| // May need to update the cursor position | ||
| maybeUpdateCursorBeforeTrimmingConsumedLedger(); | ||
| } | ||
| metadataMutex.unlock(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please re-check STATE_UPDATER inside the executor task and bail out if Closed/Fenced (while still releasing metadataMutex). |
||
| }); | ||
| } | ||
| metadataMutex.unlock(); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -1937,6 +1951,9 @@ synchronized void ledgerClosed(final LedgerHandle lh, Long lastAddConfirmed) { | |
| // The managed ledger was closed during the write operation | ||
| clearPendingAddEntries(new ManagedLedgerAlreadyClosedException("Managed ledger was already closed")); | ||
| return; | ||
| } else if (state.isFenced()) { | ||
| clearPendingAddEntries(new ManagedLedgerFencedException("Managed ledger is fenced")); | ||
| return; | ||
| } else { | ||
| // In case we get multiple write errors for different outstanding write request, we should close the ledger | ||
| // just once | ||
|
|
@@ -2070,6 +2087,17 @@ void clearPendingAddEntries(ManagedLedgerException e) { | |
| } | ||
| } | ||
|
|
||
| void clearNotInitiatedPendingAddEntries(ManagedLedgerException e) { | ||
| Iterator<OpAddEntry> iterator = pendingAddEntries.iterator(); | ||
| while (iterator.hasNext()) { | ||
| OpAddEntry op = iterator.next(); | ||
| if (op.closeIfNotInitiated()) { | ||
| op.failed(e); | ||
| iterator.remove(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Here, perhaps |
||
| } | ||
| } | ||
zhaizhibo marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| void asyncReadEntries(OpReadEntry opReadEntry) { | ||
| final State state = STATE_UPDATER.get(this); | ||
| if (state.isFenced() || state == State.Closed) { | ||
|
|
@@ -4328,7 +4356,8 @@ public boolean isCursorActive(ManagedCursor cursor) { | |
| return activeCursors.get(cursor.getName()) != null; | ||
| } | ||
|
|
||
| private boolean currentLedgerIsFull() { | ||
| @VisibleForTesting | ||
| protected boolean currentLedgerIsFull() { | ||
| if (!factory.isMetadataServiceAvailable()) { | ||
| // We don't want to trigger metadata operations if we already know that we're currently disconnected | ||
| return false; | ||
|
|
@@ -4418,11 +4447,17 @@ private void checkManagedLedgerIsOpen() throws ManagedLedgerException { | |
| public synchronized void setFenced() { | ||
| log.info("{} Moving to Fenced state", name); | ||
| STATE_UPDATER.set(this, State.Fenced); | ||
| executor.execute(() -> clearNotInitiatedPendingAddEntries(new ManagedLedgerFencedException("ManagedLedger " | ||
| + name | ||
| + " is fenced"))); | ||
| } | ||
|
|
||
| synchronized void setFencedForDeletion() { | ||
| log.info("{} Moving to FencedForDeletion state", name); | ||
| STATE_UPDATER.set(this, State.FencedForDeletion); | ||
| executor.execute(() -> clearNotInitiatedPendingAddEntries(new ManagedLedgerFencedException("ManagedLedger " | ||
| + name | ||
| + " is fenced for deletion"))); | ||
| } | ||
|
|
||
| MetaStore getStore() { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The metadataMutex is acquired in updateLedgersListAfterRollover() and must always be released. In operationComplete(), the "Closed/Fenced" branch skips the ledger update but fails to call metadataMutex.unlock(), causing a lock leak that may deadlock subsequent metadata operations. Ensure unlock() is invoked in all branches, for example, by using a finally block.