Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fix checksum calculation bug when the payload is a CompositeByteBuf with readerIndex > 0 #4196

Merged
merged 39 commits into from
Feb 7, 2024

Conversation

lhotari
Copy link
Member

@lhotari lhotari commented Jan 31, 2024

Motivation

There's a checksum calculation bug when the payload is a CompositeByteBuf with readerIndex > 0.
This bug is demonstrated in the included repro test case in commit 9fb47f5.

The problematic solution to unwrap CompositeByteBuf was introduced in PRs #2701.
PR #2701 was added to resolve a problem described in this Apache Pulsar PR: apache/pulsar#10330 . "... BookKeeper's checksum logic, the nioBuffer() method will be called and the CompositeByteBuf's internal buffers will be concatenated into a single buffer in heap memory..."

Changes

Revert the unsafe ways to "unwrap" Netty ByteBuf instances and access CompositeByteBuf directly without taking the readerIndex state into account. This approach was introduced in PRs #2701.

Netty doesn't have a public API for visiting the sub buffers, but there is a way to do this using the "ByteBuf.getBytes" method. That is the approach used in this PR.

The reason to access the underlying buffers is to ensure that no extra copies will have to be made when calculating the checksum. This PR doesn't cause performance regressions, but improves performance since there were code paths in DigestManager where the previous buggy solution for checksum calculation optimization wasn't used.

@lhotari
Copy link
Member Author

lhotari commented Jan 31, 2024

@hangc0276 @poorbarcode @BewareMyPower Please review

@lhotari
Copy link
Member Author

lhotari commented Jan 31, 2024

@lhotari lhotari force-pushed the lh-fix-compositebuffer-checksum branch from cfecfe4 to c19ae5d Compare January 31, 2024 17:29
Copy link
Contributor

@BewareMyPower BewareMyPower left a comment

Choose a reason for hiding this comment

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

How about this implementation?

    final int update(int digest, ByteBuf buffer, int offset, int len) {
        if (buffer instanceof CompositeByteBuf) {
            int readerIndex = buffer.readerIndex();
            CompositeByteBuf compositeByteBuf = (CompositeByteBuf) buffer;
            for (int i = 0; i < compositeByteBuf.numComponents(); i++) {
                ByteBuf component = compositeByteBuf.component(i);
                int readableBytes = component.writerIndex() - readerIndex;
                if (readableBytes > 0) {
                    digest = internalUpdate(digest, component, readerIndex, readableBytes);
                    readerIndex = 0;
                } else {
                    readerIndex -= component.writerIndex();
                }
            }
            return digest;
        } else {
            return internalUpdate(digest, buffer, offset, len);
        }
    }

@lhotari
Copy link
Member Author

lhotari commented Feb 1, 2024

How about this implementation?

@BewareMyPower The readerIndex state that is lost is in the wrappers. In many cases, there could be multiple layers of wrappers and the main issue is that there's no way to determine whether it's ok or not to call "unwrap". In a lot of cases, it's not ok to call "unwrap".

It's possible to use the test case in this PR and do attempts in the context of the test case to see what really happens with different approaches. The test case doesn't have all of the possible problem cases. I think that I'll add a scenario for "slice" too.

I'm fairly confident that the approach in the PR is one of rare safe ways to do "internal iteration" for ByteBuf as the Netty maintainers call it. I got some high level feedback from Netty maintainers about the approach: netty/netty#13820 . This is a gap in Netty 4. In Netty 5, there's a nice API to traverse all components.

@lhotari
Copy link
Member Author

lhotari commented Feb 1, 2024

@BewareMyPower I have refactored the solution to use a stateless visitor callback implementation where the state is passed around in a method call parameter. The benefit of this is that it reduces unnecessary object allocations. I'm not concerned about that specificly, but object allocations by callbacks was brought up in the discussion with the Netty maintainer. That's why I decided to handle that aspect too.

@lhotari
Copy link
Member Author

lhotari commented Feb 1, 2024

There's currently a failing test with heap buffers (byte[] backed buffer) and this PR is still WIP.

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

LGTM

great work

// skip visiting empty buffers
return;
}
InternalContext<T> internalContext = new InternalContext<>();
Copy link
Contributor

Choose a reason for hiding this comment

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

visitBuffers is a high-frequency call. Can we reuse the InternalContext to reduce the object allocation?

