Skip to content
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

Open
eolivelli opened this issue Jul 6, 2022 · 6 comments
Labels
Stale triage/lhotari/important lhotari's triaging label for important issues or PRs type/bug The PR fixed a bug or issue reported a bug

Comments

@eolivelli
Copy link
Contributor

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:

  • write 100.000 messages using batching
  • start a Consumer with a Shared subscription (from the beginning of the topic)
  • you will see that the PersistentDispatcherMultipleConsumers consumerFlow trigger the read of many messages

This is happening because consumerFlow calls readMoreEntries() and readMoreEntries() 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

public class CreateSubscriptionTest extends ProducerConsumerBase {

  @Test
   public void testConsumerFlowOnSharedSubscription() throws Exception {
       String topic = "persistent://my-property/my-ns/topic" + UUID.randomUUID();
       admin.topics().createNonPartitionedTopic(topic);
       String subName = "my-sub";
       int numMessages = 20_000;
       final CountDownLatch count = new CountDownLatch(numMessages);
       try (Consumer<byte[]> consumer = pulsarClient.newConsumer()
               .subscriptionMode(SubscriptionMode.Durable)
               .subscriptionType(SubscriptionType.Shared)
               .topic(topic)
               .subscriptionName(subName)
               .messageListener(new MessageListener<byte[]>() {
                   @Override
                   public void received(Consumer<byte[]> consumer, Message<byte[]> msg) {
                       //log.info("received {} - {}", msg, count.getCount());
                       consumer.acknowledgeAsync(msg);
                       count.countDown();
                   }
               })
               .subscribe();
            Producer<byte[]> producer = pulsarClient
               .newProducer()
               .blockIfQueueFull(true)
               .enableBatching(true)
               .topic(topic)
                    .create()) {
           consumer.pause();
           byte[] message = "foo".getBytes(StandardCharsets.UTF_8);
           List<CompletableFuture<?>> futures = new ArrayList<>();
           for (int i = 0; i < numMessages; i++) {
               futures.add(producer.sendAsync(message).whenComplete( (id,e) -> {
                   if (e != null) {
                       log.error("error", e);
                   }
               }));
               if (futures.size() == 1000) {
                   FutureUtil.waitForAll(futures).get();
                   futures.clear();
               }
           }
           producer.flush();
           consumer.resume();
           assertTrue(count.await(20, TimeUnit.SECONDS));
       }

   }
@eolivelli eolivelli added the type/bug The PR fixed a bug or issue reported a bug label Jul 6, 2022
@eolivelli
Copy link
Contributor Author

The problem is here, when you use MessageListener we increase permits of 1, even for Messages that are part of a batch.

protected void callMessageListener(Message<T> msg) {

So the problem is mostly with the MessageListener API, used by pulsar-perf (and also OpenMessaging benchmark)

@eolivelli
Copy link
Contributor Author

eolivelli commented Jul 7, 2022

with batch messages we fall into this case:

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.

@eolivelli
Copy link
Contributor Author

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.
But the "messages to replay" are behind the "readPosition", so those entries are evicted from the cache

@eolivelli
Copy link
Contributor Author

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).

@eolivelli
Copy link
Contributor Author

I believe that this implementation about cacheEvictionByMarkDeletedPosition will also help with this problem, because it won't let the "messages to replay" to be evicted from the cache.

#14985

@github-actions
Copy link

github-actions bot commented Aug 7, 2022

The issue had no activity for 30 days, mark with Stale label.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Stale triage/lhotari/important lhotari's triaging label for important issues or PRs type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

No branches or pull requests

2 participants