Closed equanz closed 1 month ago
@equanz @hrsakai Thanks for reporting. Coincidentally I was investigating this today and had this concern but didn't validate it.
Just guessing, but perhaps the intention of this code is to select a different client for each partition when there are multiple consumers with the same name: https://github.com/apache/pulsar/blob/4f96146f13b136644a4eb0cf4ec36699e0431929/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java#L127
Slightly related issue about the incorrect results of getConsumerKeyHashRanges: #23321
The above case leads to out-of-order redelivery.
I don't think it will cause out-of-order issue.
PersistentStickyKeyDispatcherMultipleConsumers
Get a different consumer for the same hash is expected since the consumers are changed.
I don't think it will cause out-of-order issue.
- The ordering is not guaranteed by Hash Selector, is was guaranteed by the
PersistentStickyKeyDispatcherMultipleConsumers
- The newly joined consumer will only get messages after the old consumers have acknowledged all the outstanding messages
Get a different consumer for the same hash is expected since the consumers are changed.
The result of this bug is that the target consumer will switch also for existing consumers in certain cases. @codelipenghui Did you consider that case?
I'm pretty sure that consumerList.get(hash % consumerList.size())
is wrong. The fix is explained in https://github.com/apache/pulsar/issues/23321#issuecomment-2362826648 . I'll submit a PR.
The result of this bug is that the target consumer will switch also for existing consumers in certain cases. @codelipenghui Did you consider that case?
@lhotari Oh, I got your point for now. One consumer joined will cause the key assignment change for many other consumers. Thanks for the explanation.
It looks like #8396 wasn't a correct solution at the time it was made. @codelipenghui I think that we need to address this for all maintenance branches.
It looks like #8396 wasn't a correct solution at the time it was made. @codelipenghui I think that we need to address this for all maintenance branches.
I have created #23327 to fix the issue. Please review
Just guessing, but perhaps the intention of this code is to select a different client for each partition when there are multiple consumers with the same name: https://github.com/apache/pulsar/issues/23315#issuecomment-2360761086
The list is sorted by consumerName. https://github.com/apache/pulsar/blob/4f96146f13b136644a4eb0cf4ec36699e0431929/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java#L67-L73
The behavior when the keys to be compared are the same seems to be undefined. (I haven't rechecked it, but I think, in my environment, it was in order of addition.) https://docs.oracle.com/javase/8/docs/api/java/util/ArrayList.html#sort-java.util.Comparator-
So, as you say, each partition's selector could be different. However, if the producer uses a partition key in RoundRobin or SinglePartition routing mode, specific key messages are sent to a specific partition. If this is correct, it would be no issue on the key order guarantee. cf. https://pulsar.apache.org/docs/3.3.x/concepts-messaging/#ordering-guarantee
I don't think it will cause out-of-order issue. https://github.com/apache/pulsar/issues/23315#issuecomment-2361736557
The result of this bug is that the target consumer will switch also for existing consumers in certain cases. https://github.com/apache/pulsar/issues/23315#issuecomment-2361810469
One consumer joined will cause the key assignment change for many other consumers. https://github.com/apache/pulsar/issues/23315#issuecomment-2362842362
(Thank you @lhotari !) That is correct. My concern is "the range moves between existing consumers". In my understanding, this case is not currently addressed by the dispatcher.
Possibly the same issue in this Slack message: https://www.linen.dev/s/apache-pulsar/t/23079402/hi-we-re-using-key-shared-mode-the-key-is-a-code-between-100
Search before asking
Read release policy
Version
Tested with https://github.com/apache/pulsar/tree/4f96146f13b136644a4eb0cf4ec36699e0431929 .
Minimal reproduce step
Apply the following patches and run the test.
What did you expect to see?
In auto-split hash mode, we expect that the new consumer takes the hash range from existing consumers. (The dispatcher addresses the above case by recentlyJoinedConsumers.)
So, the range doesn't move between existing consumers.
What did you see instead?
When the hash range collides, the selector stores the consumer in the list of collisions. https://github.com/apache/pulsar/blob/4f96146f13b136644a4eb0cf4ec36699e0431929/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java#L67-L73
And, get the consumer by the following calculation. https://github.com/apache/pulsar/blob/4f96146f13b136644a4eb0cf4ec36699e0431929/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ConsistentHashingStickyKeyConsumerSelector.java#L127
4 % 3 = 1
, then return the consumer which has consumerId 1 (add new consumer which has consumerId 3)4 % 4 = 0
, then return the consumer which has consumerId 0 Consumers with consumerId of 0 and 1 are existing consumers. So, the range moves between existing consumers.The above case leads to out-of-order redelivery. Shouldn't we care about this?
Anything else?
For ease, I use the same name as the consumer in this example. However, this issue is caused not only by consumers of the same name but also by coincidence hash collisions.
(This issue was originally reported by @hrsakai .)
Are you willing to submit a PR?