Copy link
Member Author

Choose a reason for hiding this comment

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

The reason to have that InternalContext is to enable sharing of all other instances. There will be only a single instance for each root level buffer so object allocation isn't a problem.

Copy link
Member Author

Choose a reason for hiding this comment

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

This PR removes some allocations that were happening in the old solution so there won't be additional allocations caused by this PR. One example of removed allocations is the reduced use of slice. When you call slice on a Netty ByteBuf, it will create an instance. The solution will be passing a byte[] or a direct buffer to the checksum calculations and the existing solutions to extract to use slice to get the buffer content won't be used any more.

For example this has been creating a lot of objects when CRC32 is used:

// Fallback to data copy if buffer is not contiguous
byte[] b = new byte[length];
buf.getBytes(index, b, 0, length);
crcValue = (int) updateBytes.invoke(null, crcValue, b, 0, b.length);

Similarly Java9IntHash was using slice here:

buffer.slice(loopOffset, length).readBytes(b, 0, length);

JniIntHash here:

return hash.resume(current, buffer.slice(offset, len).nioBuffer());

All of this is now optimized and won't end up using this code since ByteBufVisitor will be extracting (or copying) byte[] or direct buffers (memory address) without any object allocations. Previously there has been object allocations.

Copy link
Contributor

Choose a reason for hiding this comment

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

