This is an automated email from the ASF dual-hosted git repository.
penghui pushed a commit to branch branch-2.10
in repository https://gitbox.apache.org/repos/asf/pulsar.git
The following commit(s) were added to refs/heads/branch-2.10 by this push:
new b4f8be4ada1 [fix][client] Fix the message present in incoming queue
after go to DLQ (#17326)
b4f8be4ada1 is described below
commit b4f8be4ada1ae3a666a9f75c4706c4133252a5c1
Author: congbo <[email protected]>
AuthorDate: Tue Aug 30 06:57:09 2022 +0800
[fix][client] Fix the message present in incoming queue after go to DLQ
(#17326)
(cherry picked from commit 2c1d2130e65a075f5a0bb261f5bd617e88d2e945)
---
.../org/apache/pulsar/client/impl/TransactionEndToEndTest.java | 4 ++++
.../main/java/org/apache/pulsar/client/impl/ConsumerImpl.java | 10 +++++++++-
2 files changed, 13 insertions(+), 1 deletion(-)
diff --git
a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java
b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java
index 6868f49fa14..273013d6f51 100644
---
a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java
+++
b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java
@@ -1159,6 +1159,8 @@ public class TransactionEndToEndTest extends
TransactionTestBase {
// the message will be sent to DLQ, can't receive
assertNull(consumer.receive(3, TimeUnit.SECONDS));
+ assertEquals(((ConsumerImpl<?>) consumer).getAvailablePermits(), 3);
+
assertEquals(value, new String(deadLetterConsumer.receive(3,
TimeUnit.SECONDS).getValue()));
}
@@ -1223,6 +1225,8 @@ public class TransactionEndToEndTest extends
TransactionTestBase {
// the message will be sent to DLQ, can't receive
assertNull(consumer.receive(3, TimeUnit.SECONDS));
+ assertEquals(((ConsumerImpl<?>) consumer).getAvailablePermits(), 6);
+
assertEquals(value1, new String(deadLetterConsumer.receive(3,
TimeUnit.SECONDS).getValue()));
assertEquals(value2, new String(deadLetterConsumer.receive(3,
TimeUnit.SECONDS).getValue()));
}
diff --git
a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java
b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java
index f48ae2ed280..1b30bdbdc77 100644
---
a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java
+++
b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java
@@ -1356,6 +1356,9 @@ public class ConsumerImpl<T> extends ConsumerBase<T>
implements ConnectionHandle
Collections.singletonList(message));
if (redeliveryCount >
deadLetterPolicy.getMaxRedeliverCount()) {
redeliverUnacknowledgedMessages(Collections.singleton(message.getMessageId()));
+ // The message is skipped due to reaching the max
redelivery count,
+ // so we need to increase the available permits
+ increaseAvailablePermits(cnx);
return;
}
}
@@ -1523,6 +1526,12 @@ public class ConsumerImpl<T> extends ConsumerBase<T>
implements ConnectionHandle
}
if (possibleToDeadLetter != null) {
possibleToDeadLetter.add(message);
+ // Skip the message which reaches the max redelivery count.
+ if (redeliveryCount >
deadLetterPolicy.getMaxRedeliverCount()) {
+ skippedMessages++;
+ continue;
+ }
+
}
executeNotifyCallback(message);
}
@@ -1540,7 +1549,6 @@ public class ConsumerImpl<T> extends ConsumerBase<T>
implements ConnectionHandle
possibleToDeadLetter);
if (redeliveryCount > deadLetterPolicy.getMaxRedeliverCount())
{
redeliverUnacknowledgedMessages(Collections.singleton(batchMessage));
- return;
}
}
}