You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by yo...@apache.org on 2021/04/14 02:26:27 UTC

[pulsar] 03/10: [Messaging] Fix delay message block (#10078)

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

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

commit 8ac9cfb8755bfd173ceaa4246bf5487f2c2c10e1
Author: ran <ga...@126.com>
AuthorDate: Fri Apr 9 17:37:22 2021 +0800

    [Messaging] Fix delay message block (#10078)
    
    Currently, in the docker environment, if the consumer enables the retry feature and sets the retry topic in DeadLetterPolicy, the consumer will be blocked after receive retry messages many times.
    
    The delay TimerTask may run before reaching the timeout, we could find out that the last log `Timer triggered` run after the log `Start timer in 4958 millis` but we cloud compute the duration between these two logs is `4936`, it's less than `4958`, so the hasMessageAvailable check is false, if there are no more messages to read the delay messages reading will be blocked. Please check logs below.
    
    (cherry picked from commit 31f831574c3a65bae0e76801facaf2deb0b17fbb)
---
 .../delayed/InMemoryDelayedDeliveryTracker.java    |   9 +-
 .../integration/messaging/DelayMessagingTest.java  | 105 +++++++++++++++++++++
 .../src/test/resources/pulsar-messaging.xml        |   4 +-
 3 files changed, 116 insertions(+), 2 deletions(-)

diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java
index 5c37b81..bfb5b70 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/InMemoryDelayedDeliveryTracker.java
@@ -90,7 +90,14 @@ public class InMemoryDelayedDeliveryTracker implements DelayedDeliveryTracker, T
      */
     @Override
     public boolean hasMessageAvailable() {
-        return !priorityQueue.isEmpty() && priorityQueue.peekN1() <= clock.millis();
+        // Avoid the TimerTask run before reach the timeout.
+        long cutOffTime = clock.millis() + tickTimeMillis;
+        boolean hasMessageAvailable = !priorityQueue.isEmpty() && priorityQueue.peekN1() <= cutOffTime;
+        if (!hasMessageAvailable) {
+            // prevent the first delay message later than cutoffTime
+            updateTimer();
+        }
+        return hasMessageAvailable;
     }
 
     /**
diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/messaging/DelayMessagingTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/messaging/DelayMessagingTest.java
new file mode 100644
index 0000000..8b06d1d
--- /dev/null
+++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/messaging/DelayMessagingTest.java
@@ -0,0 +1,105 @@
+/**
+ * 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 java.util.concurrent.TimeUnit;
+import lombok.Cleanup;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.DeadLetterPolicy;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.apache.pulsar.tests.integration.suites.PulsarTestSuite;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+/**
+ * Delay messaging test.
+ */
+@Slf4j
+public class DelayMessagingTest extends PulsarTestSuite {
+
+    @Test(dataProvider = "ServiceUrls")
+    public void delayMsgBlockTest(String serviceUrl) throws Exception {
+        String nsName = generateNamespaceName();
+        pulsarCluster.createNamespace(nsName);
+
+        String topic = generateTopicName(nsName, "testDelayMsgBlock", true);
+        pulsarCluster.createPartitionedTopic(topic, 3);
+
+        String retryTopic = topic + "-RETRY";
+        String deadLetterTopic = topic + "-DLT";
+
+        @Cleanup
+        PulsarClient pulsarClient = PulsarClient.builder().serviceUrl(serviceUrl).build();
+
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .create();
+
+        final int redeliverCnt = 10;
+        final int delayTimeSeconds = 5;
+        @Cleanup
+        Consumer<byte[]> consumer = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("test")
+                .subscriptionType(SubscriptionType.Shared)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .enableRetry(true)
+                .deadLetterPolicy(DeadLetterPolicy.builder()
+                        .maxRedeliverCount(redeliverCnt)
+                        .retryLetterTopic(retryTopic)
+                        .deadLetterTopic(deadLetterTopic)
+                        .build())
+                .receiverQueueSize(100)
+                .ackTimeout(60, TimeUnit.SECONDS)
+                .subscribe();
+
+        producer.newMessage().value("hello".getBytes()).send();
+
+        // receive message at first time
+        Message<byte[]> message = consumer.receive(delayTimeSeconds * 2, TimeUnit.SECONDS);
+        Assert.assertNotNull(message, "Can't receive message at the first time.");
+        consumer.reconsumeLater(message, delayTimeSeconds, TimeUnit.SECONDS);
+
+        // receive retry messages
+        for (int i = 0; i < redeliverCnt; i++) {
+            message = consumer.receive(delayTimeSeconds * 2, TimeUnit.SECONDS);
+            Assert.assertNotNull(message, "Consumer can't receive message in double delayTimeSeconds time "
+                    + delayTimeSeconds * 2 + "s");
+            log.info("receive msg. reConsumeTimes: {}", message.getProperty("RECONSUMETIMES"));
+            consumer.reconsumeLater(message, delayTimeSeconds, TimeUnit.SECONDS);
+        }
+
+        @Cleanup
+        Consumer<byte[]> dltConsumer = pulsarClient.newConsumer()
+                .topic(deadLetterTopic)
+                .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
+                .subscriptionName("test")
+                .subscribe();
+
+        message = dltConsumer.receive(10, TimeUnit.SECONDS);
+        Assert.assertNotNull(message, "Dead letter topic consumer can't receive message.");
+    }
+
+}
diff --git a/tests/integration/src/test/resources/pulsar-messaging.xml b/tests/integration/src/test/resources/pulsar-messaging.xml
index 4af0a33..9045413 100644
--- a/tests/integration/src/test/resources/pulsar-messaging.xml
+++ b/tests/integration/src/test/resources/pulsar-messaging.xml
@@ -24,6 +24,8 @@
         <classes>
             <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.io.AvroKafkaSourceTest" />
         </classes>
     </test>
-</suite>
\ No newline at end of file
+</suite>