Skip to content

Commit

Permalink
Fix message being ignored when the non-persistent topic reader reconn…
Browse files Browse the repository at this point in the history
…ect. (apache#12348)

### Motivation

Consumers should not perform message ignore checks when consuming messages from non-persistent topics.
Otherwise, it may lead to a case where the message is incorrectly ignored when the non-persistent reader(or non-durable subscription non-persistent consumer) reconnects to the broker.

Currently, when the reader of the non-persistent topic has its `currentMessageQueue` empty before reconnection, its `startMessageId` is set to `lastDequeuedMessageId` after reconnection: https://github.com/apache/pulsar/blob/4ae7f6a1b38a003c9fc26844e52771b776bf64bf/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java#L856

 and if we specify `startMessageId` as the default value (-1:-1), it will be set to (0:0) after reconnection, thus causes all subsequent messages to be ignored:
https://github.com/apache/pulsar/blob/4ae7f6a1b38a003c9fc26844e52771b776bf64bf/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java#L1202-L1211
and for batch messages here:
https://github.com/apache/pulsar/blob/4ae7f6a1b38a003c9fc26844e52771b776bf64bf/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java#L1029-L1037

### Modifications

* Remove the message ignore check when the topic of the consumer is non-persistent
  • Loading branch information
RobertIndie authored Oct 14, 2021
1 parent 240826a commit 867d71c
Show file tree
Hide file tree
Showing 2 changed files with 146 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1026,7 +1026,8 @@ protected <U> MessageImpl<U> newSingleMessage(final int index,
Commands.deSerializeSingleMessageInBatch(payload, singleMessageMetadata, index, numMessages);
}

if (isSameEntry(messageId) && isPriorBatchIndex(index)) {
// If the topic is non-persistent, we should not ignore any messages.
if (this.topicName.isPersistent() && isSameEntry(messageId) && isPriorBatchIndex(index)) {
// 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 @@ -1199,7 +1200,8 @@ void messageReceived(MessageIdData messageId, int redeliveryCount, List<Long> ac
}
}

if (isSameEntry(msgId) && isPriorEntryIndex(messageId.getEntryId())) {
// If the topic is non-persistent, we should not ignore any messages.
if (this.topicName.isPersistent() && isSameEntry(msgId) && 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
Original file line number Diff line number Diff line change
@@ -0,0 +1,142 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.tests.integration.messaging;

import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNotNull;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
import lombok.Cleanup;
import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.MessageId;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.Reader;
import org.apache.pulsar.client.api.Schema;
import org.testng.annotations.Test;

@Slf4j
public class ReaderMessagingTest extends MessagingBase {

@Test(dataProvider = "ServiceAndAdminUrls")
public void testReaderReconnectAndRead(Supplier<String> serviceUrl, Supplier<String> adminUrl) throws Exception {
log.info("-- Starting {} test --", methodName);
final String topicName = getNonPartitionedTopic("test-reader-reconnect-read", false);
@Cleanup final PulsarClient client = PulsarClient.builder()
.serviceUrl(serviceUrl.get())
.build();
@Cleanup final Reader<String> reader = client.newReader(Schema.STRING)
.topic(topicName)
.subscriptionName("test-sub")
// Here we need to make sure that setting the startMessageId should not cause a change in the
// behavior of the reader under non.
.startMessageId(MessageId.earliest)
.create();

final int messagesToSend = 10;
@Cleanup final Producer<String> producer = client.newProducer(Schema.STRING)
.topic(topicName)
.enableBatching(false)
.create();
for (int i = 0; i < messagesToSend; i++) {
MessageId messageId = producer.newMessage().value("message-" + i).send();
assertNotNull(messageId);
}

for (int i = 0; i < messagesToSend; i++) {
Message<String> msg = reader.readNext();
assertEquals(msg.getValue(), "message-" + i);
}

@Cleanup
PulsarAdmin admin = PulsarAdmin.builder()
.serviceHttpUrl(adminUrl.get())
.build();

admin.topics().unload(topicName);

for (int i = 0; i < messagesToSend; i++) {
MessageId messageId = producer.newMessage().value("message-" + i).send();
assertNotNull(messageId);
}

for (int i = 0; i < messagesToSend; i++) {
Message<String> msg = reader.readNext();
assertEquals(msg.getValue(), "message-" + i);
}

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

@Test(dataProvider = "ServiceAndAdminUrls")
public void testReaderReconnectAndReadBatchMessages(Supplier<String> serviceUrl, Supplier<String> adminUrl)
throws Exception {
log.info("-- Starting {} test --", methodName);
final String topicName = getNonPartitionedTopic("test-reader-reconnect-read-batch", false);
@Cleanup final PulsarClient client = PulsarClient.builder()
.serviceUrl(serviceUrl.get())
.build();
@Cleanup final Reader<String> reader = client.newReader(Schema.STRING)
.topic(topicName)
.subscriptionName("test-sub")
// Here we need to make sure that setting the startMessageId should not cause a change in the
// behavior of the reader under non.
.startMessageId(MessageId.earliest)
.create();

final int messagesToSend = 10;
@Cleanup final Producer<String> producer = client.newProducer(Schema.STRING)
.topic(topicName)
.enableBatching(true)
.batchingMaxPublishDelay(5, TimeUnit.SECONDS)
.batchingMaxMessages(5)
.create();

for (int i = 0; i < messagesToSend; i++) {
MessageId messageId = producer.newMessage().value("message-" + i).send();
assertNotNull(messageId);
}

for (int i = 0; i < messagesToSend; i++) {
Message<String> msg = reader.readNext();
assertEquals(msg.getValue(), "message-" + i);
}

@Cleanup
PulsarAdmin admin = PulsarAdmin.builder()
.serviceHttpUrl(adminUrl.get())
.build();

admin.topics().unload(topicName);

for (int i = 0; i < messagesToSend; i++) {
MessageId messageId = producer.newMessage().value("message-" + i).send();
assertNotNull(messageId);
}

for (int i = 0; i < messagesToSend; i++) {
Message<String> msg = reader.readNext();
assertEquals(msg.getValue(), "message-" + i);
}

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

0 comments on commit 867d71c

Please sign in to comment.