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

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


The following commit(s) were added to refs/heads/branch-4.15 by this push:
     new 91800051b6 Fix the PendingAddOp is not recycled when LedgerHandler 
closed (#3321)
91800051b6 is described below

commit 91800051b6e6686d4b85b379daf62e9e12f20a65
Author: Yong Zhang <[email protected]>
AuthorDate: Wed Jun 8 18:02:32 2022 +0800

    Fix the PendingAddOp is not recycled when LedgerHandler closed (#3321)
    
    ---
    
    **Motivation**
    
    When adding an entry into a bookie, the entry data lifecycle
    is handled by the bookie client. The data buffer will be
    released after receiving a corresponding response from the
    bookie server. So the user doesn't care about the entry
    buffer releasing.
    But when the ledgerHandler is closed, the PendingAddOp is not
    recycled which leads to the data buffer never being released.
    We should release that after the callback executed.
    
    (cherry picked from commit 3d01e6e13cf5410e913859c5c92c002e986df518)
---
 .../src/main/java/org/apache/bookkeeper/client/LedgerHandle.java        | 2 ++
 .../src/main/java/org/apache/bookkeeper/client/PendingAddOp.java        | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)

diff --git 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
index 530b54c0f0..4d9090e589 100644
--- 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
+++ 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
@@ -1326,6 +1326,7 @@ public class LedgerHandle implements WriteHandle {
                         LOG.warn("Attempt to add to closed ledger: {}", 
ledgerId);
                         
op.cb.addCompleteWithLatency(BKException.Code.LedgerClosedException,
                                 LedgerHandle.this, INVALID_ENTRY_ID, 0, 
op.ctx);
+                        op.recyclePendAddOpObject();
                     }
 
                     @Override
@@ -1337,6 +1338,7 @@ public class LedgerHandle implements WriteHandle {
                 
op.cb.addCompleteWithLatency(BookKeeper.getReturnRc(clientCtx.getBookieClient(),
                                                                     
BKException.Code.InterruptedException),
                         LedgerHandle.this, INVALID_ENTRY_ID, 0, op.ctx);
+                op.recyclePendAddOpObject();
             }
             return;
         }
diff --git 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java
 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java
index 729d9fe077..223ce182f7 100644
--- 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java
+++ 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java
@@ -499,7 +499,7 @@ class PendingAddOp extends SafeRunnable implements 
WriteCallback {
         }
     }
 
-    private void recyclePendAddOpObject() {
+    public void recyclePendAddOpObject() {
         entryId = LedgerHandle.INVALID_ENTRY_ID;
         currentLedgerLength = -1;
         if (payload != null) {

Reply via email to