poorbarcode opened a new pull request, #4289: URL: https://github.com/apache/bookkeeper/pull/4289
### Motivation This problem was found by @codelipenghui and I just tried to reproduce and write this PR. `PerChannelBookieClient.addEntry` duplicates the binary data (`@param toSend`) and sends the binary data new to the IO out. Manually release the binary data new if can not be sent out due to the channel switching. **Issue:** it released the original data (`@param toSend`), it is wrong. Leading a memory leak and the bellow error log ``` "Apr 10, 2024 @ 16:06:49.893","2024-04-10T16:06:49,892+0000 [BookKeeperClientWorker-OrderedExecutor-0-0] ERROR org.apache.bookkeeper.common.util.SingleThreadExecutor - Error while running task: refCnt: 0, increment: 1" "Apr 10, 2024 @ 16:06:49.893","io.netty.util.IllegalReferenceCountException: refCnt: 0, increment: 1" "Apr 10, 2024 @ 16:06:49.893"," at io.netty.util.internal.ReferenceCountUpdater.retain(ReferenceCountUpdater.java:120) ~[io.netty-netty-common-4.1.100.Final.jar:4.1.100.Final]" "Apr 10, 2024 @ 16:06:49.893"," at io.netty.util.AbstractReferenceCounted.retain(AbstractReferenceCounted.java:61) ~[io.netty-netty-common-4.1.100.Final.jar:4.1.100.Final]" "Apr 10, 2024 @ 16:06:49.893"," at io.netty.util.internal.ReferenceCountUpdater.retain0(ReferenceCountUpdater.java:133) ~[io.netty-netty-common-4.1.100.Final.jar:4.1.100.Final]" "Apr 10, 2024 @ 16:06:49.893"," at org.apache.bookkeeper.util.ByteBufList.retain(ByteBufList.java:61) ~[org.apache.bookkeeper-bookkeeper-server-4.16.3.jar:4.16.3]" "Apr 10, 2024 @ 16:06:49.893"," at org.apache.bookkeeper.util.ByteBufList.retain(ByteBufList.java:281) ~[org.apache.bookkeeper-bookkeeper-server-4.16.3.jar:4.16.3]" "Apr 10, 2024 @ 16:06:49.893"," at org.apache.bookkeeper.proto.BookieClientImpl.addEntry(BookieClientImpl.java:307) ~[org.apache.bookkeeper-bookkeeper-server-4.16.3.jar:4.16.3]" "Apr 10, 2024 @ 16:06:49.893"," at org.apache.bookkeeper.client.PendingAddOp.sendWriteRequest(PendingAddOp.java:150) ~[org.apache.bookkeeper-bookkeeper-server-4.16.3.jar:4.16.3]" "Apr 10, 2024 @ 16:06:49.893"," at org.apache.bookkeeper.client.PendingAddOp.initiate(PendingAddOp.java:258) ~[org.apache.bookkeeper-bookkeeper-server-4.16.3.jar:4.16.3]" "Apr 10, 2024 @ 16:06:49.893"," at org.apache.bookkeeper.client.LedgerHandle.doAsyncAddEntry(LedgerHandle.java:1358) ~[org.apache.bookkeeper-bookkeeper-server-4.16.3.jar:4.16.3]" "Apr 10, 2024 @ 16:06:49.893"," at org.apache.bookkeeper.client.LedgerHandle.asyncAddEntry(LedgerHandle.java:1056) ~[org.apache.bookkeeper-bookkeeper-server-4.16.3.jar:4.16.3]" "Apr 10, 2024 @ 16:06:49.893"," at org.apache.bookkeeper.mledger.impl.OpAddEntry.initiate(OpAddEntry.java:144) ~[io.streamnative-managed-ledger-3.0.2.1.jar:3.0.2.1]" "Apr 10, 2024 @ 16:06:49.893"," at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.internalAsyncAddEntry(ManagedLedgerImpl.java:863) ~[io.streamnative-managed-ledger-3.0.2.1.jar:3.0.2.1]" "Apr 10, 2024 @ 16:06:49.893"," at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.lambda$asyncAddEntry$2(ManagedLedgerImpl.java:779) ~[io.streamnative-managed-ledger-3.0.2.1.jar:3.0.2.1]" "Apr 10, 2024 @ 16:06:49.893"," at org.apache.bookkeeper.common.util.SingleThreadExecutor.safeRunTask(SingleThreadExecutor.java:137) ~[org.apache.bookkeeper-bookkeeper-common-4.16.3.jar:4.16.3]" "Apr 10, 2024 @ 16:06:49.893"," at org.apache.bookkeeper.common.util.SingleThreadExecutor.run(SingleThreadExecutor.java:113) ~[org.apache.bookkeeper-bookkeeper-common-4.16.3.jar:4.16.3]" "Apr 10, 2024 @ 16:06:49.893"," at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[io.netty-netty-common-4.1.100.Final.jar:4.1.100.Final]" "Apr 10, 2024 @ 16:06:49.893"," at java.lang.Thread.run(Thread.java:840) ~[?:?]" ``` ### Changes Correct the code -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
