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

[fix][client] fix retry topic with exclusive mode. #23859

Open
wants to merge 4 commits 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 @@ -136,6 +136,46 @@ public void testRetryTopic() throws Exception {
checkConsumer.close();
}

/**
* Retry topic feature relies on the delay queue feature when consumer produce a delayed message
* to the retry topic. The delay queue feature is only supported in shared and key-shared subscription type.
* As a result, the subscription type of the retry topic should be shared or key-shared.
* @throws Exception
*/
@Test
public void testRetryTopicWithExclusiveMode() throws Exception {
final String topic = "persistent://my-property/my-ns/retry-topic-exclusive";
final int maxRedeliveryCount = 2;

Consumer<byte[]> consumer = pulsarClient.newConsumer(Schema.BYTES)
.topic(topic)
.subscriptionName("my-subscription")
.subscriptionType(SubscriptionType.Exclusive)
.enableRetry(true)
.deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(maxRedeliveryCount).build())
.receiverQueueSize(100)
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
.subscribe();

Producer<byte[]> producer = pulsarClient.newProducer(Schema.BYTES)
.topic(topic)
.create();

producer.send("Hello Pulsar".getBytes());
producer.close();

// receive message and set delay to 5 seconds
Message<byte[]> message = consumer.receive();
long timestamp = System.currentTimeMillis();
consumer.reconsumeLater(message, 4, TimeUnit.SECONDS);

// receive message and check the delay is at least 4 seconds
consumer.receive();
long delay = System.currentTimeMillis() - timestamp;
assertTrue(delay >= 2000);
consumer.close();
}

@Data
public static class Foo {
@Nullable
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.ConsumerBuilder;
import org.apache.pulsar.client.api.ConsumerEventListener;
import org.apache.pulsar.client.api.DeadLetterPolicy;
import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.MessageIdAdv;
Expand Down Expand Up @@ -792,6 +793,13 @@ private boolean isCumulativeAcknowledgementAllowed(SubscriptionType type) {
}

protected SubType getSubType() {
// For retry topic, we always use Shared subscription
// Because we will produce delayed messages to retry topic.
DeadLetterPolicy deadLetterPolicy = conf.getDeadLetterPolicy();
if (deadLetterPolicy != null && topic.equals(deadLetterPolicy.getRetryLetterTopic())) {
return SubType.Shared;
}

SubscriptionType type = conf.getSubscriptionType();
switch (type) {
case Exclusive:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -151,7 +151,7 @@ public CompletableFuture<Consumer<T>> subscribeAsync() {
Math.max(conf.getBatchReceivePolicy().getMaxNumMessages(), conf.getReceiverQueueSize()));
}
CompletableFuture<Void> applyDLQConfig;
if (conf.isRetryEnable() && conf.getTopicNames().size() > 0) {
if (conf.isRetryEnable() && !conf.getTopicNames().isEmpty()) {
TopicName topicFirst = TopicName.get(conf.getTopicNames().iterator().next());
//Issue 9327: do compatibility check in case of the default retry and dead letter topic name changed
String oldRetryLetterTopic = TopicName.get(topicFirst.getDomain().value(), topicFirst.getNamespaceObject(),
Expand Down
Loading