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

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


The following commit(s) were added to refs/heads/branch-4.16 by this push:
     new a77458d56c [fix] Fix ByteBuf release/retain in PerChannelBookClient 
(#4289)
a77458d56c is described below

commit a77458d56cf22e3b79d5beded8dbc0beca3ff22c
Author: fengyubiao <[email protected]>
AuthorDate: Thu Apr 18 06:37:29 2024 +0800

    [fix] Fix ByteBuf release/retain in PerChannelBookClient (#4289)
    
    * [fix] ByteBuf release/retain incorrect
    
    * improve the code comment
    
    * fix other cases
    
    * modify the code comment
    
    * improve the code
    
    * improve the test
    
    * add description
---
 .../org/apache/bookkeeper/proto/PerChannelBookieClient.java   | 11 ++++++++---
 .../java/org/apache/bookkeeper/test/BookieClientTest.java     |  3 ++-
 2 files changed, 10 insertions(+), 4 deletions(-)

diff --git 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
index b4cf194e24..3a7af5b99a 100644
--- 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
+++ 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
@@ -841,10 +841,10 @@ public class PerChannelBookieClient extends 
ChannelInboundHandlerAdapter {
                                                    cb, ctx, ledgerId, 
entryId));
         final Channel c = channel;
         if (c == null) {
-            // usually checked in writeAndFlush, but we have extra check
-            // because we need to release toSend.
+            // Manually release the binary data(variable "request") that we 
manually created when it can not be sent out
+            // because the channel is switching.
             errorOut(completionKey);
-            ReferenceCountUtil.release(toSend);
+            ReferenceCountUtil.release(request);
             return;
         } else {
             // addEntry times out on backpressure
@@ -1136,6 +1136,7 @@ public class PerChannelBookieClient extends 
ChannelInboundHandlerAdapter {
         if (channel == null) {
             LOG.warn("Operation {} failed: channel == null", 
StringUtils.requestToString(request));
             errorOut(key);
+            ReferenceCountUtil.release(request);
             return;
         }
 
@@ -1150,6 +1151,7 @@ public class PerChannelBookieClient extends 
ChannelInboundHandlerAdapter {
                     StringUtils.requestToString(request));
 
             errorOut(key, BKException.Code.TooManyRequestsException);
+            ReferenceCountUtil.release(request);
             return;
         }
 
@@ -1171,6 +1173,9 @@ public class PerChannelBookieClient extends 
ChannelInboundHandlerAdapter {
         } catch (Throwable e) {
             LOG.warn("Operation {} failed", 
StringUtils.requestToString(request), e);
             errorOut(key);
+            // If the request goes into the writeAndFlush, it should be 
handled well by Netty. So all the exceptions we
+            // get here, we can release the request.
+            ReferenceCountUtil.release(request);
         }
     }
 
diff --git 
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java
 
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java
index a110e833ac..f5202d8c0e 100644
--- 
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java
+++ 
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java
@@ -22,7 +22,6 @@ package org.apache.bookkeeper.test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
 import io.netty.buffer.UnpooledByteBufAllocator;
@@ -36,6 +35,7 @@ import java.util.Arrays;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
+import lombok.extern.slf4j.Slf4j;
 import org.apache.bookkeeper.bookie.MockUncleanShutdownDetection;
 import org.apache.bookkeeper.bookie.TestBookieImpl;
 import org.apache.bookkeeper.client.BKException;
@@ -69,6 +69,7 @@ import org.junit.Test;
 /**
  * Test the bookie client.
  */
+@Slf4j
 public class BookieClientTest {
     BookieServer bs;
     File tmpDir;

Reply via email to