I see. This PR won't introduce more object allocation. I'm not blocking this PR, and it will be perfect if we can reduce the InternalContext object allocation. For each entry, it will create two InternalContext objects.

} else if (callback.acceptsMemoryAddress(callbackContext) && visitBuffer.isDirect()
&& visitBuffer.alloc().isDirectBufferPooled()) {
// read-only buffers need to be copied before they can be directly accessed
ByteBuf copyBuffer = visitBuffer.copy(visitIndex, visitLength);
Copy link
Contributor

Choose a reason for hiding this comment

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

We will copy the ByteBuf here, will this path will high-frequency called?

Copy link
Member Author

Choose a reason for hiding this comment

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

When it gets here, the source buffer (visitBuffer) will need to be copied in any case.
Since this checks that the source buffer is a direct buffer and that the callback accepts a memory address buffer (direct buffer), it's better to make a direct copy than to use the thread local copy buffer (byte[]) for copying.

The last resort is copying the source buffer using the byte[] copying.

Netty won't allow visiting the backing direct buffer or backing array when the buffer is a read-only buffer. In all other cases, the ByteBufVisitor is able to visit all source buffers without copies. This is the most efficient solution that there is available.

@hangc0276
Copy link
Contributor

Do we need a CPU and object alloc profile to see if this change introduces any performance issues?

@lhotari
Copy link
Member Author

lhotari commented Feb 5, 2024

Do we need a CPU and object alloc profile to see if this change introduces any performance issues?

We can do that, however it is very likely that this improves performance instead of causing a regression.

Another important detail is that there's a clear bug in correctness and we need to get rid of the current unwrapping solution introduced in #2701. I don't think that it's acceptable leaving it as it is.

@lhotari
Copy link
Member Author

lhotari commented Feb 5, 2024

This issue may also be related ? apache/pulsar#21892 I got some exceptions and tracked it for quite some time @lhotari

14:29:19.961 [BookKeeperClientWorker-OrderedExecutor-2-0] ERROR org.apache.bookkeeper.common.util.SingleThreadExecutor - Error while running task: readerIndex: 31215, writerIndex: 21324 (expected: 0 <= readerIndex <= writerIndex <= capacity(65536))
java.lang.IndexOutOfBoundsException: readerIndex: 31215, writerIndex: 21324 (expected: 0 <= readerIndex <= writerIndex <= capacity(65536))
	at io.netty.buffer.AbstractByteBuf.checkIndexBounds(AbstractByteBuf.java:112) ~[io.netty-netty-buffer-4.1.104.Final.jar:4.1.104.Final]
	at io.netty.buffer.AbstractByteBuf.setIndex(AbstractByteBuf.java:144) ~[io.netty-netty-buffer-4.1.104.Final.jar:4.1.104.Final]
	at io.netty.buffer.DuplicatedByteBuf.<init>(DuplicatedByteBuf.java:56) ~[io.netty-netty-buffer-4.1.104.Final.jar:4.1.104.Final]
	at io.netty.buffer.DuplicatedByteBuf.<init>(DuplicatedByteBuf.java:42) ~[io.netty-netty-buffer-4.1.104.Final.jar:4.1.104.Final]
	at io.netty.buffer.UnpooledDuplicatedByteBuf.<init>(UnpooledDuplicatedByteBuf.java:24) ~[io.netty-netty-buffer-4.1.104.Final.jar:4.1.104.Final]
	at io.netty.buffer.AbstractPooledDerivedByteBuf$PooledNonRetainedDuplicateByteBuf.<init>(AbstractPooledDerivedByteBuf.java:164) ~[io.netty-netty-buffer-4.1.104.Final.jar:4.1.104.Final]
	at io.netty.buffer.AbstractPooledDerivedByteBuf.duplicate0(AbstractPooledDerivedByteBuf.java:157) ~[io.netty-netty-buffer-4.1.104.Final.jar:4.1.104.Final]
	at io.netty.buffer.PooledSlicedByteBuf.duplicate(PooledSlicedByteBuf.java:118) ~[io.netty-netty-buffer-4.1.104.Final.jar:4.1.104.Final]
	at io.netty.buffer.CompositeByteBuf$Component.duplicate(CompositeByteBuf.java:1947) ~[io.netty-netty-buffer-4.1.104.Final.jar:4.1.104.Final]
	at io.netty.buffer.CompositeByteBuf.component(CompositeByteBuf.java:1556) ~[io.netty-netty-buffer-4.1.104.Final.jar:4.1.104.Final]
	at org.apache.bookkeeper.proto.checksum.DigestManager.computeDigestAndPackageForSendingV2(DigestManager.java:149) ~[bookkeeper-server-4.17.0-SNAPSHOT.jar:4.17.0-SNAPSHOT]
	at org.apache.bookkeeper.proto.checksum.DigestManager.computeDigestAndPackageForSending(DigestManager.java:106) ~[bookkeeper-server-4.17.0-SNAPSHOT.jar:4.17.0-SNAPSHOT]
	at org.apache.bookkeeper.client.PendingAddOp.initiate(PendingAddOp.java:246) ~[bookkeeper-server-4.17.0-SNAPSHOT.jar:4.17.0-SNAPSHOT]
	at org.apache.bookkeeper.client.LedgerHandle.doAsyncAddEntry(LedgerHandle.java:1363) ~[bookkeeper-server-4.17.0-SNAPSHOT.jar:4.17.0-SNAPSHOT]
	at org.apache.bookkeeper.client.LedgerHandle.asyncAddEntry(LedgerHandle.java:1061) ~[bookkeeper-server-4.17.0-SNAPSHOT.jar:4.17.0-SNAPSHOT]
	at org.apache.bookkeeper.mledger.impl.OpAddEntry.initiate(OpAddEntry.java:144) ~[org.apache.pulsar-managed-ledger-3.3.0-SNAPSHOT.jar:3.3.0-SNAPSHOT]
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.internalAsyncAddEntry(ManagedLedgerImpl.java:862) ~[org.apache.pulsar-managed-ledger-3.3.0-SNAPSHOT.jar:3.3.0-SNAPSHOT]
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.lambda$asyncAddEntry$3(ManagedLedgerImpl.java:794) ~[org.apache.pulsar-managed-ledger-3.3.0-SNAPSHOT.jar:3.3.0-SNAPSHOT]
	at org.apache.bookkeeper.common.util.SingleThreadExecutor.safeRunTask(SingleThreadExecutor.java:137) ~[org.apache.bookkeeper-bookkeeper-common-4.16.3.jar:4.16.3]
	at org.apache.bookkeeper.common.util.SingleThreadExecutor.run(SingleThreadExecutor.java:107) ~[org.apache.bookkeeper-bookkeeper-common-4.16.3.jar:4.16.3]
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[io.netty-netty-common-4.1.104.Final.jar:4.1.104.Final]
	at java.lang.Thread.run(Thread.java:833) ~[?:?]

@graysonzeng This looks like a multi-threading / thread-safety issue. Possibly multiple threads are mutating the same Netty ByteBuf instance. ByteBuf isn't thread safe. It might not be related to the problem that this PR is fixing.

@merlimat merlimat added this to the 4.17.0 milestone Feb 7, 2024
@merlimat merlimat merged commit 9c373f7 into apache:master Feb 7, 2024
16 checks passed
lhotari added a commit to lhotari/bookkeeper that referenced this pull request Feb 8, 2024
…ith readerIndex > 0 (apache#4196)

* Add a test that reproduces a bug in checksum calculation

* Revert "Fixed unnecessary copy to heap (apache#2701)" changes to ByteBufList

This partially reverts commit 3c9c710.

* Remove CompositeBuffer unwrapping in DigestManager

* Rename update -> internalUpdate so that unwrapping logic could be added to update

* Remove unnecessary unwrapping logic in Java9IntHash

* Add safe way to handle CompositeByteBuf

* Add license header

* Fix checkstyle

* Refactor ByteBuf visitor solution

* Fix checkstyle

* Reformat

* Refactor recursive visiting

* Revisit equals, hashCode and toString

* Refactor test case

* Add support for UnpooledHeapByteBuf.getBytes which passes an array

* Add support for visiting buffers backed by byte[] arrays

- getBytes calls setBytes with a byte[] argument for
  heap ByteBufs

* Move ByteBufVisitor to org.apache.bookkeeper.util package

* Update javadoc

* Refactor to use stateless visitor so that instance can be shared

* Improve test so that a single scenario can be used for debugging

* Fix bug in Java9IntHash calculation that assumed crc32c_update(x) == ~crc32c_update(~x)

- Java9IntHash uses private methods from java.util.zip.CRC32C class,
  updateBytes and updateDirectByteBuffer.
  When inspecting the use and interface contract, it doesn't match
  how it is used in Java9IntHash. This PR addresses that by introducing
  a separate initial value for initializing the accumulated value
  so that the initial value could match the logic in
  java.util.zip.CRC32C.reset method. There's also a separate
  method for finalizing the accumulated value into a final
  checksum value. This is to match the java.util.zip.CRC32C.getValue
  method's logic (uses bitwise complement operator ~).

- With a quick glance, it might appear that the previous logic is similar.
  However it isn't since I have a failing test which gets fixed with this
  change. I haven't yet added the Java9IntHash level unit test case to prove how
  it differs. It must be related to integer value overflow. For the CRC32C function,
  I believe it means that it cannot be assumed in all cases that
  func(x) == ~func(~x). That's the assumption that the previous code was making.
  It probably applies for many inputs, but not all. It would break in overflow
  cases.

* Fix checkstyle

* Fix checkstyle

* Fix missing depth increment that prevents StackOverflowException

* Properly handle the depth increase and decrease

* Remove unnecessary condition

* Use more efficient way to read bytes to the target array

* Don't use ByteBufVisitor if it's not necessary

* Revert "Fix bug in Java9IntHash calculation that assumed crc32c_update(x) == ~crc32c_update(~x)"

This reverts commit 272e962.

* Fix issue in resume byte[] version that was added

- input and output should be complemented. explanation has been added to the
  resume ByteBuf method

* Polish ByteBufVisitor

- reuse GetBytesCallbackByteBuf instance for handling the root ByteBuf instance

* Use extracted method

* Fix bug with array handling

* Polish ByteBufVisitor

* Optimize the buffer copying in the case where array or memory address cannot be accessed

- read-only buffers will need to be copied before reading
  - use ByteBuf.copy for direct buffers with pooled allocator when the algorithm can accept
    a memory address buffer
- use the 64kB threadlocal byte[] buffer for copying all other inputs

* Check if memory address is accepted

* Improve comments about complement (current = ~current) in resume

* Print thread dump when build is cancelled

* Filter empty buffers and arrays in ByteBufVisitor

(cherry picked from commit 9c373f7)
merlimat pushed a commit that referenced this pull request Feb 9, 2024
…ith readerIndex > 0 (#4196) (#4205)

* Add a test that reproduces a bug in checksum calculation

* Revert "Fixed unnecessary copy to heap (#2701)" changes to ByteBufList

This partially reverts commit 3c9c710.

* Remove CompositeBuffer unwrapping in DigestManager

* Rename update -> internalUpdate so that unwrapping logic could be added to update

* Remove unnecessary unwrapping logic in Java9IntHash

* Add safe way to handle CompositeByteBuf

* Add license header

* Fix checkstyle

* Refactor ByteBuf visitor solution

* Fix checkstyle

* Reformat

* Refactor recursive visiting

* Revisit equals, hashCode and toString

* Refactor test case

* Add support for UnpooledHeapByteBuf.getBytes which passes an array

* Add support for visiting buffers backed by byte[] arrays

- getBytes calls setBytes with a byte[] argument for
  heap ByteBufs

* Move ByteBufVisitor to org.apache.bookkeeper.util package

* Update javadoc

* Refactor to use stateless visitor so that instance can be shared

* Improve test so that a single scenario can be used for debugging

* Fix bug in Java9IntHash calculation that assumed crc32c_update(x) == ~crc32c_update(~x)

- Java9IntHash uses private methods from java.util.zip.CRC32C class,
  updateBytes and updateDirectByteBuffer.
  When inspecting the use and interface contract, it doesn't match
  how it is used in Java9IntHash. This PR addresses that by introducing
  a separate initial value for initializing the accumulated value
  so that the initial value could match the logic in
  java.util.zip.CRC32C.reset method. There's also a separate
  method for finalizing the accumulated value into a final
  checksum value. This is to match the java.util.zip.CRC32C.getValue
  method's logic (uses bitwise complement operator ~).

- With a quick glance, it might appear that the previous logic is similar.
  However it isn't since I have a failing test which gets fixed with this
  change. I haven't yet added the Java9IntHash level unit test case to prove how
  it differs. It must be related to integer value overflow. For the CRC32C function,
  I believe it means that it cannot be assumed in all cases that
  func(x) == ~func(~x). That's the assumption that the previous code was making.
  It probably applies for many inputs, but not all. It would break in overflow
  cases.

* Fix checkstyle

* Fix checkstyle

* Fix missing depth increment that prevents StackOverflowException

* Properly handle the depth increase and decrease

* Remove unnecessary condition

* Use more efficient way to read bytes to the target array

* Don't use ByteBufVisitor if it's not necessary

* Revert "Fix bug in Java9IntHash calculation that assumed crc32c_update(x) == ~crc32c_update(~x)"

This reverts commit 272e962.

* Fix issue in resume byte[] version that was added

- input and output should be complemented. explanation has been added to the
  resume ByteBuf method

* Polish ByteBufVisitor

- reuse GetBytesCallbackByteBuf instance for handling the root ByteBuf instance

* Use extracted method

* Fix bug with array handling

* Polish ByteBufVisitor

* Optimize the buffer copying in the case where array or memory address cannot be accessed

- read-only buffers will need to be copied before reading
  - use ByteBuf.copy for direct buffers with pooled allocator when the algorithm can accept
    a memory address buffer
- use the 64kB threadlocal byte[] buffer for copying all other inputs

* Check if memory address is accepted

* Improve comments about complement (current = ~current) in resume

* Print thread dump when build is cancelled

* Filter empty buffers and arrays in ByteBufVisitor

(cherry picked from commit 9c373f7)
@StevenLuMT
Copy link
Member

Good work

shoothzj pushed a commit that referenced this pull request May 24, 2024
### Motivation

ByteBufVisitor added in #4196 contains some code that isn't used and covered by unit tests. 
It's better to remove such code since it makes it harder to reason about the solution. The ByteBufVisitor solution will unwrap all direct buffers that can be unwrapped without the code that is to be removed. I believe that I forgot this code from some earlier phase of the solution where it was necessary to include this.
I ran some local tests and didn't see that the code was used for the original purpose that it was added for (supporting read-only buffers).

### Changes

Remove the unused code.
shoothzj pushed a commit that referenced this pull request May 25, 2024
### Motivation

ByteBufVisitor added in #4196 contains some code that isn't used and covered by unit tests.
It's better to remove such code since it makes it harder to reason about the solution. The ByteBufVisitor solution will unwrap all direct buffers that can be unwrapped without the code that is to be removed. I believe that I forgot this code from some earlier phase of the solution where it was necessary to include this.
I ran some local tests and didn't see that the code was used for the original purpose that it was added for (supporting read-only buffers).

### Changes

Remove the unused code.

(cherry picked from commit 84fd255)
shoothzj pushed a commit that referenced this pull request May 25, 2024
### Motivation

ByteBufVisitor added in #4196 contains some code that isn't used and covered by unit tests.
It's better to remove such code since it makes it harder to reason about the solution. The ByteBufVisitor solution will unwrap all direct buffers that can be unwrapped without the code that is to be removed. I believe that I forgot this code from some earlier phase of the solution where it was necessary to include this.
I ran some local tests and didn't see that the code was used for the original purpose that it was added for (supporting read-only buffers).

### Changes

Remove the unused code.

(cherry picked from commit 84fd255)
Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
…ith readerIndex > 0 (apache#4196)

* Add a test that reproduces a bug in checksum calculation

* Revert "Fixed unnecessary copy to heap (apache#2701)" changes to ByteBufList

This partially reverts commit 3c9c710.

* Remove CompositeBuffer unwrapping in DigestManager

* Rename update -> internalUpdate so that unwrapping logic could be added to update

* Remove unnecessary unwrapping logic in Java9IntHash

* Add safe way to handle CompositeByteBuf

* Add license header

* Fix checkstyle

* Refactor ByteBuf visitor solution

* Fix checkstyle

* Reformat

* Refactor recursive visiting

* Revisit equals, hashCode and toString

* Refactor test case

* Add support for UnpooledHeapByteBuf.getBytes which passes an array

* Add support for visiting buffers backed by byte[] arrays

- getBytes calls setBytes with a byte[] argument for
  heap ByteBufs

* Move ByteBufVisitor to org.apache.bookkeeper.util package

* Update javadoc

* Refactor to use stateless visitor so that instance can be shared

* Improve test so that a single scenario can be used for debugging

* Fix bug in Java9IntHash calculation that assumed crc32c_update(x) == ~crc32c_update(~x)

- Java9IntHash uses private methods from java.util.zip.CRC32C class,
  updateBytes and updateDirectByteBuffer.
  When inspecting the use and interface contract, it doesn't match
  how it is used in Java9IntHash. This PR addresses that by introducing
  a separate initial value for initializing the accumulated value
  so that the initial value could match the logic in
  java.util.zip.CRC32C.reset method. There's also a separate
  method for finalizing the accumulated value into a final
  checksum value. This is to match the java.util.zip.CRC32C.getValue
  method's logic (uses bitwise complement operator ~).

- With a quick glance, it might appear that the previous logic is similar.
  However it isn't since I have a failing test which gets fixed with this
  change. I haven't yet added the Java9IntHash level unit test case to prove how
  it differs. It must be related to integer value overflow. For the CRC32C function,
  I believe it means that it cannot be assumed in all cases that
  func(x) == ~func(~x). That's the assumption that the previous code was making.
  It probably applies for many inputs, but not all. It would break in overflow
  cases.

* Fix checkstyle

* Fix checkstyle

* Fix missing depth increment that prevents StackOverflowException

* Properly handle the depth increase and decrease

* Remove unnecessary condition

* Use more efficient way to read bytes to the target array

* Don't use ByteBufVisitor if it's not necessary

* Revert "Fix bug in Java9IntHash calculation that assumed crc32c_update(x) == ~crc32c_update(~x)"

This reverts commit 272e962.

* Fix issue in resume byte[] version that was added

- input and output should be complemented. explanation has been added to the
  resume ByteBuf method

* Polish ByteBufVisitor

- reuse GetBytesCallbackByteBuf instance for handling the root ByteBuf instance

* Use extracted method

* Fix bug with array handling

* Polish ByteBufVisitor

* Optimize the buffer copying in the case where array or memory address cannot be accessed

- read-only buffers will need to be copied before reading
  - use ByteBuf.copy for direct buffers with pooled allocator when the algorithm can accept
    a memory address buffer
- use the 64kB threadlocal byte[] buffer for copying all other inputs

* Check if memory address is accepted

* Improve comments about complement (current = ~current) in resume

* Print thread dump when build is cancelled

* Filter empty buffers and arrays in ByteBufVisitor
Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
### Motivation

ByteBufVisitor added in apache#4196 contains some code that isn't used and covered by unit tests. 
It's better to remove such code since it makes it harder to reason about the solution. The ByteBufVisitor solution will unwrap all direct buffers that can be unwrapped without the code that is to be removed. I believe that I forgot this code from some earlier phase of the solution where it was necessary to include this.
I ran some local tests and didn't see that the code was used for the original purpose that it was added for (supporting read-only buffers).

### Changes

Remove the unused code.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

8 participants