-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
Batch message serialization ignores batch size #9557
Comments
@Lanayx Sorry for the late response, I think #8285 is fixed this problem. Currently, we don't use the toByteArray method to do the seek operation. It is achieved by https://github.com/apache/pulsar/pull/8285/files#diff-f6e4c1c4091aa10525f331e48e66b29f22b9f7987755c1b4fb887e24f198bed6R1934-R1942. The fix is released at 2.6.2 and 2.7.0. What pulsar version you have seen this issue? |
Hi, @codelipenghui It seems you've misunderstood the issue :) My concern is that this line will give incorrect result if batched message id passed to the method had been taken from byte array (serialized and deserialized) since bath size is missing in byte representation of batched message id.
Rather than storing batch size, I would expect fixing that by changing this new seek logic so it won't depend on batch size. |
@Lanayx I see, thanks for the clarification. We will push a fix soon. |
@Lanayx I am trying to reproduce this issue, I am using the following case to test but everything looks good. Could you please take a look at the test case?
|
@zymap Hi! This test works, but not because the seek logic isn't broken, but because seek message id is stored as startMessageId in consumer and extra messages are ignored on receive. So basically if you have batch of 10000 messages and do seek to the last one of the batch all 10000 will be returned from broker, but 9999 will be ignored due to the startMessageId. My understanding is that this logic was supposed to serve for precise reset, so extra messages are not sent to the consumer. |
@Lanayx The broker does not do deserialization and filter the messages from a batch, this is expected behavior. And yes, this issue should be fixed since the consumer does not pass the right position to the broker, this will lead to the message acknowledgment issues(filtered messages by the client never get changes to ack, the mark delete position will stop to move forward) @zymap I think you can try to close the consumer after the seek operation and restart to consume messages, I think you will get the msg1 from your test. |
--- Fixes: apache#9557 **Motivation** Batch size does not include in the serialize and deserialize process.
--- Fixes: apache#9557 **Motivation** Batch size does not include in the serialize and deserialize process.
--- Fixes: apache#9557 **Motivation** Batch size does not include in the serialize and deserialize process.
--- Fixes: apache#9557 **Motivation** Batch size does not include in the serialize and deserialize process.
--- Fixes: #9557 **Motivation** Batch size does not include in the serialize and deserialize process.
--- Fixes: apache#9557 **Motivation** Batch size does not include in the serialize and deserialize process.
Describe the bug
Since batch index message acknowledgement was added the seek method was updated to support more precise seek using ack sets (source code). However when the seek is performed by the message that was serialized and deserialized, the batchSize is set to zero (since it is ignored during serialization), which leads to discrepancy between messageId forms and seek results.
To Reproduce
Compare BatchMessageIdImpl in runtime before and after serialization+deserialization, the batchSize will be always 0 in latter case. This leads to inconsistent behavior of Consumer seek by messageId
Expected behavior
There should be no difference between regular BatchMessageIdImpl and recovered from byteArray
The text was updated successfully, but these errors were encountered: