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

Expose include head message for durable consumers #5622

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 @@ -1478,8 +1478,10 @@ public void persistentTopicsCursorReset(String topicName) throws Exception {
topicName = "persistent://prop-xyz/ns1/" + topicName;

// create consumer and subscription
Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName("my-sub")
.subscriptionType(SubscriptionType.Exclusive).acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();
Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName)
.subscriptionName("my-sub").startMessageIdInclusive()
.subscriptionType(SubscriptionType.Exclusive)
.acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();

assertEquals(admin.topics().getSubscriptions(topicName), Lists.newArrayList("my-sub"));

Expand Down Expand Up @@ -1529,8 +1531,10 @@ public void persistentTopicsCursorResetAfterReset(String topicName) throws Excep
topicName = "persistent://prop-xyz/ns1/" + topicName;

// create consumer and subscription
Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName("my-sub")
.subscriptionType(SubscriptionType.Exclusive).acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();
Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName)
.subscriptionName("my-sub").startMessageIdInclusive()
.subscriptionType(SubscriptionType.Exclusive)
.acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();

assertEquals(admin.topics().getSubscriptions(topicName), Lists.newArrayList("my-sub"));

Expand Down Expand Up @@ -1601,7 +1605,8 @@ public void persistentTopicsCursorResetAndFailover() throws Exception {
admin.namespaces().setRetention(namespace, new RetentionPolicies(10, 10));

// Create consumer and failover subscription
Consumer<byte[]> consumerA = pulsarClient.newConsumer().topic(topicName).subscriptionName(subName)
Consumer<byte[]> consumerA = pulsarClient.newConsumer().topic(topicName)
.subscriptionName(subName).startMessageIdInclusive()
.consumerName("consumerA").subscriptionType(SubscriptionType.Failover)
.acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();

Expand Down Expand Up @@ -1667,8 +1672,10 @@ public void partitionedTopicsCursorReset(String topicName) throws Exception {
admin.topics().createPartitionedTopic(topicName, 4);

// create consumer and subscription
Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName("my-sub")
.subscriptionType(SubscriptionType.Exclusive).acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();
Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName)
.subscriptionName("my-sub").startMessageIdInclusive()
.subscriptionType(SubscriptionType.Exclusive)
.acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();

List<String> topics = admin.topics().getList("prop-xyz/ns1");
assertEquals(topics.size(), 4);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1455,8 +1455,10 @@ public void persistentTopicsCursorReset(String topicName) throws Exception {
topicName = "persistent://prop-xyz/use/ns1/" + topicName;

// create consumer and subscription
Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName("my-sub")
.subscriptionType(SubscriptionType.Exclusive).acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();
Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName)
.subscriptionName("my-sub").startMessageIdInclusive()
.subscriptionType(SubscriptionType.Exclusive)
.acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();

assertEquals(admin.topics().getSubscriptions(topicName), Lists.newArrayList("my-sub"));

Expand Down Expand Up @@ -1506,8 +1508,10 @@ public void persistentTopicsCursorResetAfterReset(String topicName) throws Excep
topicName = "persistent://prop-xyz/use/ns1/" + topicName;

// create consumer and subscription
Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName("my-sub")
.subscriptionType(SubscriptionType.Exclusive).acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();
Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName)
.subscriptionName("my-sub").startMessageIdInclusive()
.subscriptionType(SubscriptionType.Exclusive)
.acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();

assertEquals(admin.topics().getSubscriptions(topicName), Lists.newArrayList("my-sub"));

