Skip to content

Conversation

@poorbarcode
Copy link
Contributor

@poorbarcode poorbarcode commented Apr 16, 2024

Thanks

This root cause was found by @codelipenghui and I just tried to reproduce and write this PR.

Motivation

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. See the explanation below.

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) ~[?:?]"

Explanation

Background: stacks of add entry

  1. BookieClientImpl.addEntry.
    a. Retain the ByteBuf before get PerChannelBookieClient. We call this ByteBuf as toSend in the following sections. toSend.recCnf is 2 now.
  2. Get PerChannelBookieClient
  3. ChannelReadyForAddEntryCallback.operationComplete
    a. PerChannelBookieClient.addEntry
    a-1. Build a new ByteBuf for request command. We call this ByteBuf new as request in the following sections.
    a-2. channle.writeAndFlush(request) or release the ByteBuf when channel is switching.
    b. Release the ByteBuf since it has been retained at step 1. toSend.recCnf should be 1 now.

(Highlight) the difference of V2 and V3 is only in the step "3-a-1. Build a new ByteBuf for request command"

1. The Step 3-a-1: V2 & Small payload

  • 3-a-1: Build a new ByteBuf for request command
    • request = toSend.retainedDuplicate().
  • 3-a-2: Release the ByteBuf
    • toSend.release()

Explanation: regarding this case, request and toSend share the same refCnf, it is fine to release a random one.

2. The Step 3-a-1: V2 & Large payload

  • 3-a-1: Build a new ByteBuf for request command
    • request = ByteBufList.clone(toSend).
  • 3-a-2: Release the ByteBuf
    • toSend.release()

Explanation: regarding this case, request and toSend are using different refCnf. Since the request can not be send out due to the channel switching, BK client should release request. It would cause a refCnf incorrect error if release an incorrect one. Highlight: since the internal ByteBufs of request and toSend are the same, it will not leading a memory leak even if it releases an incorrect one.

3. The Step 3-a-1: V3

  • 3-a-1: Build a new ByteBuf for request command
    • request: build a absolutely one new ByteBuf.
  • 3-a-2: Release the ByteBuf
    • toSend.release()

Explanation: regarding this case, request and toSend are using different refCnf. Since the request can not be send out due to the channel switching, BK client should release request. It would cause an refCnf incorrect error if releasing a incorrect one. Highlight: since the exactly ByteBuf of request and toSend are different, it will leading a memory leak if it releases an incorrect one.

Changes

Correct the code

