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 calculate checkSum when using Java9IntHash #4140

Merged
merged 8 commits into from
Dec 7, 2023

Conversation

poorbarcode
Copy link
Contributor

@poorbarcode poorbarcode commented Dec 1, 2023

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 calculate checksum. But it forgets to forward the offset of the ByteBuf.

https://github.com/apache/bookkeeper/blob/master/circe-checksum/src/main/java/com/scurrilous/circe/checksum/Java9IntHash.java#L110-L117

byte[] b = TL_BUFFER.get();
int toRead = len;
while (toRead > 0) {
    int length = Math.min(toRead, b.length);
    buffer.slice(offset, len).readBytes(b, 0, length); // Here, the variable `offset` never change.
    negCrc = resume(negCrc, b, 0, length);
    toRead -= length;
}

The log of issue:

2023-12-01T08:16:58,235+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
2023-12-01T08:16:58,241+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
2023-12-01T08:16:58,247+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
2023-12-01T08:16:58,254+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
2023-12-01T08:16:58,260+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
2023-12-01T08:16:58,266+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
2023-12-01T08:16:58,273+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
2023-12-01T08:16:58,279+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
2023-12-01T08:16:58,285+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
2023-12-01T08:16:58,292+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
2023-12-01T08:16:58,298+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
2023-12-01T08:16:58,304+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
2023-12-01T08:16:58,310+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
2023-12-01T08:16:58,317+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
2023-12-01T08:16:58,323+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
2023-12-01T08:16:58,329+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
2023-12-01T08:16:58,336+0000 [pulsar-io-8-5] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://x/x/x}] [pulsar.repl.c1-->c2] Failed to verify checksum
Screenshot 2023-12-02 at 06 35 22

Changes

  • Fix the bug.
  • Improve for the ByteBuf typed CompositeByteBuf

@merlimat
Copy link
Contributor

merlimat commented Dec 1, 2023

@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 CompositeByteBuf with more than 1 segment.

@poorbarcode
Copy link
Contributor Author

poorbarcode commented Dec 1, 2023

@merlimat I added an improvement for the ByteBuf typed CompositeByteBuf. Could you review this PR again?

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 CompositeByteBuf with more than 1 segment.

When I try to run the test with JDK-17, the method method_updateBytes.setAccessible(true) will get an error Unable to make private static int java.util.zip.CRC32C.updateBytes(int,byte[],int,int) accessible: module java.base does not "opens java.util.zip" to unnamed module.

I got the cause: if the package name of the new test is under com.scurrilous.circe.checksum, the error above is gone.

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) {
Copy link
Member

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? :)

Copy link
Contributor Author

@poorbarcode poorbarcode Dec 3, 2023

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));
Copy link
Member

@mattisonchao mattisonchao Dec 3, 2023

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?

Copy link
Contributor Author

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).

Copy link
Contributor Author

@poorbarcode poorbarcode Dec 3, 2023

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);
Copy link
Member

Choose a reason for hiding this comment

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

write bytes to b2?

Copy link
Contributor Author

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

Copy link
Member

@mattisonchao mattisonchao left a 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. :)

Copy link
Member

@mattisonchao mattisonchao left a 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;
}
}

Copy link
Member

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. :)

Copy link
Contributor Author

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;
}

Copy link
Contributor

@merlimat merlimat left a comment

Choose a reason for hiding this comment

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

👍

@poorbarcode poorbarcode closed this Dec 5, 2023
@poorbarcode poorbarcode reopened this Dec 5, 2023
@poorbarcode
Copy link
Contributor Author

Re-run tests

@TinyQAQ
Copy link

TinyQAQ commented Dec 5, 2023

I have patched this pr, but I still get this error
10.50.88.81 : 2023-12-05T20:11:37,849+0800 [pulsar-io-3-39] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://dc/dc_binlog_erp/binlog_dws_msku_trade_1h_hot_rf}] [mysql-10-50-35-146] Failed to verify checksum

@poorbarcode poorbarcode closed this Dec 5, 2023
@poorbarcode poorbarcode reopened this Dec 5, 2023
@poorbarcode
Copy link
Contributor Author

rerun failure checks

@zymap
Copy link
Member

zymap commented Dec 7, 2023

I have patched this pr, but I still get this error
10.50.88.81 : 2023-12-05T20:11:37,849+0800 [pulsar-io-3-39] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://dc/dc_binlog_erp/binlog_dws_msku_trade_1h_hot_rf}] [mysql-10-50-35-146] Failed to verify checksum

Is it the same case? @yubiao

@yubiao
Copy link

yubiao commented Dec 7, 2023

I have no clue. Did you @ the wrong person?

@merlimat merlimat merged commit be1749c into apache:master Dec 7, 2023
47 of 68 checks passed
merlimat pushed a commit that referenced this pull request Dec 7, 2023
* fix calculate checkSum when using Java9IntHash

* -

* improve performence

* add test

* edit test

* rename test

* fix license

* address comments
@merlimat
Copy link
Contributor

merlimat commented Dec 7, 2023

@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 (🙄 )

@hangc0276
Copy link
Contributor

@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.

@poorbarcode
Copy link
Contributor Author

@TinyQAQ @zymap

I have patched this pr, but I still get this error
10.50.88.81 : 2023-12-05T20:11:37,849+0800 [pulsar-io-3-39] ERROR org.apache.pulsar.broker.service.Producer - [PersistentTopic{topic=persistent://dc/dc_binlog_erp/binlog_dws_msku_trade_1h_hot_rf}] [mysql-10-50-35-146] Failed to verify checksum

Regarding this case, it was caused by another issue that will fixed by apache/pulsar#21684

yangl pushed a commit to yangl/bookkeeper that referenced this pull request Dec 11, 2023
* fix calculate checkSum when using Java9IntHash

* -

* improve performence

* add test

* edit test

* rename test

* fix license

* address comments
@lhotari
Copy link
Member

lhotari commented Jan 31, 2024

I found another bug in the checksum calculation which has existed for some time (not caused by this PR). Fixed by #4196

Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
* fix calculate checkSum when using Java9IntHash

* -

* improve performence

* add test

* edit test

* rename test

* fix license

* address comments
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.

10 participants