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

Fixed dispatcher skipping delivery of a batch during concurrent replays #5499

Merged
merged 1 commit into from
Oct 30, 2019
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -300,14 +300,16 @@ public void readMoreEntries() {

}

Set<PositionImpl> messagesToReplayNow = getMessagesToReplayNow(messagesToRead);

if (!messagesToReplayNow.isEmpty()) {
if (havePendingReplayRead) {
if (havePendingReplayRead) {
if (log.isDebugEnabled()) {
log.debug("[{}] Skipping replay while awaiting previous read to complete", name);
return;
}
return;
}

Set<PositionImpl> messagesToReplayNow = getMessagesToReplayNow(messagesToRead);

if (!messagesToReplayNow.isEmpty()) {
if (log.isDebugEnabled()) {
log.debug("[{}] Schedule replay of {} messages for {} consumers", name, messagesToReplayNow.size(),
consumerList.size());
Expand Down Expand Up @@ -453,7 +455,6 @@ public synchronized void readEntriesComplete(List<Entry> entries, Object ctx) {
}

protected void sendMessagesToConsumers(ReadType readType, List<Entry> entries) {

if (entries == null || entries.size() == 0) {
return;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -208,4 +208,67 @@ public void testEverythingFilteredInMultipleReads()
assertTrue(receivedMsgs.contains("msg-" + i));
}
}

@Test
public void testDelayedDeliveryWithMultipleConcurrentReadEntries()
throws Exception {
String topic = "persistent://public/default/testDelayedDelivery-" + System.nanoTime();

@Cleanup
Consumer<String> consumer = pulsarClient.newConsumer(Schema.STRING)
.topic(topic)
.subscriptionName("shared-sub")
.subscriptionType(SubscriptionType.Shared)
.receiverQueueSize(1) // Use small prefecthing to simulate the multiple read batches
.subscribe();

// Simulate race condition with high frequency of calls to dispatcher.readMoreEntries()
PersistentDispatcherMultipleConsumers d = (PersistentDispatcherMultipleConsumers) ((PersistentTopic) pulsar
.getBrokerService().getTopicReference(topic).get()).getSubscription("shared-sub").getDispatcher();
Thread t = new Thread(() -> {
while (true) {
synchronized (d) {
d.readMoreEntries();
}

try {
Thread.sleep(1);
} catch (InterruptedException e) {
return;
}
}
});
t.start();

@Cleanup
Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
.topic(topic)
.create();

final int N = 1000;

for (int i = 0; i < N; i++) {
producer.newMessage()
.value("msg-" + i)
.deliverAfter(5, TimeUnit.SECONDS)
.sendAsync();
}

producer.flush();

Message<String> msg = consumer.receive(100, TimeUnit.MILLISECONDS);
assertNull(msg);

Set<String> receivedMsgs = new TreeSet<>();
for (int i = 0; i < N; i++) {
msg = consumer.receive(10, TimeUnit.SECONDS);
receivedMsgs.add(msg.getValue());
}

assertEquals(receivedMsgs.size(), N);
for (int i = 0; i < N; i++) {
assertTrue(receivedMsgs.contains("msg-" + i));
}
t.interrupt();
}
}