Skip to content

Commit

Permalink
[fix][client] Fix multi-topic consumer stuck after redeliver messages (
Browse files Browse the repository at this point in the history
…#18491)

(cherry picked from commit 7a93ff9)
  • Loading branch information
poorbarcode authored and liangyepianzhou committed Dec 5, 2022
1 parent feaad45 commit ce4172d
Show file tree
Hide file tree
Showing 2 changed files with 55 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,21 +20,22 @@

import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNull;

import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;

import lombok.Cleanup;
import lombok.extern.slf4j.Slf4j;

import org.apache.pulsar.broker.BrokerTestUtil;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.ProducerConsumerBase;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.api.SubscriptionType;
import org.awaitility.Awaitility;
import org.awaitility.reflect.WhiteboxImpl;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.DataProvider;
Expand Down Expand Up @@ -258,4 +259,54 @@ public void testNegativeAcksWithBackoff(boolean batching, boolean usePartitions,
consumer.close();
producer.close();
}
@Test(invocationCount = 5)
public void testMultiTopicConsumerConcurrentRedeliverAndReceive() throws Exception {
final String topic = BrokerTestUtil.newUniqueName("my-topic");
admin.topics().createPartitionedTopic(topic, 2);

final int receiverQueueSize = 10;

@Cleanup
MultiTopicsConsumerImpl<Integer> consumer =
(MultiTopicsConsumerImpl<Integer>) pulsarClient.newConsumer(Schema.INT32)
.topic(topic)
.subscriptionName("sub")
.receiverQueueSize(receiverQueueSize)
.subscribe();
ExecutorService internalPinnedExecutor =
WhiteboxImpl.getInternalState(consumer, "internalPinnedExecutor");

@Cleanup
Producer<Integer> producer = pulsarClient.newProducer(Schema.INT32)
.topic(topic)
.enableBatching(false)
.create();

for (int i = 0; i < receiverQueueSize; i++){
producer.send(i);
}

Awaitility.await().until(() -> consumer.incomingMessages.size() == receiverQueueSize);

// For testing the race condition of issue #18491
// We need to inject a delay for the pinned internal thread
Thread.sleep(1000L);
internalPinnedExecutor.submit(() -> consumer.redeliverUnacknowledgedMessages()).get();
// Make sure the message redelivery is completed. The incoming queue will be cleaned up during the redelivery.
internalPinnedExecutor.submit(() -> {}).get();

Set<Integer> receivedMsgs = new HashSet<>();
for (;;){
Message<Integer> msg = consumer.receive(2, TimeUnit.SECONDS);
if (msg == null){
break;
}
receivedMsgs.add(msg.getValue());
}
Assert.assertEquals(receivedMsgs.size(), 10);

producer.close();
consumer.close();
admin.topics().deletePartitionedTopic("persistent://public/default/" + topic);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -691,8 +691,8 @@ public void redeliverUnacknowledgedMessages() {
});
clearIncomingMessages();
unAckedMessageTracker.clear();
resumeReceivingFromPausedConsumersIfNeeded();
});
resumeReceivingFromPausedConsumersIfNeeded();
}

@Override
Expand Down

0 comments on commit ce4172d

Please sign in to comment.