You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2022/09/06 05:22:58 UTC

[GitHub] [pulsar] mattisonchao opened a new pull request, #17483: [fix][broker] StickyKey Dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

mattisonchao opened a new pull request, #17483:
URL: https://github.com/apache/pulsar/pull/17483

   ### Motivation
   
   See error stack trace:
   
   ```java
   [Unreachable] char[1024] “2022-08-31T08:05:02,202+0000 [BookKeeperClientWorker-OrderedExecutor-0-0] ERROR org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - Unknown exception for ManagedLedgerException.
   java.lang.ArrayIndexOutOfBoundsException: -22012
   	at java.util.ArrayList.elementData(ArrayList.java:424) ~[?:1.8.0_322]
   	at java.util.ArrayList.get(ArrayList.java:437) ~[?:1.8.0_322]
   	at org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers.sendMessagesToConsumers(PersistentStickyKeyDispatcherMultipleConsumers.java:256)
   	at org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers.readEntriesComplete(PersistentDispatcherMultipleConsumers.java:517)
   	at org.apache.bookkeeper.mledger.impl.ManagedCursorImpl$12.readEntryComplete(ManagedCursorImpl.java:1392)
   	at org.apache.bookkeeper.mledger.impl.EntryCacheImpl.”
   ```
   Relative code:
   
   https://github.com/apache/pulsar/blob/cfe95ddacc16bd28cf076a7374dff06012f9d21d/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java#L237-L262
   
   When `unackedMessages` is greater than `maxUnackedMessages`, because `messagesForC` is a negative number, `ArrayIndexOutOfBoundsException` will be thrown on line 254. Then the exception will cause the message not to be delivered.
   
   ### Modifications
   
   - Add check to avoid negative results
   
   ### Verifying this change
   
   - [x] Make sure that the change passes the CI checks.
   
   ### Documentation
   
   - [x] `doc-not-needed` 
   (Please explain why)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] Jason918 commented on a diff in pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
