Skip to content

[fix][broker]fix Repeated messages of shared dispatcher #18227

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

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
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 @@ -52,6 +52,10 @@ public void add(long ledgerId, long entryId, long stickyKeyHash) {
messagesToRedeliver.add(ledgerId, entryId);
}

public boolean contains(long ledgerId, long entryId) {
return messagesToRedeliver.contains(ledgerId, entryId);
}

public void remove(long ledgerId, long entryId) {
if (hashesToBeBlocked != null) {
hashesToBeBlocked.remove(ledgerId, entryId);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -587,6 +587,9 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis
if (needTrimAckedMessages()) {
cursor.trimDeletedEntries(entries);
}
if (readType == ReadType.Replay) {
entries.removeIf(entry -> !redeliveryMessages.contains(entry.getLedgerId(), entry.getEntryId()));
}
Comment on lines +590 to +592
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have tried to remove these lines to check the newly added test concurrentlyReceiveDeliveriedMessages

It always gets passed with invocationCount = 100

Copy link
Contributor Author

@poorbarcode poorbarcode Nov 1, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have modified the test concurrentlyRedeliverAndCloseLastConsumer to make that the probability of the problem occurring is greater than 50%.


int entriesToDispatch = entries.size();
// Trigger read more messages
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
import io.netty.util.Timeout;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.lang.reflect.Field;
import java.nio.ByteBuffer;
import java.nio.file.Files;
import java.nio.file.Paths;
Expand Down Expand Up @@ -76,14 +77,21 @@
import lombok.EqualsAndHashCode;
import org.apache.avro.Schema.Parser;
import org.apache.bookkeeper.common.concurrent.FutureUtils;
import org.apache.bookkeeper.mledger.AsyncCallbacks;
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.ManagedLedgerException;
import org.apache.bookkeeper.mledger.ManagedLedgerFactory;
import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.bookkeeper.mledger.impl.cache.EntryCache;
import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager;
import org.apache.commons.lang3.RandomUtils;
import org.apache.commons.lang3.reflect.FieldUtils;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers;
import org.apache.pulsar.broker.service.persistent.PersistentSubscription;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.api.schema.GenericRecord;
Expand All @@ -109,6 +117,8 @@
import org.apache.pulsar.common.schema.SchemaType;
import org.apache.pulsar.common.util.FutureUtil;
import org.awaitility.Awaitility;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.Assert;
Expand Down Expand Up @@ -2383,6 +2393,226 @@ public void testPriorityConsumer() throws Exception {
log.info("-- Exiting {} test --", methodName);
}

@Test(timeOut = 300000)
public void concurrentlyReceiveDeliveredMessages() throws Exception {
String topic = "persistent://my-property/my-ns/tp-" + UUID.randomUUID().toString();
String subName = "sub";
final int totalPublishMessages = 50;

// create consumer tasks.
List<ConsumerTask> taskList = new ArrayList<>(10);
List<Thread> threadList = new ArrayList<>(10);
for (int i = 0; i < 10; i++){
PulsarClient pulsarClient = newPulsarClient(lookupUrl.toString(), 0);
ConsumerTask consumerTask = new ConsumerTask(pulsarClient, topic, subName);
taskList.add(consumerTask);
threadList.add(new Thread(consumerTask, consumerTask.consumerName));
}

// create topic, subscription and mock dispatcher.
Producer<byte[]> producer = pulsarClient.newProducer().topic(topic)
.enableBatching(false)
.messageRoutingMode(MessageRoutingMode.SinglePartition)
.create();
Consumer<byte[]> consumerForCreateSubscription = pulsarClient.newConsumer()
.topic(topic).subscriptionName(subName).consumerName("for_create_dispatcher")
.subscriptionType(SubscriptionType.Shared).receiverQueueSize(0)
.acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();
final PersistentDispatcherMultipleConsumers shardDispatcher = getShardDispatcher(topic, subName);

// Make redeliver messages.
for (int i = 0; i < totalPublishMessages; i++) {
final String message = "my-message-" + i;
producer.send(message.getBytes());
}
List<PositionImpl> redeliveryPositions = new ArrayList<>();
Consumer<byte[]> consumerForRedeliver = pulsarClient.newConsumer()
.topic(topic).subscriptionName(subName).consumerName("for_redeliver")
.subscriptionType(SubscriptionType.Shared).receiverQueueSize(totalPublishMessages)
.acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();
for (int i = 0; i < totalPublishMessages; i++) {
Message message = consumerForRedeliver.receive();
MessageIdImpl messageId = (MessageIdImpl) message.getMessageId();
redeliveryPositions.add(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId()));
}
consumerForRedeliver.close();
Awaitility.await().until(() -> shardDispatcher.getConsumers().size() == 1);

// Start all consumers.
threadList.forEach(thread -> thread.start());

// Wait all task finish.
for (Thread t : threadList){
t.join();
}

// Verify.
for (ConsumerTask task : taskList){
assertFalse(task.fail);
}
Assert.assertEquals(taskList.stream().map(t -> t.receivedEntries.size())
.reduce((a, b) -> a + b).get().intValue(), totalPublishMessages);

// cleanup.
consumerForCreateSubscription.close();
producer.close();
admin.topics().delete(topic, false);
}

private void makeReadEntryDelay20MillisSeconds(String topic, String subName) throws Exception {
PersistentTopic persistentTopic =
(PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get();
persistentTopic.getManagedLedger();
PersistentSubscription persistentSubscription = persistentTopic.getSubscription(subName);
ManagedCursorImpl managedCursor = (ManagedCursorImpl) persistentSubscription.getCursor();
final ManagedLedgerImpl originalManagedLedger = (ManagedLedgerImpl) managedCursor.getManagedLedger();
ManagedLedgerImpl spyManagedLedger = spy(originalManagedLedger);
doAnswer(new Answer() {
@Override
public Object answer(InvocationOnMock invocation) throws Throwable {
PositionImpl position = (PositionImpl) invocation.getArguments()[0];
final AsyncCallbacks.ReadEntryCallback originalCallback =
(AsyncCallbacks.ReadEntryCallback) invocation.getArguments()[1];
Object ctx = invocation.getArguments()[2];
originalManagedLedger.asyncReadEntry(position, new AsyncCallbacks.ReadEntryCallback(){

@Override
public void readEntryComplete(Entry entry, Object ctx) {
try {
Thread.sleep(20);
} catch (InterruptedException e) {
}
originalCallback.readEntryComplete(entry, ctx);
}

@Override
public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
originalCallback.readEntryFailed(exception, ctx);
}
}, ctx);
return null;
}
}).when(spyManagedLedger)
.asyncReadEntry(any(PositionImpl.class), any(AsyncCallbacks.ReadEntryCallback.class), any());
Field field_ledger = ManagedCursorImpl.class.getDeclaredField("ledger");
field_ledger.setAccessible(true);
field_ledger.set(managedCursor, spyManagedLedger);
}

@Test(timeOut = 30000)
public void concurrentlyRedeliverAndCloseLastConsumer() throws Exception {
String topic = "persistent://my-property/my-ns/tp-" + UUID.randomUUID().toString();
String subName = "sub";
final int totalPublishMessages = 50;
// Trigger subscription create.
Consumer<byte[]> consumerForCreateSubscription = pulsarClient.newConsumer()
.topic(topic).subscriptionName(subName).consumerName("for_create_dispatcher")
.subscriptionType(SubscriptionType.Shared).receiverQueueSize(0)
.acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();
consumerForCreateSubscription.close();

// Make many messages in delivery queue.
Producer<byte[]> producer = pulsarClient.newProducer().topic(topic)
.enableBatching(false)
.messageRoutingMode(MessageRoutingMode.SinglePartition)
.create();
PulsarClient newPulsarClient = newPulsarClient(lookupUrl.toString(), 0);
Consumer<byte[]> consumer = newPulsarClient.newConsumer()
.topic(topic).subscriptionName(subName)
.subscriptionType(SubscriptionType.Shared).receiverQueueSize(10)
.acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();
for (int i = 0; i < totalPublishMessages; i++) {
final String message = "my-message-" + i;
producer.send(message.getBytes());
}
consumer.receive();
// Make read entry delay, increase the probability of problems, see:#18289.
makeReadEntryDelay20MillisSeconds(topic, subName);

// Make "close last consumer" and "redeliver messages" execute concurrently.
log.info("===> trigger redeliver");
consumer.redeliverUnacknowledgedMessages();
consumer.close();
log.info("===> close first consumer");

// Receive messages and wait task finish.
List<ConsumerTask> taskList = new ArrayList<>(10);
List<Thread> threadList = new ArrayList<>(10);
for (int i = 0; i < 10; i++){
PulsarClient pulsarClient = newPulsarClient(lookupUrl.toString(), 0);
ConsumerTask consumerTask = new ConsumerTask(pulsarClient, topic, subName);
taskList.add(consumerTask);
threadList.add(new Thread(consumerTask, consumerTask.consumerName));
}
// wait tasks finished.
threadList.forEach(thread -> thread.start());
for (Thread t : threadList){
t.join();
}
// Verify.
for (ConsumerTask task : taskList){
assertFalse(task.fail);
}
Assert.assertEquals(taskList.stream().map(t -> t.receivedEntries.size())
.reduce((a, b) -> a + b).get().intValue(), totalPublishMessages);
// cleanup.
producer.close();
admin.topics().delete(topic, false);
}

private PersistentDispatcherMultipleConsumers getShardDispatcher(String topic, String subName) {
PersistentTopic persistentTopic =
(PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get();
PersistentSubscription persistentSubscription = persistentTopic.getSubscription(subName);
return (PersistentDispatcherMultipleConsumers) persistentSubscription.getDispatcher();
}

private static class ConsumerTask implements Runnable {

private static final AtomicInteger consumerNamSequence = new AtomicInteger();

private final List<Long> receivedEntries = new ArrayList<>();
private final PulsarClient pulsarClient;
private final String consumerName;
private final String topicName;
private final String subName;
private volatile boolean fail = false;

public ConsumerTask(PulsarClient newPulsarClient, String topicName, String subName){
this.pulsarClient = newPulsarClient;
this.topicName = topicName;
this.subName = subName;
this.consumerName = subName + "-" + consumerNamSequence.incrementAndGet();
}

@Override
public void run() {
try {
log.info("===> start consumer task");
Consumer<byte[]> consumer = pulsarClient.newConsumer()
.topic(topicName).subscriptionName(subName)
.subscriptionType(SubscriptionType.Shared).receiverQueueSize(10)
.consumerName(consumerName)
.acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();
while(true){
Message message = consumer.receive(2, TimeUnit.SECONDS);
if (message == null){
break;
}
MessageIdImpl messageId = (MessageIdImpl)message.getMessageId();
consumer.acknowledge(messageId);
log.info("===> GET " + messageId.getLedgerId() + ":" + messageId.getEntryId() + ", Consumer: " + consumerName);
receivedEntries.add(messageId.getEntryId());
}
consumer.close();
pulsarClient.close();
} catch (Exception e) {
fail = true;
throw new RuntimeException(e);
}
}
}

/**
* <pre>
* Verifies Dispatcher dispatches messages properly with shared-subscription consumers with combination of blocked
Expand Down
Loading