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 failover/exclusive consumer with batch cumulate ack issue. #18454

Merged
merged 4 commits into from
Nov 15, 2022
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 @@ -116,10 +116,23 @@ public void testConsumerDedup(boolean batchingEnabled, int receiverQueueSize) th
}
producer.flush();

for (int i = 0; i < 30; i++) {
Message<String> msg = consumer.receive();
assertEquals(msg.getValue(), "new-message-" + i);
consumer.acknowledge(msg);
if (batchingEnabled) {
for (int i = 0; i < 30; i++) {
Message<String> msg = consumer.receive();
assertEquals(msg.getValue(), "hello-" + i);
consumer.acknowledge(msg);
}
for (int i = 0; i < 30; i++) {
Message<String> msg = consumer.receive();
assertEquals(msg.getValue(), "new-message-" + i);
consumer.acknowledge(msg);
}
} else {
for (int i = 0; i < 30; i++) {
Message<String> msg = consumer.receive();
assertEquals(msg.getValue(), "new-message-" + i);
consumer.acknowledge(msg);
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@

import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import lombok.Cleanup;
import lombok.extern.slf4j.Slf4j;
Expand All @@ -35,6 +36,7 @@
import org.apache.pulsar.client.api.ProducerConsumerBase;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.client.api.SubscriptionType;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.DataProvider;
Expand Down Expand Up @@ -362,4 +364,79 @@ public void testNegativeAcksWithBatchAckEnabled() throws Exception {
// There should be no more messages
assertNull(consumer.receive(100, TimeUnit.MILLISECONDS));
}

@Test
public void testFailoverConsumerBatchCumulateAck() throws Exception {
final String topic = BrokerTestUtil.newUniqueName("my-topic");
admin.topics().createPartitionedTopic(topic, 2);

@Cleanup
Consumer<Integer> consumer = pulsarClient.newConsumer(Schema.INT32)
.topic(topic)
.subscriptionName("sub")
.subscriptionType(SubscriptionType.Failover)
.enableBatchIndexAcknowledgment(true)
.acknowledgmentGroupTime(100, TimeUnit.MILLISECONDS)
.receiverQueueSize(10)
.subscribe();

@Cleanup
Producer<Integer> producer = pulsarClient.newProducer(Schema.INT32)
.topic(topic)
.batchingMaxMessages(10)
.batchingMaxPublishDelay(3, TimeUnit.SECONDS)
.blockIfQueueFull(true)
.create();

int count = 0;
Set<Integer> datas = new HashSet<>();
CountDownLatch producerLatch = new CountDownLatch(10);
while (count < 10) {
datas.add(count);
producer.sendAsync(count).whenComplete((m, e) -> {
producerLatch.countDown();
});
count++;
}
producerLatch.await();
CountDownLatch consumerLatch = new CountDownLatch(1);
new Thread(new Runnable() {
@Override
public void run() {
consumer.receiveAsync()
.thenCompose(m -> {
log.info("received one msg : {}", m.getMessageId());
datas.remove(m.getValue());
return consumer.acknowledgeCumulativeAsync(m);
})
.thenAccept(ignore -> {
try {
Thread.sleep(500);
consumer.redeliverUnacknowledgedMessages();
} catch (Exception e) {
throw new RuntimeException(e);
}
})
.whenComplete((r, e) -> {
consumerLatch.countDown();
});
}
}).start();
consumerLatch.await();
Thread.sleep(500);
count = 0;
while(true) {
Message<Integer> msg = consumer.receive(5, TimeUnit.SECONDS);
if (msg == null) {
break;
}
consumer.acknowledgeCumulative(msg);
Thread.sleep(200);
datas.remove(msg.getValue());
log.info("received msg : {}", msg.getMessageId());
count++;
}
Assert.assertEquals(count, 9);
Assert.assertEquals(0, datas.size());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1275,9 +1275,9 @@ void messageReceived(CommandMessage cmdMessage, ByteBuf headersAndPayload, Clien
final int numMessages = msgMetadata.getNumMessagesInBatch();
final int numChunks = msgMetadata.hasNumChunksFromMsg() ? msgMetadata.getNumChunksFromMsg() : 0;
final boolean isChunkedMessage = numChunks > 1;

MessageIdImpl msgId = new MessageIdImpl(messageId.getLedgerId(), messageId.getEntryId(), getPartitionIndex());
if (acknowledgmentsGroupingTracker.isDuplicate(msgId)) {
if (numMessages == 1 && !msgMetadata.hasNumMessagesInBatch()
&& acknowledgmentsGroupingTracker.isDuplicate(msgId)) {
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Ignoring message as it was already being acked earlier by same consumer {}/{}",
topic, subscription, consumerName, msgId);
Expand Down Expand Up @@ -1541,7 +1541,10 @@ void receiveIndividualMessagesFromBatch(BrokerEntryMetadata brokerEntryMetadata,
skippedMessages++;
continue;
}

}
if (acknowledgmentsGroupingTracker.isDuplicate(message.getMessageId())) {
skippedMessages++;
continue;
}
executeNotifyCallback(message);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -261,7 +261,11 @@ private void receiveMessageFromConsumer(ConsumerImpl<T> consumer, boolean batchR
return;
}
// Process the message, add to the queue and trigger listener or async callback
messages.forEach(msg -> messageReceived(consumer, msg));
messages.forEach(msg -> {
if (isValidConsumerEpoch((MessageImpl<T>) msg)) {
messageReceived(consumer, msg);
}
});

int size = incomingMessages.size();
int maxReceiverQueueSize = getCurrentReceiverQueueSize();
Expand Down