Jason918 commented on code in PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#discussion_r965662569


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java:
##########
@@ -0,0 +1,183 @@
+/**
+ * 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.client.impl;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import lombok.Cleanup;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.awaitility.Awaitility;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+@Test(groups = "broker-impl")
+public class KeySharedSubscriptionTest extends ProducerConsumerBase {
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        conf.setMaxUnackedMessagesPerConsumer(1000);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @Override
+    @AfterMethod(alwaysRun = true)
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction()
+            throws PulsarClientException {
+        PulsarClient pulsarClient = PulsarClient.builder().
+                serviceUrl(lookupUrl.toString())
+                .build();
+        final int totalMsg = 50000;
+        String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5);
+        Map<Consumer<?>, List<MessageId>> nameToId = Maps.newConcurrentMap();
+        Set<MessageId> pubMessages = Sets.newConcurrentHashSet();
+        Set<MessageId> recMessages = Sets.newConcurrentHashSet();
+        AtomicLong lastActiveTime = new AtomicLong();
+        AtomicBoolean canAcknowledgement = new AtomicBoolean(false);
+
+        @Cleanup
+        Consumer<byte[]> consumer1 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .consumerName("con-1")
+                .messageListener((cons1, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons1,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    recMessages.add(msg.getMessageId());
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons1.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer2 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons2, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons2,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    recMessages.add(msg.getMessageId());
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons2.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-2")
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer3 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons3, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons3,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    recMessages.add(msg.getMessageId());
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons3.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-3")
+                .subscribe();
+
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)
+                .batchingMaxPublishDelay(5, TimeUnit.MILLISECONDS)
+                // We chose 999 because the maximum unacked message is 1000
+                .batchingMaxMessages(999)
+                .create();
+
+        for (int i = 0; i < totalMsg; i++) {

Review Comment:
   Is it possible to reduce the number of `totalMsg`, this seems would increase unit test load significantly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on code in PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#discussion_r965545438


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java:
##########
@@ -0,0 +1,170 @@
+/**
+ * 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.client.impl;
+
+import com.google.common.collect.Maps;
+import lombok.Cleanup;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.pulsar.client.api.*;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+@Test(groups = "broker-impl")
+public class KeySharedSubscriptionTest extends ProducerConsumerBase {
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        conf.setMaxUnackedMessagesPerConsumer(1000);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @Override
+    @AfterMethod(alwaysRun = true)
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction()
+            throws PulsarClientException {
+        PulsarClient pulsarClient = PulsarClient.builder().
+                serviceUrl(lookupUrl.toString())
+                .build();
+        String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5);
+        Map<Consumer<?>, List<MessageId>> nameToId = Maps.newConcurrentMap();
+        AtomicLong lastActiveTime = new AtomicLong();
+        AtomicInteger msgCount = new AtomicInteger();
+        AtomicBoolean canAcknowledgement = new AtomicBoolean(false);
+
+        @Cleanup
+        Consumer<byte[]> consumer1 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .consumerName("con-1")
+                .messageListener((cons1, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons1,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons1.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer2 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons2, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons2,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons2.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-2")
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer3 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons3, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons3,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons3.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-3")
+                .subscribe();
+
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)
+                .create();
+
+        for (int i = 0; i < 50000; i++) {
+            producer.sendAsync(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8));
+        }
+
+        // Wait for all consumers can not read more messages. the consumers are stuck by max unacked messages.
+        while (true) {
+            long differentTime = System.currentTimeMillis() - lastActiveTime.get();
+            if (differentTime > TimeUnit.SECONDS.toMillis(20)) {
+                break;
+            }
+        }
+
+        // All consumers can acknowledge messages as they continue to receive messages.
+        canAcknowledgement.set(true);
+
+        // Acknowledgment of currently received messages to get out of stuck state due to unack message
+        for (Map.Entry<Consumer<?>, List<MessageId>> entry : nameToId.entrySet()) {
+            Consumer<?> consumer = entry.getKey();
+            consumer.acknowledge(entry.getValue());
+        }
+        // refresh active time
+        lastActiveTime.set(System.currentTimeMillis());
+
+        // Wait for all consumers to continue receiving messages.
+        while (true) {
+            long differentTime = System.currentTimeMillis() - lastActiveTime.get();
+            if (differentTime > TimeUnit.SECONDS.toMillis(20)) {
+                break;
+            }
+        }
+
+        //Determine if all messages have been received.
+        //If the dispatcher is stuck, we can not receive enough messages.
+        Assert.assertEquals(msgCount.get(), 50000);

Review Comment:
   Maybe we need to check the messages not only the messages count, for example if we missed 2 messages, but 2 messages is consumed duplicated, the test will also get passed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on code in PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#discussion_r965544665


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java:
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.client.impl;
+
+import com.google.common.collect.Maps;
+import lombok.Cleanup;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.pulsar.client.api.*;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+@Test(groups = "broker-impl")
+public class KeySharedSubscriptionTest extends ProducerConsumerBase {
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        conf.setMaxUnackedMessagesPerConsumer(1000);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @Override
+    @AfterMethod(alwaysRun = true)
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testKeyShareSubscriptionWillNotStuck() throws PulsarClientException {
+        PulsarClient pulsarClient = PulsarClient.builder().
+                serviceUrl(lookupUrl.toString())
+                .build();
+        String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5);
+        Map<Consumer<?>, List<MessageId>> nameToId = Maps.newConcurrentMap();
+        AtomicLong lastActiveTime = new AtomicLong();
+        AtomicInteger msgCount = new AtomicInteger();
+        AtomicBoolean canAcknowledgement = new AtomicBoolean(false);
+
+        @Cleanup
+        Consumer<byte[]> consumer1 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .consumerName("con-1")
+                .messageListener((cons1, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons1,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons1.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer2 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons2, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons2,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons2.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-2")
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer3 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons3, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons3,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons3.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-3")
+                .subscribe();
+
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)
+                .create();
+        for (int i = 0; i < 50000; i++) {
+            producer.sendAsync(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8));
+        }
+
+        while (true) {
+            long differentTime = System.currentTimeMillis() - lastActiveTime.get();
+            if (differentTime > TimeUnit.SECONDS.toMillis(20)) {
+                break;
+            }
+        }

Review Comment:
   I see. Can we use Awaitiblity here? 
   We don't have any poll interval for the while loop, it looks like busy loop which will waste CPU.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] mattisonchao commented on a diff in pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#discussion_r965509716


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java:
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.client.impl;
+
+import com.google.common.collect.Maps;
+import lombok.Cleanup;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.pulsar.client.api.*;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+@Test(groups = "broker-impl")
+public class KeySharedSubscriptionTest extends ProducerConsumerBase {
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        conf.setMaxUnackedMessagesPerConsumer(1000);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @Override
+    @AfterMethod(alwaysRun = true)
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testKeyShareSubscriptionWillNotStuck() throws PulsarClientException {
+        PulsarClient pulsarClient = PulsarClient.builder().
+                serviceUrl(lookupUrl.toString())
+                .build();
+        String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5);
+        Map<Consumer<?>, List<MessageId>> nameToId = Maps.newConcurrentMap();
+        AtomicLong lastActiveTime = new AtomicLong();
+        AtomicInteger msgCount = new AtomicInteger();
+        AtomicBoolean canAcknowledgement = new AtomicBoolean(false);
+
+        @Cleanup
+        Consumer<byte[]> consumer1 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .consumerName("con-1")
+                .messageListener((cons1, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons1,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons1.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer2 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons2, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons2,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons2.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-2")
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer3 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons3, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons3,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons3.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-3")
+                .subscribe();
+
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)

Review Comment:
   The default delay is enough to reproduce the problem stably. 



##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java:
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.client.impl;
+
+import com.google.common.collect.Maps;
+import lombok.Cleanup;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.pulsar.client.api.*;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+@Test(groups = "broker-impl")
+public class KeySharedSubscriptionTest extends ProducerConsumerBase {
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        conf.setMaxUnackedMessagesPerConsumer(1000);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @Override
+    @AfterMethod(alwaysRun = true)
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testKeyShareSubscriptionWillNotStuck() throws PulsarClientException {
+        PulsarClient pulsarClient = PulsarClient.builder().
+                serviceUrl(lookupUrl.toString())
+                .build();
+        String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5);
+        Map<Consumer<?>, List<MessageId>> nameToId = Maps.newConcurrentMap();
+        AtomicLong lastActiveTime = new AtomicLong();
+        AtomicInteger msgCount = new AtomicInteger();
+        AtomicBoolean canAcknowledgement = new AtomicBoolean(false);
+
+        @Cleanup
+        Consumer<byte[]> consumer1 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .consumerName("con-1")
+                .messageListener((cons1, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons1,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons1.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer2 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons2, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons2,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons2.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-2")
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer3 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons3, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons3,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons3.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-3")
+                .subscribe();
+
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)
+                .create();
+        for (int i = 0; i < 50000; i++) {
+            producer.sendAsync(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8));
+        }
+
+        while (true) {
+            long differentTime = System.currentTimeMillis() - lastActiveTime.get();
+            if (differentTime > TimeUnit.SECONDS.toMillis(20)) {
+                break;
+            }
+        }

Review Comment:
   fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] lhotari commented on pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
lhotari commented on PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#issuecomment-1237725441

   ```
   at org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers.sendMessagesToConsumers(PersistentStickyKeyDispatcherMultipleConsumers.java:256)
   ```
   which Pulsar version is this stack trace from?
   
   Would it be possible to add a test that reproduces the problem using the public APIs? If this happens when there are more unknowledged messages than the defined limit, I'd assume that it's fairly easy to reproduce?
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] mattisonchao commented on a diff in pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#discussion_r966675008


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java:
##########
@@ -634,7 +634,9 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis
             // round-robin dispatch batch size for this consumer
             int availablePermits = c.isWritable() ? c.getAvailablePermits() : 1;
             if (c.getMaxUnackedMessages() > 0) {
-                availablePermits = Math.min(availablePermits, c.getMaxUnackedMessages() - c.getUnackedMessages());
+                // Avoid negative number
+                int remainUnAckedMessages = Math.max(c.getMaxUnackedMessages() - c.getUnackedMessages(), 0);

Review Comment:
   >Do we understand why c.getUnackedMessages() is greater than c.getMaxUnackedMessages()?
   
   Because in the current implementation, we will add `unackedMessages` after sent messages. if we use batch messages, the `unackedMessages` is probably greater than `MaxUnackedMessages`.
   
   Plus, #16670 #16718 want to add this check to limit the consumer.
   
   >Based on the names, that seems like there is another bug if c.getMaxUnackedMessages() - c.getUnackedMessages() is negative.
   
   Sure, But I find we have two sections using it. And I fix It all in this PR.
   
   >Perhaps the idea that the maxUnackedMessages is just a soft limit that we shouldn't exceed by too much?
   
   Yes, maybe we have another method to help more robust support it, but it looks like it is another improvement.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] codelipenghui commented on pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#issuecomment-1237748010

   @mattisonchao I think the issue is introduced by https://github.com/apache/pulsar/pull/16718? it should be a blocker for 2.11.0 release.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] mattisonchao commented on a diff in pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#discussion_r965509716


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java:
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.client.impl;
+
+import com.google.common.collect.Maps;
+import lombok.Cleanup;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.pulsar.client.api.*;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+@Test(groups = "broker-impl")
+public class KeySharedSubscriptionTest extends ProducerConsumerBase {
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        conf.setMaxUnackedMessagesPerConsumer(1000);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @Override
+    @AfterMethod(alwaysRun = true)
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testKeyShareSubscriptionWillNotStuck() throws PulsarClientException {
+        PulsarClient pulsarClient = PulsarClient.builder().
+                serviceUrl(lookupUrl.toString())
+                .build();
+        String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5);
+        Map<Consumer<?>, List<MessageId>> nameToId = Maps.newConcurrentMap();
+        AtomicLong lastActiveTime = new AtomicLong();
+        AtomicInteger msgCount = new AtomicInteger();
+        AtomicBoolean canAcknowledgement = new AtomicBoolean(false);
+
+        @Cleanup
+        Consumer<byte[]> consumer1 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .consumerName("con-1")
+                .messageListener((cons1, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons1,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons1.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer2 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons2, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons2,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons2.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-2")
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer3 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons3, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons3,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons3.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-3")
+                .subscribe();
+
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)

Review Comment:
   The default delay is enough to reproduce the problem.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] mattisonchao commented on a diff in pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#discussion_r965509716


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java:
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.client.impl;
+
+import com.google.common.collect.Maps;
+import lombok.Cleanup;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.pulsar.client.api.*;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+@Test(groups = "broker-impl")
+public class KeySharedSubscriptionTest extends ProducerConsumerBase {
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        conf.setMaxUnackedMessagesPerConsumer(1000);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @Override
+    @AfterMethod(alwaysRun = true)
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testKeyShareSubscriptionWillNotStuck() throws PulsarClientException {
+        PulsarClient pulsarClient = PulsarClient.builder().
+                serviceUrl(lookupUrl.toString())
+                .build();
+        String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5);
+        Map<Consumer<?>, List<MessageId>> nameToId = Maps.newConcurrentMap();
+        AtomicLong lastActiveTime = new AtomicLong();
+        AtomicInteger msgCount = new AtomicInteger();
+        AtomicBoolean canAcknowledgement = new AtomicBoolean(false);
+
+        @Cleanup
+        Consumer<byte[]> consumer1 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .consumerName("con-1")
+                .messageListener((cons1, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons1,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons1.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer2 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons2, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons2,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons2.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-2")
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer3 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons3, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons3,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons3.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-3")
+                .subscribe();
+
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)

Review Comment:
   The default delay is enough to reproduce the problem stably.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] mattisonchao merged pull request #17483: [fix][broker] Multiple consumer dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
mattisonchao merged PR #17483:
URL: https://github.com/apache/pulsar/pull/17483


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] michaeljmarshall commented on a diff in pull request #17483: [fix][broker] Multiple consumer dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on code in PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#discussion_r967673174


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java:
##########
@@ -634,7 +634,9 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis
             // round-robin dispatch batch size for this consumer
             int availablePermits = c.isWritable() ? c.getAvailablePermits() : 1;
             if (c.getMaxUnackedMessages() > 0) {
-                availablePermits = Math.min(availablePermits, c.getMaxUnackedMessages() - c.getUnackedMessages());
+                // Avoid negative number
+                int remainUnAckedMessages = Math.max(c.getMaxUnackedMessages() - c.getUnackedMessages(), 0);

Review Comment:
   > Because in the current implementation, we will add `unackedMessages` after sent messages. if we use batch messages, the `unackedMessages` is probably greater than `MaxUnackedMessages`.
   
   This was the detail I was missing, thank you for explaining. I read through the code a bit more, and it makes sense.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on code in PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#discussion_r965484551


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java:
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.client.impl;
+
+import com.google.common.collect.Maps;
+import lombok.Cleanup;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.pulsar.client.api.*;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+@Test(groups = "broker-impl")
+public class KeySharedSubscriptionTest extends ProducerConsumerBase {
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        conf.setMaxUnackedMessagesPerConsumer(1000);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @Override
+    @AfterMethod(alwaysRun = true)
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testKeyShareSubscriptionWillNotStuck() throws PulsarClientException {
+        PulsarClient pulsarClient = PulsarClient.builder().
+                serviceUrl(lookupUrl.toString())
+                .build();
+        String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5);
+        Map<Consumer<?>, List<MessageId>> nameToId = Maps.newConcurrentMap();
+        AtomicLong lastActiveTime = new AtomicLong();
+        AtomicInteger msgCount = new AtomicInteger();
+        AtomicBoolean canAcknowledgement = new AtomicBoolean(false);
+
+        @Cleanup
+        Consumer<byte[]> consumer1 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .consumerName("con-1")
+                .messageListener((cons1, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons1,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons1.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer2 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons2, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons2,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons2.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-2")
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer3 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons3, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons3,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons3.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-3")
+                .subscribe();
+
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)

Review Comment:
   Do you need to change the batch delay?
   To make sure we have always sending large batches.



##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java:
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.client.impl;
+
+import com.google.common.collect.Maps;
+import lombok.Cleanup;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.pulsar.client.api.*;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+@Test(groups = "broker-impl")
+public class KeySharedSubscriptionTest extends ProducerConsumerBase {
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        conf.setMaxUnackedMessagesPerConsumer(1000);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @Override
+    @AfterMethod(alwaysRun = true)
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testKeyShareSubscriptionWillNotStuck() throws PulsarClientException {
+        PulsarClient pulsarClient = PulsarClient.builder().
+                serviceUrl(lookupUrl.toString())
+                .build();
+        String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5);
+        Map<Consumer<?>, List<MessageId>> nameToId = Maps.newConcurrentMap();
+        AtomicLong lastActiveTime = new AtomicLong();
+        AtomicInteger msgCount = new AtomicInteger();
+        AtomicBoolean canAcknowledgement = new AtomicBoolean(false);
+
+        @Cleanup
+        Consumer<byte[]> consumer1 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .consumerName("con-1")
+                .messageListener((cons1, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons1,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons1.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer2 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons2, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons2,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons2.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-2")
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer3 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons3, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons3,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons3.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-3")
+                .subscribe();
+
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)
+                .create();
+        for (int i = 0; i < 50000; i++) {
+            producer.sendAsync(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8));
+        }
+
+        while (true) {
+            long differentTime = System.currentTimeMillis() - lastActiveTime.get();
+            if (differentTime > TimeUnit.SECONDS.toMillis(20)) {
+                break;
+            }
+        }

Review Comment:
   Could you please help add some description? I'm sure how the code will affect the test.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] lhotari commented on pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
lhotari commented on PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#issuecomment-1237728039

   I'm just wondering if there's an off-by-one error in this logic when `messagesForC == 0`
   
   https://github.com/apache/pulsar/blob/825b68db7bed1c79af4b7b69b48bee76ebe75af5/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java#L250-L260


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] mattisonchao commented on a diff in pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#discussion_r965576612


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java:
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.client.impl;
+
+import com.google.common.collect.Maps;
+import lombok.Cleanup;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.pulsar.client.api.*;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+@Test(groups = "broker-impl")
+public class KeySharedSubscriptionTest extends ProducerConsumerBase {
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        conf.setMaxUnackedMessagesPerConsumer(1000);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @Override
+    @AfterMethod(alwaysRun = true)
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testKeyShareSubscriptionWillNotStuck() throws PulsarClientException {
+        PulsarClient pulsarClient = PulsarClient.builder().
+                serviceUrl(lookupUrl.toString())
+                .build();
+        String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5);
+        Map<Consumer<?>, List<MessageId>> nameToId = Maps.newConcurrentMap();
+        AtomicLong lastActiveTime = new AtomicLong();
+        AtomicInteger msgCount = new AtomicInteger();
+        AtomicBoolean canAcknowledgement = new AtomicBoolean(false);
+
+        @Cleanup
+        Consumer<byte[]> consumer1 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .consumerName("con-1")
+                .messageListener((cons1, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons1,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons1.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer2 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons2, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons2,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons2.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-2")
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer3 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons3, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons3,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons3.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-3")
+                .subscribe();
+
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)
+                .create();
+        for (int i = 0; i < 50000; i++) {
+            producer.sendAsync(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8));
+        }
+
+        while (true) {
+            long differentTime = System.currentTimeMillis() - lastActiveTime.get();
+            if (differentTime > TimeUnit.SECONDS.toMillis(20)) {
+                break;
+            }
+        }

Review Comment:
   good idea.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] codelipenghui commented on a diff in pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on code in PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#discussion_r965543053


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java:
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.client.impl;
+
+import com.google.common.collect.Maps;
+import lombok.Cleanup;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.pulsar.client.api.*;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+@Test(groups = "broker-impl")
+public class KeySharedSubscriptionTest extends ProducerConsumerBase {
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        conf.setMaxUnackedMessagesPerConsumer(1000);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @Override
+    @AfterMethod(alwaysRun = true)
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testKeyShareSubscriptionWillNotStuck() throws PulsarClientException {
+        PulsarClient pulsarClient = PulsarClient.builder().
+                serviceUrl(lookupUrl.toString())
+                .build();
+        String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5);
+        Map<Consumer<?>, List<MessageId>> nameToId = Maps.newConcurrentMap();
+        AtomicLong lastActiveTime = new AtomicLong();
+        AtomicInteger msgCount = new AtomicInteger();
+        AtomicBoolean canAcknowledgement = new AtomicBoolean(false);
+
+        @Cleanup
+        Consumer<byte[]> consumer1 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .consumerName("con-1")
+                .messageListener((cons1, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons1,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons1.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer2 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons2, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons2,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons2.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-2")
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer3 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons3, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons3,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons3.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-3")
+                .subscribe();
+
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)

Review Comment:
   But, we don't know what will happen in the CI environment. Using the fixed batch size can avoid the flaky test.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] mattisonchao commented on pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#issuecomment-1240134936

   @lhotari  Test added, Please take a look.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] michaeljmarshall commented on a diff in pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on code in PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#discussion_r966562277


##########
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java:
##########
@@ -634,7 +634,9 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis
             // round-robin dispatch batch size for this consumer
             int availablePermits = c.isWritable() ? c.getAvailablePermits() : 1;
             if (c.getMaxUnackedMessages() > 0) {
-                availablePermits = Math.min(availablePermits, c.getMaxUnackedMessages() - c.getUnackedMessages());
+                // Avoid negative number
+                int remainUnAckedMessages = Math.max(c.getMaxUnackedMessages() - c.getUnackedMessages(), 0);

Review Comment:
   Do we understand why `c.getUnackedMessages()` is greater than `c.getMaxUnackedMessages()`? I read through the test, but I didn't see what is causing the negative `availablePermits`.
   
   Based on the names, that seems like there is another bug if `c.getMaxUnackedMessages() - c.getUnackedMessages()` is negative. Perhaps the idea that the `maxUnackedMessages` is just a soft limit that we shouldn't exceed by too much?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] mattisonchao commented on a diff in pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#discussion_r965704841


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java:
##########
@@ -0,0 +1,183 @@
+/**
+ * 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.client.impl;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import lombok.Cleanup;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.ProducerConsumerBase;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.awaitility.Awaitility;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+@Test(groups = "broker-impl")
+public class KeySharedSubscriptionTest extends ProducerConsumerBase {
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        conf.setMaxUnackedMessagesPerConsumer(1000);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @Override
+    @AfterMethod(alwaysRun = true)
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction()
+            throws PulsarClientException {
+        PulsarClient pulsarClient = PulsarClient.builder().
+                serviceUrl(lookupUrl.toString())
+                .build();
+        final int totalMsg = 50000;
+        String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5);
+        Map<Consumer<?>, List<MessageId>> nameToId = Maps.newConcurrentMap();
+        Set<MessageId> pubMessages = Sets.newConcurrentHashSet();
+        Set<MessageId> recMessages = Sets.newConcurrentHashSet();
+        AtomicLong lastActiveTime = new AtomicLong();
+        AtomicBoolean canAcknowledgement = new AtomicBoolean(false);
+
+        @Cleanup
+        Consumer<byte[]> consumer1 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .consumerName("con-1")
+                .messageListener((cons1, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons1,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    recMessages.add(msg.getMessageId());
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons1.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer2 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons2, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons2,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    recMessages.add(msg.getMessageId());
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons2.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-2")
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer3 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons3, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons3,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    recMessages.add(msg.getMessageId());
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons3.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-3")
+                .subscribe();
+
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)
+                .batchingMaxPublishDelay(5, TimeUnit.MILLISECONDS)
+                // We chose 999 because the maximum unacked message is 1000
+                .batchingMaxMessages(999)
+                .create();
+
+        for (int i = 0; i < totalMsg; i++) {

Review Comment:
   fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [pulsar] mattisonchao commented on a diff in pull request #17483: [fix][broker] Sticky key dispatcher stuck when `unackedMessages` greater than `maxUnackedMessages`

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on code in PR #17483:
URL: https://github.com/apache/pulsar/pull/17483#discussion_r965566058


##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java:
##########
@@ -0,0 +1,162 @@
+/**
+ * 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.client.impl;
+
+import com.google.common.collect.Maps;
+import lombok.Cleanup;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.pulsar.client.api.*;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+@Test(groups = "broker-impl")
+public class KeySharedSubscriptionTest extends ProducerConsumerBase {
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        conf.setMaxUnackedMessagesPerConsumer(1000);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @Override
+    @AfterMethod(alwaysRun = true)
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testKeyShareSubscriptionWillNotStuck() throws PulsarClientException {
+        PulsarClient pulsarClient = PulsarClient.builder().
+                serviceUrl(lookupUrl.toString())
+                .build();
+        String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5);
+        Map<Consumer<?>, List<MessageId>> nameToId = Maps.newConcurrentMap();
+        AtomicLong lastActiveTime = new AtomicLong();
+        AtomicInteger msgCount = new AtomicInteger();
+        AtomicBoolean canAcknowledgement = new AtomicBoolean(false);
+
+        @Cleanup
+        Consumer<byte[]> consumer1 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .consumerName("con-1")
+                .messageListener((cons1, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons1,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons1.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer2 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons2, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons2,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons2.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-2")
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer3 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons3, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons3,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons3.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-3")
+                .subscribe();
+
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)

Review Comment:
   fixed.



##########
pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeySharedSubscriptionTest.java:
##########
@@ -0,0 +1,170 @@
+/**
+ * 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.client.impl;
+
+import com.google.common.collect.Maps;
+import lombok.Cleanup;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.pulsar.client.api.*;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+@Test(groups = "broker-impl")
+public class KeySharedSubscriptionTest extends ProducerConsumerBase {
+
+    @Override
+    @BeforeMethod
+    protected void setup() throws Exception {
+        conf.setMaxUnackedMessagesPerConsumer(1000);
+        super.internalSetup();
+        super.producerBaseSetup();
+    }
+
+    @Override
+    @AfterMethod(alwaysRun = true)
+    protected void cleanup() throws Exception {
+        super.internalCleanup();
+    }
+
+    @Test
+    public void testCanRecoverConsumptionWhenLiftMaxUnAckedMessagesRestriction()
+            throws PulsarClientException {
+        PulsarClient pulsarClient = PulsarClient.builder().
+                serviceUrl(lookupUrl.toString())
+                .build();
+        String topic = "broker-close-test-" + RandomStringUtils.randomAlphabetic(5);
+        Map<Consumer<?>, List<MessageId>> nameToId = Maps.newConcurrentMap();
+        AtomicLong lastActiveTime = new AtomicLong();
+        AtomicInteger msgCount = new AtomicInteger();
+        AtomicBoolean canAcknowledgement = new AtomicBoolean(false);
+
+        @Cleanup
+        Consumer<byte[]> consumer1 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .consumerName("con-1")
+                .messageListener((cons1, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons1,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons1.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer2 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons2, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons2,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons2.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-2")
+                .subscribe();
+        @Cleanup
+        Consumer<byte[]> consumer3 = pulsarClient.newConsumer()
+                .topic(topic)
+                .subscriptionName("sub-1")
+                .subscriptionType(SubscriptionType.Key_Shared)
+                .messageListener((cons3, msg) -> {
+                    lastActiveTime.set(System.currentTimeMillis());
+                    nameToId.computeIfAbsent(cons3,(k) -> new ArrayList<>())
+                            .add(msg.getMessageId());
+                    msgCount.incrementAndGet();
+                    if (canAcknowledgement.get()) {
+                        try {
+                            cons3.acknowledge(msg);
+                        } catch (PulsarClientException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                })
+                .consumerName("con-3")
+                .subscribe();
+
+        @Cleanup
+        Producer<byte[]> producer = pulsarClient.newProducer()
+                .topic(topic)
+                .enableBatching(true)
+                .create();
+
+        for (int i = 0; i < 50000; i++) {
+            producer.sendAsync(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8));
+        }
+
+        // Wait for all consumers can not read more messages. the consumers are stuck by max unacked messages.
+        while (true) {
+            long differentTime = System.currentTimeMillis() - lastActiveTime.get();
+            if (differentTime > TimeUnit.SECONDS.toMillis(20)) {
+                break;
+            }
+        }
+
+        // All consumers can acknowledge messages as they continue to receive messages.
+        canAcknowledgement.set(true);
+
+        // Acknowledgment of currently received messages to get out of stuck state due to unack message
+        for (Map.Entry<Consumer<?>, List<MessageId>> entry : nameToId.entrySet()) {
+            Consumer<?> consumer = entry.getKey();
+            consumer.acknowledge(entry.getValue());
+        }
+        // refresh active time
+        lastActiveTime.set(System.currentTimeMillis());
+
+        // Wait for all consumers to continue receiving messages.
+        while (true) {
+            long differentTime = System.currentTimeMillis() - lastActiveTime.get();
+            if (differentTime > TimeUnit.SECONDS.toMillis(20)) {
+                break;
+            }
+        }
+
+        //Determine if all messages have been received.
+        //If the dispatcher is stuck, we can not receive enough messages.
+        Assert.assertEquals(msgCount.get(), 50000);

Review Comment:
   fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org