Expand Down Expand Up @@ -1577,8 +1581,10 @@ public void partitionedTopicsCursorReset(String topicName) throws Exception {
admin.topics().createPartitionedTopic(topicName, 4);

// create consumer and subscription
Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName("my-sub")
.subscriptionType(SubscriptionType.Exclusive).acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();
Consumer<byte[]> consumer = pulsarClient.newConsumer().topic(topicName)
.subscriptionName("my-sub").startMessageIdInclusive()
.subscriptionType(SubscriptionType.Exclusive)
.acknowledgmentGroupTime(0, TimeUnit.SECONDS).subscribe();

List<String> topics = admin.topics().getList("prop-xyz/use/ns1");
assertEquals(topics.size(), 4);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.Random;
import java.util.concurrent.Callable;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
Expand Down Expand Up @@ -108,6 +109,22 @@ protected void setup() throws Exception {
super.producerBaseSetup();
}

@DataProvider
public static Object[][] variationsForExpectedPos() {
return new Object[][] {
// batching / start-inclusive / num-of-messages
{true, true, 10 },
{true, false, 10 },
{false, true, 10 },
{false, false, 10 },

{true, true, 100 },
{true, false, 100 },
{false, true, 100 },
{false, false, 100 },
};
}

@AfterMethod
@Override
protected void cleanup() throws Exception {
Expand Down Expand Up @@ -435,6 +452,7 @@ public void testBackoffAndReconnect(int batchMessageDelayMs) throws Exception {
Consumer<byte[]> consumer = pulsarClient.newConsumer()
.topic("persistent://my-property/my-ns/my-topic4")
.subscriptionName("my-subscriber-name")
.startMessageIdInclusive()
.subscribe();
ProducerBuilder<byte[]> producerBuilder = pulsarClient.newProducer()
.topic("persistent://my-property/my-ns/my-topic4");
Expand Down Expand Up @@ -668,6 +686,7 @@ public void testConcurrentConsumerReceiveWhileReconnect(int batchMessageDelayMs)
String subName = UUID.randomUUID().toString();
final Consumer<byte[]> consumer = pulsarClient.newConsumer()
.topic("persistent://my-property/my-ns/my-topic7").subscriptionName(subName)
.startMessageIdInclusive()
.receiverQueueSize(recvQueueSize).subscribe();
ExecutorService executor = Executors.newCachedThreadPool();

Expand Down Expand Up @@ -2962,6 +2981,7 @@ public void testFlushBatchDisabled() throws Exception {

Consumer<byte[]> consumer = pulsarClient.newConsumer()
.topic("persistent://my-property/my-ns/test-flush-disabled")
.startMessageIdInclusive()
.subscriptionName("my-subscriber-name").subscribe();

ProducerBuilder<byte[]> producerBuilder = pulsarClient.newProducer()
Expand Down Expand Up @@ -3197,5 +3217,50 @@ public void testPartitionedTopicWithOnePartition() throws Exception {

log.info("-- Exiting {} test --", methodName);
}


@Test(dataProvider = "variationsForExpectedPos")
public void testReaderStartMessageIdAtExpectedPos(boolean batching, boolean startInclusive, int numOfMessages)
throws Exception {
final String topicName = "persistent://my-property/my-ns/ConsumerStartMessageIdAtExpectedPos";
final int resetIndex = new Random().nextInt(numOfMessages); // Choose some random index to reset
final int firstMessage = startInclusive ? resetIndex : resetIndex + 1; // First message of reset

Producer<byte[]> producer = pulsarClient.newProducer()
.topic(topicName)
.enableBatching(batching)
.create();

MessageId resetPos = null;
for (int i = 0; i < numOfMessages; i++) {
MessageId msgId = producer.send(String.format("msg num %d", i).getBytes());
if (resetIndex == i) {
resetPos = msgId;
}
}

ConsumerBuilder<byte[]> consumerBuilder = pulsarClient.newConsumer()
.topic(topicName);

if (startInclusive) {
consumerBuilder.startMessageIdInclusive();
}

Consumer<byte[]> consumer = consumerBuilder.subscriptionName("my-subscriber-name").subscribe();
consumer.seek(resetPos);
Set<String> messageSet = Sets.newHashSet();
for (int i = firstMessage; i < numOfMessages; i++) {
Message<byte[]> message = consumer.receive();
String receivedMessage = new String(message.getData());
String expectedMessage = String.format("msg num %d", i);
testMessageOrderAndDuplicates(messageSet, receivedMessage, expectedMessage);
}

assertEquals(((ConsumerImpl) consumer).numMessagesInQueue(), 0);

// Processed messages should be the number of messages in the range: [FirstResetMessage..TotalNumOfMessages]
assertEquals(messageSet.size(), numOfMessages - firstMessage);

consumer.close();
producer.close();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -251,6 +251,7 @@ public void testBackoffAndReconnect(int batchMessageDelayMs) throws Exception {
.topic("persistent://my-property/use/my-ns/my-topic4")
.subscriptionName("my-subscriber-name")
.subscriptionType(SubscriptionType.Exclusive)
.startMessageIdInclusive()
.subscribe();
Producer<byte[]> producer = pulsarClient.newProducer()
.topic("persistent://my-property/use/my-ns/my-topic4")
Expand Down Expand Up @@ -501,8 +502,8 @@ public void testConcurrentConsumerReceiveWhileReconnect(int batchMessageDelayMs)
final Consumer<byte[]> consumer = pulsarClient.newConsumer()
.topic("persistent://my-property/use/my-ns/my-topic7")
.subscriptionName(subName)
.receiverQueueSize(recvQueueSize)
.subscribe();
.startMessageIdInclusive()
.receiverQueueSize(recvQueueSize).subscribe();
ExecutorService executor = Executors.newCachedThreadPool();

final CyclicBarrier barrier = new CyclicBarrier(numConsumersThreads + 1);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -415,6 +415,7 @@ public void testResetCursor(SubscriptionType subType) throws Exception {
admin.namespaces().setRetention(topicName.getNamespace(), policy);

ConsumerBuilder<byte[]> consumerBuilder = pulsarClient.newConsumer().topic(topicName.toString())
.startMessageIdInclusive()
.subscriptionName(subsId).subscriptionType(subType).messageListener((consumer, msg) -> {
try {
synchronized (received) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -535,4 +535,12 @@ public interface ConsumerBuilder<T> extends Cloneable {
* @param keySharedPolicy The {@link KeySharedPolicy} want to specify
*/
ConsumerBuilder<T> keySharedPolicy(KeySharedPolicy keySharedPolicy);

/**
* Set the consumer to include the given position of any reset operation like {@link Consumer#seek(long) or
* {@link Consumer#seek(MessageId)}}.
*
* @return the consumer builder instance
*/
ConsumerBuilder<T> startMessageIdInclusive();
}
Original file line number Diff line number Diff line change
Expand Up @@ -323,6 +323,12 @@ public ConsumerBuilder<T> autoUpdatePartitions(boolean autoUpdate) {
return this;
}

@Override
public ConsumerBuilder<T> startMessageIdInclusive() {
conf.setResetIncludeHead(true);
return this;
}

public ConsumerConfigurationData<T> getConf() {
return conf;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -805,7 +805,7 @@ void messageReceived(MessageIdData messageId, int redeliveryCount, ByteBuf heade
// and return undecrypted payload
if (isMessageUndecryptable || (numMessages == 1 && !msgMetadata.hasNumMessagesInBatch())) {

if (isNonDurableAndSameEntryAndLedger(messageId) && isPriorEntryIndex(messageId.getEntryId())) {
if (isResetIncludedAndSameEntryLedger(messageId) && isPriorEntryIndex(messageId.getEntryId())) {
// We need to discard entries that were prior to startMessageId
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Ignoring message from before the startMessageId: {}", subscription,
Expand Down Expand Up @@ -958,7 +958,7 @@ void receiveIndividualMessagesFromBatch(MessageMetadata msgMetadata, int redeliv
ByteBuf singleMessagePayload = Commands.deSerializeSingleMessageInBatch(uncompressedPayload,
singleMessageMetadataBuilder, i, batchSize);

if (isNonDurableAndSameEntryAndLedger(messageId) && isPriorBatchIndex(i)) {
if (isResetIncludedAndSameEntryLedger(messageId) && isPriorBatchIndex(i)) {
// If we are receiving a batch message, we need to discard messages that were prior
// to the startMessageId
if (log.isDebugEnabled()) {
Expand Down Expand Up @@ -1029,8 +1029,8 @@ private boolean isPriorBatchIndex(long idx) {
return resetIncludeHead ? idx < startMessageId.getBatchIndex() : idx <= startMessageId.getBatchIndex();
}

private boolean isNonDurableAndSameEntryAndLedger(MessageIdData messageId) {
return subscriptionMode == SubscriptionMode.NonDurable && startMessageId != null
private boolean isResetIncludedAndSameEntryLedger(MessageIdData messageId) {
return !resetIncludeHead && startMessageId != null
&& messageId.getLedgerId() == startMessageId.getLedgerId()
&& messageId.getEntryId() == startMessageId.getEntryId();
}
Expand Down