This is an automated email from the ASF dual-hosted git repository.

penghui pushed a commit to branch branch-2.8
in repository https://gitbox.apache.org/repos/asf/pulsar.git


The following commit(s) were added to refs/heads/branch-2.8 by this push:
     new 8e2b003  Fix message being ignored when the non-persistent topic 
reader reconnect. (#12348)
8e2b003 is described below

commit 8e2b003e5901c397ec3e472c67989854ea90e385
Author: Zike Yang <[email protected]>
AuthorDate: Thu Oct 14 20:31:25 2021 +0800

    Fix message being ignored when the non-persistent topic reader reconnect. 
(#12348)
    
    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
    
    * Remove the message ignore check when the topic of the consumer is 
non-persistent
    
    (cherry picked from commit 867d71cfbe34a659e4c6b8a3a6446b71f4a05c52)
---
 .../apache/pulsar/client/impl/ConsumerImpl.java    |   5 +-
 .../integration/messaging/ReaderMessagingTest.java | 142 +++++++++++++++++++++
 .../src/test/resources/pulsar-messaging.xml        |   1 +
 3 files changed, 146 insertions(+), 2 deletions(-)

diff --git 
a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java 
b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java
index 7a20ff4..871f840 100644
--- 
a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java
+++ 
b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java
@@ -1043,7 +1043,8 @@ public class ConsumerImpl<T> extends ConsumerBase<T> 
implements ConnectionHandle
                 }
             }
 
-            if (isSameEntry(messageId) && 
isPriorEntryIndex(messageId.getEntryId())) {
+            // If the topic is non-persistent, we should not ignore any 
messages.
+            if (this.topicName.isPersistent() && isSameEntry(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,
@@ -1253,7 +1254,7 @@ public class ConsumerImpl<T> extends ConsumerBase<T> 
implements ConnectionHandle
                 ByteBuf singleMessagePayload = 
Commands.deSerializeSingleMessageInBatch(uncompressedPayload,
                         singleMessageMetadata, i, batchSize);
 
-                if (isSameEntry(messageId) && isPriorBatchIndex(i)) {
+                if (this.topicName.isPersistent() && isSameEntry(messageId) && 
isPriorBatchIndex(i)) {
                     // If we are receiving a batch message, we need to discard 
messages that were prior
                     // to the startMessageId
                     if (log.isDebugEnabled()) {
diff --git 
a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/messaging/ReaderMessagingTest.java
 
b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/messaging/ReaderMessagingTest.java
new file mode 100644
index 0000000..cdae130
--- /dev/null
+++ 
b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/messaging/ReaderMessagingTest.java
@@ -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);
+    }
+}
diff --git a/tests/integration/src/test/resources/pulsar-messaging.xml 
b/tests/integration/src/test/resources/pulsar-messaging.xml
index aa31852..b0e021f 100644
--- a/tests/integration/src/test/resources/pulsar-messaging.xml
+++ b/tests/integration/src/test/resources/pulsar-messaging.xml
@@ -25,6 +25,7 @@
             <class 
name="org.apache.pulsar.tests.integration.messaging.PersistentTopicMessagingTest"
 />
             <class 
name="org.apache.pulsar.tests.integration.messaging.NonPersistentTopicMessagingTest"
 />
             <class 
name="org.apache.pulsar.tests.integration.messaging.DelayMessagingTest" />
+            <class 
name="org.apache.pulsar.tests.integration.messaging.ReaderMessagingTest" />
             <class name="org.apache.pulsar.tests.integration.admin.AdminTest" 
/>
             <class 
name="org.apache.pulsar.tests.integration.io.sources.AvroKafkaSourceTest" />
         </classes>

Reply via email to