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/10/08 09:10:17 UTC

[GitHub] [pulsar] codelipenghui commented on a diff in pull request #17318: [fix][client] Messages with inconsistent consumer epochs are not filtered when using batch receive and trigger timeout.

codelipenghui commented on code in PR #17318:
URL: https://github.com/apache/pulsar/pull/17318#discussion_r990615227


##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageRedeliveryTracker.java:
##########
@@ -71,9 +72,13 @@ public void run(Timeout t) throws Exception {
                     }
                     headPartition.clear();
                     redeliveryTimePartitions.addLast(headPartition);
-                    triggerRedelivery(consumerBase);
+                    messageIds = getRedeliveryMessages(consumerBase);
                 } finally {
                     writeLock.unlock();
+                    if (messageIds != null && !messageIds.isEmpty()) {
+                        consumerBase.onAckTimeoutSend(messageIds);
+                        consumerBase.redeliverUnacknowledgedMessages(messageIds);

Review Comment:
   @shibd Could you please provide more details of the deadlock?



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java:
##########
@@ -839,7 +839,7 @@ protected boolean canEnqueueMessage(Message<T> message) {
 
     protected boolean enqueueMessageAndCheckBatchReceive(Message<T> message) {
         int messageSize = message.size();
-        if (canEnqueueMessage(message) && incomingMessages.offer(message)) {
+        if (isValidConsumerEpoch(message) && canEnqueueMessage(message) && incomingMessages.offer(message)) {

Review Comment:
   And do we need to increase the available permits? Otherwise, the broker might stop dispatching messages to the consumer due to the consumer doesn't have available permits (the messages are skipped)



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/UnAckedMessageRedeliveryTracker.java:
##########
@@ -71,9 +72,13 @@ public void run(Timeout t) throws Exception {
                     }
                     headPartition.clear();
                     redeliveryTimePartitions.addLast(headPartition);
-                    triggerRedelivery(consumerBase);
+                    messageIds = getRedeliveryMessages(consumerBase);
                 } finally {
                     writeLock.unlock();
+                    if (messageIds != null && !messageIds.isEmpty()) {
+                        consumerBase.onAckTimeoutSend(messageIds);
+                        consumerBase.redeliverUnacknowledgedMessages(messageIds);

Review Comment:
   Oh, I think you mean you need to change triggerRedelivery -> getRedeliveryMessages because of the deadlock issue right?



##########
pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java:
##########
@@ -839,7 +839,7 @@ protected boolean canEnqueueMessage(Message<T> message) {
 
     protected boolean enqueueMessageAndCheckBatchReceive(Message<T> message) {
         int messageSize = message.size();
-        if (canEnqueueMessage(message) && incomingMessages.offer(message)) {
+        if (isValidConsumerEpoch(message) && canEnqueueMessage(message) && incomingMessages.offer(message)) {

Review Comment:
   If the consumer has pending receive requests. The message will be used to complete the pending receive requests directly. 
   
   ```java
    internalPinnedExecutor.execute(() -> {
               if (hasNextPendingReceive()) {
                   notifyPendingReceivedCallback(message, null);
               } else if (enqueueMessageAndCheckBatchReceive(message) && hasPendingBatchReceive()) {
                   notifyPendingBatchReceivedCallBack();
               }
           });
   ```
   
   In this case, we are not able to filter out the messages that with unexpected epoch?



-- 
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