poorbarcode opened a new pull request, #4140:
URL: https://github.com/apache/bookkeeper/pull/4140

   ### 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
   
   ```java
   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
   ```
   
   <img width="836" alt="Screenshot 2023-12-02 at 06 35 22" 
src="https://github.com/apache/bookkeeper/assets/25195800/fc607f85-0012-48ba-a90e-b383ef4f5881";>
   
   
   ### Changes
   
   - Fix the bug.
   - TODO: write a test
   


-- 
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]

Reply via email to