-
Notifications
You must be signed in to change notification settings - Fork 907
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 calculate checkSum when using Java9IntHash #4140
Conversation
@poorbarcode Great catch! I was wondering how we did not catch in the tests and if would be easy to add a new test. I would assume we would have to run the test with JDK>=9 and pass a buffer which is not purely direct memory or heap array based, eg. a |
@merlimat I added an improvement for the ByteBuf typed
When I try to run the test with I got the cause: if the package name of the new test is under So, all things are right now. |
@@ -106,14 +107,22 @@ public int resume(int current, ByteBuf buffer, int offset, int len) { | |||
} else if (buffer.hasArray()) { | |||
int arrayOffset = buffer.arrayOffset() + offset; | |||
negCrc = resume(negCrc, buffer.array(), arrayOffset, len); | |||
} else if (buffer instanceof CompositeByteBuf) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why not use separate PR to improve this one? :)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It felt like the same location change, so did it together 😜
} else if (buffer instanceof CompositeByteBuf) { | ||
CompositeByteBuf compositeByteBuf = (CompositeByteBuf) buffer; | ||
for (int i = 0; i < compositeByteBuf.numComponents(); i ++) { | ||
negCrc = resume(~negCrc, compositeByteBuf.component(i)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why negate negCrc
here? do we have any compatibility issues with old data?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It calls resume(int, buffer)
here, and the method resume(int, buffer)
will do a negative again(see https://github.com/apache/bookkeeper/pull/4140/files#diff-26fdfdd08f57a3fef17453ae4f2c09a38c3c299b5580d71d352643f60ca04c44R103).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry, it is a bug(the test didn't cover it because here). I rewrote ~negCrc
to loopedCurrent
, the bug has been fixed and it is easier to read now.
ByteBuf b2 = ByteBufAllocator.DEFAULT.heapBuffer(3); | ||
b2.writeBytes(new byte[]{4,5,6}); | ||
ByteBuf b3 = ByteBufAllocator.DEFAULT.heapBuffer(hugeDataLen); | ||
b2.writeBytes(hugeData); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
write bytes to b2?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah, it is b3
. fixed
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry, I have some questions about CompositeByteBuf
optimize. :)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM now :)
negCrc = resume(negCrc, b, 0, length); | ||
toRead -= length; | ||
loopOffset += length; | ||
} | ||
} | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually, I want you to move the return to every branch, that wouldn't leave any trap in the future. :)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Moving return
to every logic branch would make ~
everywhere(just like below). So I think current implementation is better to read:
public int resume(int current, ByteBuf buffer, int offset, int len) {
if (buffer.hasMemoryAddress()) {
return ~resume(~current, buffer.memoryAddress(), offset, len);
}
if (buffer.hasArray()) {
int arrayOffset = buffer.arrayOffset() + offset;
return ~resume(~current, buffer.array(), arrayOffset, len);
}
if (buffer instanceof CompositeByteBuf) {
CompositeByteBuf compositeByteBuf = (CompositeByteBuf) buffer;
int loopedCurrent = current;
for (int i = 0; i < compositeByteBuf.numComponents(); i ++) {
loopedCurrent = resume(loopedCurrent, compositeByteBuf.component(i));
}
return loopedCurrent;
}
int negCrc = ~current;
byte[] b = TL_BUFFER.get();
int toRead = len;
int loopOffset = offset;
while (toRead > 0) {
int length = Math.min(toRead, b.length);
buffer.slice(loopOffset, length).readBytes(b, 0, length);
negCrc = resume(negCrc, b, 0, length);
toRead -= length;
loopOffset += length;
}
return ~negCrc;
}
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
👍
Re-run tests |
I have patched this pr, but I still get this error |
rerun failure checks |
Is it the same case? @yubiao |
I have no clue. Did you @ the wrong person? |
* fix calculate checkSum when using Java9IntHash * - * improve performence * add test * edit test * rename test * fix license * address comments
@poorbarcode @hangc0276 I cherry-picked into 4.16. There is no need to get it to 4.15 branch since this error was introduced in #3810 (🙄 ) |
@merlimat Sure, I will trigger a new release in the next week. |
Regarding this case, it was caused by another issue that will fixed by apache/pulsar#21684 |
* fix calculate checkSum when using Java9IntHash * - * improve performence * add test * edit test * rename test * fix license * address comments
I found another bug in the checksum calculation which has existed for some time (not caused by this PR). Fixed by #4196 |
* fix calculate checkSum when using Java9IntHash * - * improve performence * add test * edit test * rename test * fix license * address comments
Motivation
In the method
Java9IntHash.resume(int current, ByteBuf buffer, int offset, int len)
, It tries to read all the data batch an batch, and finally to calculatechecksum
. But it forgets to forward theoffset
of theByteBuf
.https://github.com/apache/bookkeeper/blob/master/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Java9IntHash.java#L110-L117
The log of issue:
Changes