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 wrong redelivery count while redeliver when consumer disconnected. #5895

Merged
merged 2 commits into from
Dec 24, 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 @@ -258,8 +258,11 @@ public ChannelPromise sendMessages(final List<Entry> entries, EntryBatchSizes ba
consumerId, entry.getLedgerId(), entry.getEntryId());
}

int redeliveryCount = redeliveryTracker
.getRedeliveryCount(PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId()));
int redeliveryCount = 0;
PositionImpl position = PositionImpl.get(messageId.getLedgerId(), messageId.getEntryId());
if (redeliveryTracker.contains(position)) {
redeliveryCount = redeliveryTracker.incrementAndGetRedeliveryCount(position);
}
ctx.write(Commands.newMessage(consumerId, messageId, redeliveryCount, metadataAndPayload), ctx.voidPromise());
messageId.recycle();
messageIdBuilder.recycle();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,4 +62,16 @@ public void removeBatch(List<Position> positions) {
public void clear() {
trackerCache.clear();
}

@Override
public boolean contains(Position position) {
PositionImpl positionImpl = (PositionImpl) position;
return trackerCache.containsKey(positionImpl.getLedgerId(), positionImpl.getEntryId());
}

@Override
public void addIfAbsent(Position position) {
PositionImpl positionImpl = (PositionImpl) position;
trackerCache.putIfAbsent(positionImpl.getLedgerId(), positionImpl.getEntryId(), 0, 0L);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -33,4 +33,8 @@ public interface RedeliveryTracker {
void removeBatch(List<Position> positions);

void clear();

boolean contains(Position position);

void addIfAbsent(Position position);
}
Original file line number Diff line number Diff line change
Expand Up @@ -52,4 +52,14 @@ public void removeBatch(List<Position> positions) {
public void clear() {
// no-op
}

@Override
public boolean contains(Position position) {
return false;
}

@Override
public void addIfAbsent(Position position) {
// no-op
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -213,6 +213,7 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE
}

messagesToRedeliver.clear();
redeliveryTracker.clear();
if (closeFuture != null) {
log.info("[{}] All consumers removed. Subscription is disconnected", name);
closeFuture.complete(null);
Expand All @@ -224,6 +225,7 @@ public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceE
}
consumer.getPendingAcks().forEach((ledgerId, entryId, batchSize, none) -> {
messagesToRedeliver.add(ledgerId, entryId);
redeliveryTracker.addIfAbsent(PositionImpl.get(ledgerId, entryId));
});
totalAvailablePermits -= consumer.getAvailablePermits();
readMoreEntries();
Expand Down Expand Up @@ -637,7 +639,7 @@ public synchronized void redeliverUnacknowledgedMessages(Consumer consumer) {
public synchronized void redeliverUnacknowledgedMessages(Consumer consumer, List<PositionImpl> positions) {
positions.forEach(position -> {
messagesToRedeliver.add(position.getLedgerId(), position.getEntryId());
redeliveryTracker.incrementAndGetRedeliveryCount(position);
redeliveryTracker.addIfAbsent(position);
});
if (log.isDebugEnabled()) {
log.debug("[{}-{}] Redelivering unacknowledged messages for consumer {}", name, consumer, positions);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -336,7 +336,7 @@ private synchronized void internalRedeliverUnacknowledgedMessages(Consumer consu
@Override
public void redeliverUnacknowledgedMessages(Consumer consumer, List<PositionImpl> positions) {
// We cannot redeliver single messages to single consumers to preserve ordering.
positions.forEach(redeliveryTracker::incrementAndGetRedeliveryCount);
positions.forEach(redeliveryTracker::addIfAbsent);
redeliverUnacknowledgedMessages(consumer);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ public void testDeadLetterTopic() throws Exception {
.topic(topic)
.subscriptionName("my-subscription")
.subscriptionType(SubscriptionType.Shared)
.ackTimeout(3, TimeUnit.SECONDS)
.ackTimeout(1, TimeUnit.SECONDS)
.deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(maxRedeliveryCount).build())
.receiverQueueSize(100)
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
Expand Down Expand Up @@ -134,7 +134,7 @@ public void testDeadLetterTopicWithMultiTopic() throws Exception {
.topic(topic1, topic2)
.subscriptionName("my-subscription")
.subscriptionType(SubscriptionType.Shared)
.ackTimeout(3, TimeUnit.SECONDS)
.ackTimeout(1, TimeUnit.SECONDS)
.deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(maxRedeliveryCount).build())
.receiverQueueSize(100)
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
Expand Down Expand Up @@ -210,7 +210,7 @@ public void testDeadLetterTopicByCustomTopicName() throws Exception {
.topic(topic)
.subscriptionName("my-subscription")
.subscriptionType(SubscriptionType.Shared)
.ackTimeout(3, TimeUnit.SECONDS)
.ackTimeout(1, TimeUnit.SECONDS)
.receiverQueueSize(100)
.deadLetterPolicy(DeadLetterPolicy.builder()
.maxRedeliverCount(maxRedeliveryCount)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,8 @@
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;

import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.TimeUnit;

public class ExposeMessageRedeliveryCountTest extends ProducerConsumerBase {
Expand All @@ -50,7 +52,7 @@ public void testRedeliveryCount() throws PulsarClientException {
.topic(topic)
.subscriptionName("my-subscription")
.subscriptionType(SubscriptionType.Shared)
.ackTimeout(3, TimeUnit.SECONDS)
.ackTimeout(1, TimeUnit.SECONDS)
.receiverQueueSize(100)
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
.subscribe();
Expand Down Expand Up @@ -87,7 +89,7 @@ public void testRedeliveryCountWithPartitionedTopic() throws PulsarClientExcepti
.topic(topic)
.subscriptionName("my-subscription")
.subscriptionType(SubscriptionType.Shared)
.ackTimeout(3, TimeUnit.SECONDS)
.ackTimeout(1, TimeUnit.SECONDS)
.receiverQueueSize(100)
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
.subscribe();
Expand All @@ -114,4 +116,63 @@ public void testRedeliveryCountWithPartitionedTopic() throws PulsarClientExcepti

admin.topics().deletePartitionedTopic(topic);
}

@Test(timeOut = 30000)
public void testRedeliveryCountWhenConsumerDisconnected() throws PulsarClientException, InterruptedException {

String topic = "persistent://my-property/my-ns/testRedeliveryCountWhenConsumerDisconnected";

Consumer<String> consumer0 = pulsarClient.newConsumer(Schema.STRING)
.topic(topic)
.subscriptionName("s1")
.subscriptionType(SubscriptionType.Shared)
.subscribe();

Consumer<String> consumer1 = pulsarClient.newConsumer(Schema.STRING)
.topic(topic)
.subscriptionName("s1")
.subscriptionType(SubscriptionType.Shared)
.subscribe();

Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
.topic(topic)
.enableBatching(true)
.batchingMaxMessages(5)
.batchingMaxPublishDelay(1, TimeUnit.SECONDS)
.create();

final int messages = 10;
for (int i = 0; i < messages; i++) {
producer.send("my-message-" + i);
}

List<Message<String>> receivedMessagesForConsumer0 = new ArrayList<>();
List<Message<String>> receivedMessagesForConsumer1 = new ArrayList<>();

for (int i = 0; i < messages; i++) {
Message<String> msg = consumer0.receive(1, TimeUnit.SECONDS);
if (msg != null) {
receivedMessagesForConsumer0.add(msg);
} else {
break;
}
}

for (int i = 0; i < messages; i++) {
Message<String> msg = consumer1.receive(1, TimeUnit.SECONDS);
if (msg != null) {
receivedMessagesForConsumer1.add(msg);
} else {
break;
} }

Assert.assertEquals(receivedMessagesForConsumer0.size() + receivedMessagesForConsumer1.size(), messages);

consumer0.close();

for (int i = 0; i < receivedMessagesForConsumer0.size(); i++) {
Assert.assertEquals(consumer1.receive().getRedeliveryCount(), 1);
}

}
}