-
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
Shared subscription + Batch messages: consumerFlow trigger reading the same messages from storage #16421
Comments
The problem is here, when you use MessageListener we increase permits of 1, even for Messages that are part of a batch. pulsar/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java Line 1087 in 926834e
So the problem is mostly with the MessageListener API, used by pulsar-perf (and also OpenMessaging benchmark) |
with batch messages we fall into this case: Line 625 in 8b1d7a0
basically the dispatcher reads N entries from Storage (BK or Offloader) and the Consumer does not have enough permits (because we sent entries with multiple messages), so we are throwing away part of the entries that we read. Those entries are added to the list of "messages to replay" and so they will be read again from storage next time. |
After doing some testing I have found that the problem is that the ManagedLedger EntryCache is evicted only by taking into account the "readPosition" of the cursor. |
In order to solve the problem we should not evict from the cache the entries that are still in "messagesToReply", otherwise we will keep reading from BK (or from TieredStorage). |
I believe that this implementation about |
The issue had no activity for 30 days, mark with Stale label. |
I am doing some testing on Shared subscription and Batch messages with the current Pulsar master.
The behaviour that I am observing is that when you have Batch messages the Consumer is sending flow control messages for more messages that it can handle.
This is how to reproduce the problem:
consumerFlow
trigger the read of many messagesThis is happening because
consumerFlow
callsreadMoreEntries()
andreadMoreEntries()
sees that there are messages to be re-delivered, because the consumer still haven't acknowledged them.This is turn requests the ManagedCursor to read the data from storage.
I have observed this behaviour while working on offloader performances, but it also happens with regular BK based ledgers.
This simple test case reproduces the problem, I append it to this test
pulsar/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/CreateSubscriptionTest.java
Line 66 in 1ba180c
The text was updated successfully, but these errors were encountered: