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

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


The following commit(s) were added to refs/heads/branch-2.9 by this push:
     new 5f777fb7b50 [fix][client] Fix failover/exclusive consumer with batch 
cumulate ack issue. (#18454)
5f777fb7b50 is described below

commit 5f777fb7b506c9bae1f209e553dbfffece34854c
Author: Jiwei Guo <[email protected]>
AuthorDate: Tue Nov 15 11:37:16 2022 +0800

    [fix][client] Fix failover/exclusive consumer with batch cumulate ack 
issue. (#18454)
    
    (cherry picked from commit 7712aa396bfca55a79a45761e0a405e90185e0f8)
---
 .../impl/ConsumerDedupPermitsUpdateTest.java       | 21 ++++--
 .../pulsar/client/impl/NegativeAcksTest.java       | 83 +++++++++++++++++++++-
 .../apache/pulsar/client/impl/ConsumerImpl.java    |  9 ++-
 .../client/impl/TypedMessageBuilderImpl.java       |  2 +
 4 files changed, 105 insertions(+), 10 deletions(-)

diff --git 
a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerDedupPermitsUpdateTest.java
 
b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerDedupPermitsUpdateTest.java
index 4c9922acbec..ceb7d7fd484 100644
--- 
a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerDedupPermitsUpdateTest.java
+++ 
b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConsumerDedupPermitsUpdateTest.java
@@ -116,10 +116,23 @@ public class ConsumerDedupPermitsUpdateTest extends 
ProducerConsumerBase {
         }
         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);
+            }
         }
     }
 
diff --git 
a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/NegativeAcksTest.java
 
b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/NegativeAcksTest.java
index 5eb43af38f7..ced0bec8fcb 100644
--- 
a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/NegativeAcksTest.java
+++ 
b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/NegativeAcksTest.java
@@ -18,11 +18,9 @@
  */
 package org.apache.pulsar.client.impl;
 
-import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertNull;
-
 import java.util.HashSet;
 import java.util.Set;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
 import lombok.Cleanup;
@@ -35,11 +33,14 @@ import org.apache.pulsar.client.api.Producer;
 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;
 import org.testng.annotations.Test;
 
+import static org.testng.Assert.*;
+
 @Slf4j
 @Test(groups = "broker-impl")
 public class NegativeAcksTest extends ProducerConsumerBase {
@@ -154,4 +155,80 @@ public class NegativeAcksTest extends ProducerConsumerBase 
{
         consumer.close();
         producer.close();
     }
+
+    @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++;
+        }
+        // because don't have the redeliver epoch, so receive the message 
count is bigger than 9
+        assertTrue(count > 9);
+        Assert.assertEquals(0, datas.size());
+    }
 }
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 94531479236..89e434d41d4 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
@@ -1175,9 +1175,9 @@ public class ConsumerImpl<T> extends ConsumerBase<T> 
implements ConnectionHandle
         final int numMessages = msgMetadata.getNumMessagesInBatch();
         final int numChunks = msgMetadata.hasNumChunksFromMsg() ? 
msgMetadata.getNumChunksFromMsg() : 0;
         final boolean isChunkedMessage = numChunks > 1 && 
conf.getSubscriptionType() != SubscriptionType.Shared;
-
         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);
@@ -1430,7 +1430,10 @@ public class ConsumerImpl<T> extends ConsumerBase<T> 
implements ConnectionHandle
                         skippedMessages++;
                         continue;
                     }
-
+                }
+                if 
(acknowledgmentsGroupingTracker.isDuplicate(message.getMessageId())) {
+                    skippedMessages++;
+                    continue;
                 }
                 executeNotifyCallback(message);
             }
diff --git 
a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java
 
b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java
index 329861b6d0c..acc60cfcfe7 100644
--- 
a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java
+++ 
b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java
@@ -29,6 +29,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.pulsar.client.api.Message;
 import org.apache.pulsar.client.api.MessageId;
 import org.apache.pulsar.client.api.PulsarClientException;

Reply via email to