final Channel c = channel;
if (c == null) {
// usually checked in writeAndFlush, but we have extra check
// because we need to release toSend.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

update also the comment

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

@lhotari
Copy link
Member

lhotari commented Apr 16, 2024

Please make the title more specific. for example "Fix ByteBuf release/retain in PerChannelBookClient" etc.

// because we need to release toSend.
errorOut(completionKey);
ReferenceCountUtil.release(toSend);
ReferenceCountUtil.release(request);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does this also result in the correct behavior when useV2WireProtocol == false?

It doesn't look consistent how the V3 protocol is currently handled since there's no retained duplicate and the backing Netty ByteBuf isn't released.
In the case of V3 protocol, the body might consist of com.google.protobuf.NioByteString instances that wrap the Nio ByteBuffer instances backed by Netty ByteBuf instances which also control the lifecycle of the Nio ByteBuffer instances.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It looks like the concern about the NioBuffer lifecycle was raised by @sijie already in the original PR review: #791 (comment) .

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added release at these locations

Copy link
Contributor Author

@poorbarcode poorbarcode Apr 16, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does this also result in the correct behavior when useV2WireProtocol == false?

No memory leak when useV2WireProtocol is false, but the error io.netty.util.IllegalReferenceCountException: refCnt: 0, increment: 1 described in the Motivation will also occur.

Copy link
Member

@horizonzy horizonzy left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice catch. LGTM

hezhangjian
hezhangjian previously approved these changes Apr 16, 2024
@hezhangjian hezhangjian dismissed their stale review April 16, 2024 06:17

may not retain in v3

@poorbarcode poorbarcode changed the title [fix] ByteBuf release/retain incorrect [fix] Fix ByteBuf release/retain in PerChannelBookClient Apr 16, 2024
@poorbarcode
Copy link
Contributor Author

Please make the title more specific. for example "Fix ByteBuf release/retain in PerChannelBookClient" etc.

Done

} catch (Throwable e) {
LOG.warn("Operation {} failed", StringUtils.requestToString(request), e);
errorOut(key);
if (!calledWrite) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a little weird. I think this catch wants to catch the channel is null for some reason. That means we may forget to release if the channel is null at line 1218. As I understand, 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.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Improved

@zymap zymap added the type/bug label Apr 16, 2024
@zymap zymap added this to the 4.18.0 milestone Apr 16, 2024
Copy link
Contributor

@hangc0276 hangc0276 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good job!

if (channel == null) {
LOG.warn("Operation {} failed: channel == null", StringUtils.requestToString(request));
errorOut(key);
ReferenceCountUtil.release(request);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it's the wrong thing to do generally in this method. I'll share a new PR where I explain an alternative approach that handles both the V3 protocol and the UnsafeByteOperations issue and the issue that was found.

Copy link
Contributor Author

@poorbarcode poorbarcode Apr 16, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To trace the context easier, I move the second comment here.

I think it's the wrong thing to do generally in this method. I'll share a new PR where I explain an alternative approach that handles both the V3 protocol and the UnsafeByteOperations issue and the issue that was found.

I don't think that the current changes in this PR are ok. I created #4293 to show what I think that should be done.

I added a section named Explanation in the Motivation, it explains that the current change is correct. Thanks for mentioning me and great suggestions ❤️

Copy link
Member

@lhotari lhotari left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think that the current changes in this PR are ok. I created #4293 to show what I think that should be done.

@poorbarcode poorbarcode requested a review from lhotari April 16, 2024 20:01
@poorbarcode
Copy link
Contributor Author

poorbarcode commented Apr 16, 2024

I don't think that the current changes in this PR are ok. I created #4293 to show what I think that should be done.

Replied this comment here

@lhotari
Copy link
Member

lhotari commented Apr 17, 2024

Replied this comment here

Thanks @poorbarcode . Let's merge this PR first. I'll then rebase #4293 to fix the invalid handling of the wrapped nio ByteBuffer lifecycle. That's a remaining problem in this area. It seems that it's not only a problem with V3 protocol.

@merlimat merlimat merged commit 1694d67 into apache:master Apr 17, 2024
merlimat pushed a commit that referenced this pull request Apr 17, 2024
* [fix] ByteBuf release/retain incorrect

* improve the code comment

* fix other cases

* modify the code comment

* improve the code

* improve the test

* add description
merlimat pushed a commit that referenced this pull request Apr 17, 2024
* [fix] ByteBuf release/retain incorrect

* improve the code comment

* fix other cases

* modify the code comment

* improve the code

* improve the test

* add description
@lhotari
Copy link
Member

lhotari commented Apr 22, 2024

Please review #4293. It addresses the remaining gaps in ByteBuf release/retain in PerChannelBookieClient that this PR
#4289 doesn't cover.

hezhangjian pushed a commit that referenced this pull request May 24, 2024
…4293)

### Motivation

This addresses the remaining gaps of #4289 in handling ByteBuf retain/release.
This PR will also address the concern about NioBuffer lifecycle brought up in the review of the original PR review: #791 (comment) .

This PR fixes several problems:
* ByteString buffer lifecycle in client, follows ByteBufList lifecycle
* ByteBufList lifecycle, moved to write promise
* Calling of write promises in AuthHandler which buffers messages while authentication is in progress. It was ignoring the promises.

### Changes

- add 2 callback parameters to writeAndFlush: cleanupActionFailedBeforeWrite and cleanupActionAfterWrite
  - use these callback actions for proper cleanup
- extract a utility class ByteStringUtil for wrapping ByteBufList or ByteBuf as concatenated zero copy ByteString
- properly handle releasing of ByteBufList in the write promise
- properly handle calling promises that are buffered while authentication is in progress
hezhangjian pushed a commit that referenced this pull request May 25, 2024
…4293)

### Motivation

This addresses the remaining gaps of #4289 in handling ByteBuf retain/release.
This PR will also address the concern about NioBuffer lifecycle brought up in the review of the original PR review: #791 (comment) .

This PR fixes several problems:
* ByteString buffer lifecycle in client, follows ByteBufList lifecycle
* ByteBufList lifecycle, moved to write promise
* Calling of write promises in AuthHandler which buffers messages while authentication is in progress. It was ignoring the promises.

### Changes

- add 2 callback parameters to writeAndFlush: cleanupActionFailedBeforeWrite and cleanupActionAfterWrite
  - use these callback actions for proper cleanup
