This is an automated email from the ASF dual-hosted git repository.

yong pushed a commit to branch branch-4.16
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git

commit b519cf9053c8cd2718010fdb8b4628a1f183fd66
Author: houxiaoyu <[email protected]>
AuthorDate: Mon Jul 10 10:29:19 2023 +0800

    Recycle LongWrapper finally (#4007)
    
    (cherry picked from commit 528f373aac0afd5bfe71baa2922cbeab4fedf4ac)
---
 .../bookie/storage/ldb/LedgerMetadataIndex.java    | 52 ++++++++++++----------
 1 file changed, 29 insertions(+), 23 deletions(-)

diff --git 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java
 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java
index cf6cbfcdc2..235b554988 100644
--- 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java
+++ 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java
@@ -299,38 +299,44 @@ public class LedgerMetadataIndex implements Closeable {
     public void flush() throws IOException {
         LongWrapper key = LongWrapper.get();
 
-        int updatedLedgers = 0;
-        Entry<Long, LedgerData> entry;
-        while ((entry = pendingLedgersUpdates.poll()) != null) {
-            key.set(entry.getKey());
-            byte[] value = entry.getValue().toByteArray();
-            ledgersDb.put(key.array, value);
-            ++updatedLedgers;
-        }
+        try {
+            int updatedLedgers = 0;
+            Entry<Long, LedgerData> entry;
+            while ((entry = pendingLedgersUpdates.poll()) != null) {
+                key.set(entry.getKey());
+                byte[] value = entry.getValue().toByteArray();
+                ledgersDb.put(key.array, value);
+                ++updatedLedgers;
+            }
 
-        if (log.isDebugEnabled()) {
-            log.debug("Persisting updates to {} ledgers", updatedLedgers);
-        }
+            if (log.isDebugEnabled()) {
+                log.debug("Persisting updates to {} ledgers", updatedLedgers);
+            }
 
-        ledgersDb.sync();
-        key.recycle();
+            ledgersDb.sync();
+        } finally {
+            key.recycle();
+        }
     }
 
     public void removeDeletedLedgers() throws IOException {
         LongWrapper key = LongWrapper.get();
 
-        int deletedLedgers = 0;
-        for (Long ledgerId : pendingDeletedLedgers) {
-            key.set(ledgerId);
-            ledgersDb.delete(key.array);
-        }
+        try {
+            int deletedLedgers = 0;
+            for (Long ledgerId : pendingDeletedLedgers) {
+                key.set(ledgerId);
+                ledgersDb.delete(key.array);
+            }
 
-        if (log.isDebugEnabled()) {
-            log.debug("Persisting deletes of ledgers {}", deletedLedgers);
-        }
+            if (log.isDebugEnabled()) {
+                log.debug("Persisting deletes of ledgers {}", deletedLedgers);
+            }
 
-        ledgersDb.sync();
-        key.recycle();
+            ledgersDb.sync();
+        } finally {
+            key.recycle();
+        }
     }
 
     private ReentrantLock lockForLedger(long ledgerId) {

Reply via email to