This is an automated email from the ASF dual-hosted git repository. yong pushed a commit to branch branch-4.15 in repository https://gitbox.apache.org/repos/asf/bookkeeper.git
commit 63157d500129de33f86edc0fb2d81c1502336138 Author: StevenLuMT <[email protected]> AuthorDate: Wed Dec 7 17:30:17 2022 +0800 [fix][server]fix memory leak when closeRecovered,failed on clearing newEnsemblesFromRecovery (#3672) Co-authored-by: lushiji <[email protected]> (cherry picked from commit a19d849a370d0b3eed73bb90f329b47f00730146) --- .../org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java index 1d21fc9377..0f6f4d67aa 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java @@ -330,11 +330,14 @@ class ReadOnlyLedgerHandle extends LedgerHandle implements LedgerMetadataListene return builder.withClosedState().withLastEntryId(lac).withLength(len).build(); }, this::setLedgerMetadata).run(); - f.thenRun(() -> { - synchronized (metadataLock) { - newEnsemblesFromRecovery.clear(); - } - }); + f.whenComplete((result, exception) -> { + synchronized (metadataLock) { + newEnsemblesFromRecovery.clear(); + } + if (exception != null) { + LOG.error("When closeRecovered,failed on clearing newEnsemblesFromRecovery.", exception); + } + }); return f; }