- extract a utility class ByteStringUtil for wrapping ByteBufList or ByteBuf as concatenated zero copy ByteString
- properly handle releasing of ByteBufList in the write promise
- properly handle calling promises that are buffered while authentication is in progress

(cherry picked from commit 0ef2f99)
zhiheng123 pushed a commit to zhiheng123/bookkeeper that referenced this pull request May 26, 2024
…pache#4293)

This addresses the remaining gaps of apache#4289 in handling ByteBuf retain/release.
This PR will also address the concern about NioBuffer lifecycle brought up in the review of the original PR review: apache#791 (comment) .

This PR fixes several problems:
* ByteString buffer lifecycle in client, follows ByteBufList lifecycle
* ByteBufList lifecycle, moved to write promise
* Calling of write promises in AuthHandler which buffers messages while authentication is in progress. It was ignoring the promises.

- add 2 callback parameters to writeAndFlush: cleanupActionFailedBeforeWrite and cleanupActionAfterWrite
  - use these callback actions for proper cleanup
- extract a utility class ByteStringUtil for wrapping ByteBufList or ByteBuf as concatenated zero copy ByteString
- properly handle releasing of ByteBufList in the write promise
- properly handle calling promises that are buffered while authentication is in progress
lhotari added a commit to lhotari/bookkeeper that referenced this pull request May 27, 2024
…pache#4293)

### Motivation

This addresses the remaining gaps of apache#4289 in handling ByteBuf retain/release.
This PR will also address the concern about NioBuffer lifecycle brought up in the review of the original PR review: apache#791 (comment) .

This PR fixes several problems:
* ByteString buffer lifecycle in client, follows ByteBufList lifecycle
* ByteBufList lifecycle, moved to write promise
* Calling of write promises in AuthHandler which buffers messages while authentication is in progress. It was ignoring the promises.

### Changes

- add 2 callback parameters to writeAndFlush: cleanupActionFailedBeforeWrite and cleanupActionAfterWrite
  - use these callback actions for proper cleanup
- extract a utility class ByteStringUtil for wrapping ByteBufList or ByteBuf as concatenated zero copy ByteString
- properly handle releasing of ByteBufList in the write promise
- properly handle calling promises that are buffered while authentication is in progress

(cherry picked from commit 0ef2f99)

# Conflicts:
#	bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
hezhangjian pushed a commit that referenced this pull request May 27, 2024
…4293) (#4396)

### Motivation

This addresses the remaining gaps of #4289 in handling ByteBuf retain/release.
This PR will also address the concern about NioBuffer lifecycle brought up in the review of the original PR review: #791 (comment) .

This PR fixes several problems:
* ByteString buffer lifecycle in client, follows ByteBufList lifecycle
* ByteBufList lifecycle, moved to write promise
* Calling of write promises in AuthHandler which buffers messages while authentication is in progress. It was ignoring the promises.

### Changes

- add 2 callback parameters to writeAndFlush: cleanupActionFailedBeforeWrite and cleanupActionAfterWrite
  - use these callback actions for proper cleanup
- extract a utility class ByteStringUtil for wrapping ByteBufList or ByteBuf as concatenated zero copy ByteString
- properly handle releasing of ByteBufList in the write promise
- properly handle calling promises that are buffered while authentication is in progress

(cherry picked from commit 0ef2f99)

# Conflicts:
#	bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
* [fix] ByteBuf release/retain incorrect

* improve the code comment

* fix other cases

* modify the code comment

* improve the code

* improve the test

* add description
Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
…pache#4293)

### Motivation

This addresses the remaining gaps of apache#4289 in handling ByteBuf retain/release.
This PR will also address the concern about NioBuffer lifecycle brought up in the review of the original PR review: apache#791 (comment) .

This PR fixes several problems:
* ByteString buffer lifecycle in client, follows ByteBufList lifecycle
* ByteBufList lifecycle, moved to write promise
* Calling of write promises in AuthHandler which buffers messages while authentication is in progress. It was ignoring the promises.

### Changes

- add 2 callback parameters to writeAndFlush: cleanupActionFailedBeforeWrite and cleanupActionAfterWrite
  - use these callback actions for proper cleanup
- extract a utility class ByteStringUtil for wrapping ByteBufList or ByteBuf as concatenated zero copy ByteString
- properly handle releasing of ByteBufList in the write promise
- properly handle calling promises that are buffered while authentication is in progress
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

8 participants