This is an automated email from the ASF dual-hosted git repository.
chenhang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git
The following commit(s) were added to refs/heads/master by this push:
new 37b63c927a Fix check read failed entry memory leak issue. (#4513)
37b63c927a is described below
commit 37b63c927a8be55e17c3e31dded44d0f6221ce5a
Author: Yan Zhao <[email protected]>
AuthorDate: Wed May 28 10:07:27 2025 +0800
Fix check read failed entry memory leak issue. (#4513)
* Fix check read failed entry memory leak issue.
* address the comments.
---
.../java/org/apache/bookkeeper/replication/ReplicationWorker.java | 5 +++++
1 file changed, 5 insertions(+)
diff --git
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java
index c32487a30b..19248631bc 100644
---
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java
+++
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java
@@ -61,6 +61,7 @@ import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.client.BookKeeperAdmin;
import org.apache.bookkeeper.client.EnsemblePlacementPolicy;
import org.apache.bookkeeper.client.LedgerChecker;
+import org.apache.bookkeeper.client.LedgerEntry;
import org.apache.bookkeeper.client.LedgerFragment;
import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.client.api.LedgerMetadata;
@@ -351,6 +352,10 @@ public class ReplicationWorker implements Runnable {
lh.asyncReadEntries(entryIdToRead, entryIdToRead, (rc, ledHan,
seq, ctx) -> {
long thisEntryId = (Long) ctx;
if (rc == BKException.Code.OK) {
+ while (seq.hasMoreElements()) {
+ LedgerEntry entry = seq.nextElement();
+ entry.getEntryBuffer().release();
+ }
entriesUnableToReadForThisLedger.remove(thisEntryId);
if (numOfResponsesToWaitFor.decrementAndGet() == 0) {
multiReadComplete.countDown();