BTW, I use 2.5.0. Also, I've tested with 1 consumer process (with 20 listener threads) for each subscription, then this inconsistency didn't happen. So, it seems like happening when the number of consumers for a subscription is changing during consuming.
On Fri, Mar 27, 2020 at 12:22 PM Hiroyuki Yamada <[email protected]> wrote: > > Hi, > > I found 2 consumers (with Key_Shared) with different subscriptions get > messages for the same key in an inconsistent order. > Roughly speaking about what happened is, > consumer 1 with a subscription "sub-1" get messages 1,2,3 for a key > "A" in this order, but > consumer 2 with a subscription "sub-2" get the messages for the key > "A" in the order of 1,3,2. > > So far, it happens when the number of consumers for a subscription is > changing during consuming, so > if the number of consumers is the same (stable) during consuming, this > seems not happening as far as I checked. > > I think it violates the ordering guarantees of Key_Shared so is it a bug ? > or there is no ordering guarantee for the same key between multiple > subscriptions ? > (I feel reordering of messages could happen but inconsistent order is an > issue) > Or am I doing something wrong ? > It would be great if anyone can help me. > > Here is how to reproduce. > > 0. Start pulsar > 1. create a partitioned topic named > "persistent://my-tenant/my-namespace/my-topic8" with the number of > partitions 8 > 2. get the code to reproduce > $ git clone https://github.com/feeblefakie/misc.git (note that the > code is recently updated) > $ cd /misc/pulsar > $ ./gradlew installDist > 3. Start a producer > $ build/install/pulsar/bin/my-producer > # A single thread producer produces <"0", "timestamp">, <"1", > "timestamp"> ... <"999", "timestamp">, <"0", "timestamp">, <"1", > "timestamp"> ... in this order (it produces 10000 messages in total > like this) > # the code is > https://github.com/feeblefakie/misc/blob/master/pulsar/src/main/java/MyProducer.java > . > 4. Start consumers with "subscription1" > $ build/install/pulsar/bin/my-consumer subscription1 > /tmp/sub1 > # starting 20 consumers one by one (so the number of consumers are > changing during consuming) > # the code is > https://github.com/feeblefakie/misc/blob/master/pulsar/src/main/java/MyConsumer.ava > . > 5. Start consumers with "subscription2" > $ build/install/pulsar/bin/my-consumer subscription2 > /tmp/sub2 > 6. Wait they consume all the messages > 7. Compare results to see inconsistencies between consumers for the > same key with different subscriptions > (in my case, for example, they are consistent for "100" "200" "400" > "500", "900", but inconsistent for "300", "600", "700", "800" so the > inconsistency is not rare) > $ grep " 300 " /tmp/sub1 (looking up messages with key "300" and it > shows "consumerName key value". note that the value is the timestamp > the message is produced at) > 60b52 300 1585274568287 > 60b52 300 1585274570859 > 60b52 300 1585274573087 > 60b52 300 1585274575273 <- the order is different (this seems correct) > 60b52 300 1585274577509 > 60b52 300 1585274579777 > 60b52 300 1585274582377 > 7c19e 300 1585274585053 > 7c19e 300 1585274587499 > 7c19e 300 1585274589994 > $ grep " 300 " /tmp/sub2 > 50898 300 1585274568287 > 50898 300 1585274570859 > 50898 300 1585274573087 > a6f2c 300 1585274587499 <- the order is different (this message is > produced a lot (14 seconds) after the next message, so it is pretty > weird) > 50898 300 1585274575273 > a6f2c 300 1585274589994 > 50898 300 1585274577509 > 50898 300 1585274579777 > 50898 300 1585274582377 > 50898 300 1585274585053 > > Thanks, > Hiroyuki
