Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[pulsar-broker] Stop to dispatch when skip message temporally since Key_Shared consumer stuck on delivery #7553

Merged

Conversation

equanz
Copy link
Contributor

@equanz equanz commented Jul 16, 2020

Motivation

In some case of Key_Shared consumer, messages ordering was broken.
Here is how to reproduce(I think it is one of case to reproduce this issue).

  1. Connect Consumer1 to Key_Shared subscription sub and stop to receive
    • receiverQueueSize: 500
  2. Connect Producer and publish 500 messages with key (i % 10)
  3. Connect Consumer2 to same subscription and start to receive
  4. Producer publish more 500 messages with same key generation algorithm
  5. After that, Consumer1 start to receive
  6. Check Consumer2 message ordering
    • sometimes message ordering was broken in same key

Consumer1:

Connected: Tue Jul 14 09:36:39 JST 2020
[pulsar-client-io-1-1] WARN com.scurrilous.circe.checksum.Crc32cIntChecksum - Failed to load Circe JNI library. Falling back to Java based CRC32c provider
[pulsar-timer-4-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - [persistent://public/default/key-shared-test] [sub0] [820f0] Prefetched messages: 499 --- Consume throughput received: 0.02 msgs/s --- 0.00 Mbit/s --- Ack sent rate: 0.00 ack/s --- Failed messages: 0 --- batch messages: 0 ---Failed acks: 0
Received: my-message-0 PublishTime: 1594687006203 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-1 PublishTime: 1594687006243 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-2 PublishTime: 1594687006247 Date: Tue Jul 14 09:37:46 JST 2020
...
Received: my-message-498 PublishTime: 1594687008727 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-499 PublishTime: 1594687008731 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-500 PublishTime: 1594687038742 Date: Tue Jul 14 09:37:46 JST 2020
...
Received: my-message-990 PublishTime: 1594687040094 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-994 PublishTime: 1594687040103 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-995 PublishTime: 1594687040105 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-997 PublishTime: 1594687040113 Date: Tue Jul 14 09:37:46 JST 2020

Consumer2:

Connected: Tue Jul 14 09:37:03 JST 2020
[pulsar-client-io-1-1] WARN com.scurrilous.circe.checksum.Crc32cIntChecksum - Failed to load Circe JNI library. Falling back to Java based CRC32c provider
Received: my-message-501 MessageId: 4:1501:-1 PublishTime: 1594687038753 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-502 MessageId: 4:1502:-1 PublishTime: 1594687038755 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-503 MessageId: 4:1503:-1 PublishTime: 1594687038759 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-506 MessageId: 4:1506:-1 PublishTime: 1594687038785 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-508 MessageId: 4:1508:-1 PublishTime: 1594687038812 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-901 MessageId: 4:1901:-1 PublishTime: 1594687039871 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-509 MessageId: 4:1509:-1 PublishTime: 1594687038815 Date: Tue Jul 14 09:37:46 JST 2020
ordering was broken, key: 1 oldNum: 901 newNum: 511
Received: my-message-511 MessageId: 4:1511:-1 PublishTime: 1594687038826 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-512 MessageId: 4:1512:-1 PublishTime: 1594687038830 Date: Tue Jul 14 09:37:46 JST 2020
...

I think this issue is caused by #7105.
Here is an example.

  1. dispatch messages
  2. Consumer2 was stuck and totalMessagesSent=0
    • Consumer2 availablePermits was 0
  3. skip redeliver messages temporally
    • Consumer2 availablePermits was back to 1
  4. dispatch new messages
    • new message was dispatched to Consumer2
  5. back to redeliver messages
  6. dispatch messages
    • ordering was broken

Modifications

Stop to dispatch when skip message temporally since Key_Shared consumer stuck on delivery.

Verifying this change

  • Make sure that the change passes the CI checks.

It seems that this issue should check in some test cases.
However, I think this issue is corner case and not easier to check. If it is not, please tell me.

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): (no)
  • The public API: (no)
  • The schema: (no)
  • The default values of configurations: (no)
  • The wire protocol: (no)
  • The rest endpoints: (no)
  • The admin cli options: (no)
  • Anything that affects deployment: (no)

Documentation

  • Does this pull request introduce a new feature? (no)

@nkurihar nkurihar added area/broker type/bug The PR fixed a bug or issue reported a bug labels Jul 16, 2020
@sijie sijie requested a review from codelipenghui July 17, 2020 03:56
@sijie sijie added this to the 2.7.0 milestone Jul 17, 2020
@equanz
Copy link
Contributor Author

equanz commented Jul 18, 2020

/pulsarbot run-failure-checks

@equanz equanz changed the title [pulsar-broker] Stop to dispatch manually when skip message temporally since Key_Shared consumer stuck on delivery [pulsar-broker] Stop to dispatch when skip message temporally since Key_Shared consumer stuck on delivery Jul 18, 2020
@equanz equanz force-pushed the stop_to_dispatch_manually_key_shared branch from 8e32be2 to 536b6a5 Compare July 18, 2020 18:23
@codelipenghui codelipenghui requested a review from merlimat July 20, 2020 01:21
@feeblefakie
Copy link

I've checked this PR with the procedure in #7455 to see if there are any out-of-order messages.
But now consumers don't even consume anything.
The same thing happens for pulsar-perf so I don't think it's the issue of the procedure.

@equanz
Copy link
Contributor Author

equanz commented Jul 27, 2020

@feeblefakie Thank you for your check. But I couldn't reproduce your issue (my procedure below) now. Is this debug log occurred in your standalone? Or please tell me more detail.

test procedure

  1. build this code
  2. start standalone with --wipe-data and default config (so, use AUTO_SPLIT)
  3. create partitioned topic with 32 partition
  4. create subscription sub0, and sub1 as latest position to same topic
  5. produce messages by your code
  6. consume messages as sub0 by your code
    • out: 10000 records
  7. consume messages as sub1 by your code
    • out: 10000 records
  8. check output by your script

and no NG was found(with 1000 OK).

@feeblefakie
Copy link

feeblefakie commented Jul 28, 2020

@equanz
Thanks for checking.
I'm not sure why but the not-consuming issue can't be reproduced for some reason.
So now I can consume messages.

I also re-tested and still see NG.
Can you try several times ?
Sometimes it doesn't occur but it does most of the time.

Regarding the debug log, sorry I'm not sure how to enable debug log in Pulsar.
I updated conf/log4j2.yaml status to DEBUG from INFO, but it doesn't seem to be logging with DEBUG.
So I don't see the stuckConsumers message yet.

@equanz
Copy link
Contributor Author

equanz commented Aug 4, 2020

Sorry to be late.

I also re-tested and still see NG.
Can you try several times ?
Sometimes it doesn't occur but it does most of the time.

Try to check 7 times but I couldn't reproduce this issue.
I think #7455 possibly be another issue...

Regarding the debug log, sorry I'm not sure how to enable debug log in Pulsar.

Here is an example. I thought try to check not-consuming issue by this log.

diff --git a/conf/log4j2.yaml b/conf/log4j2.yaml
index 34826740ca..b4dc4dabbb 100644
--- a/conf/log4j2.yaml
+++ b/conf/log4j2.yaml
@@ -151,6 +151,11 @@ Configuration:
         AppenderRef:
           - ref: Console

+      - name: org.apache.pulsar.broker.service.persistent.PersistentStickyKeyDispatcherMultipleConsumers
+        level: debug
+        AppenderRef:
+          - ref: Console
+
     # Logger to inject filter script
 #     - name: org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl
 #       level: debug

I'm not sure why but the not-consuming issue can't be reproduced for some reason.

I understand. If not-consuming issue is caused by this issue, I must fix.

@feeblefakie
Copy link

@equanz Hmm, that is pretty weird.
It happens to me 9 out of 10.
The cause of #7455 seems like the number of consumers are changing (increasing) during consuming,
but is it the case for your testing ?
(Sorry my procedures to reproduce is a bit not correct.
I added step 3 to register subscriptions beforehand.
https://github.com/feeblefakie/misc/blob/master/pulsar/HOW-TO-REPRODUCE.md )

If I start consumers before producers, the issue won't happen since the number of consumers is constant.

@equanz
Copy link
Contributor Author

equanz commented Aug 4, 2020

The cause of #7455 seems like the number of consumers are changing (increasing) during consuming,
but is it the case for your testing ?

I think my issue is occurred by some consumers with different throughput in same subscription.

@equanz equanz force-pushed the stop_to_dispatch_manually_key_shared branch from 536b6a5 to 813f08c Compare August 5, 2020 23:54
@equanz
Copy link
Contributor Author

equanz commented Aug 6, 2020

/pulsarbot run-failure-checks

2 similar comments
@equanz
Copy link
Contributor Author

equanz commented Aug 6, 2020

/pulsarbot run-failure-checks

@equanz
Copy link
Contributor Author

equanz commented Aug 7, 2020

/pulsarbot run-failure-checks

@sijie
Copy link
Member

sijie commented Aug 11, 2020

@codelipenghui can you take a look?

@codelipenghui
Copy link
Contributor

@equanz Is it possible to add a unit test? so that we can better understand the problem and make sure it is fixed.

@equanz
Copy link
Contributor Author

equanz commented Aug 13, 2020

@codelipenghui I understand. As stated above, I think this issue is corner case and probably not easier to check.
But I'll try to add some tests.

@equanz
Copy link
Contributor Author

equanz commented Aug 17, 2020

@codelipenghui Added unit test to broker side. So, please take another look.

@codelipenghui
Copy link
Contributor

@equanz Thanks a lot, I will take a look soon.

@codelipenghui codelipenghui merged commit c7ac08b into apache:master Sep 2, 2020
@equanz equanz deleted the stop_to_dispatch_manually_key_shared branch September 2, 2020 23:08
lbenc135 pushed a commit to lbenc135/pulsar that referenced this pull request Sep 5, 2020
…ey_Shared consumer stuck on delivery (apache#7553)

### Motivation
In some case of Key_Shared consumer, messages ordering was broken.
Here is how to reproduce(I think it is one of case to reproduce this issue).

1. Connect Consumer1 to Key_Shared subscription `sub` and stop to receive
   - receiverQueueSize: 500
2. Connect Producer and publish 500 messages with key `(i % 10)`
3. Connect Consumer2 to same subscription and start to receive
   - receiverQueueSize: 1
   - since apache#7106 , Consumer2 can't receive (expected)
4. Producer publish more 500 messages with same key generation algorithm
5. After that, Consumer1 start to receive
6. Check Consumer2 message ordering
   - sometimes message ordering was broken in same key

Consumer1:
```
Connected: Tue Jul 14 09:36:39 JST 2020
[pulsar-client-io-1-1] WARN com.scurrilous.circe.checksum.Crc32cIntChecksum - Failed to load Circe JNI library. Falling back to Java based CRC32c provider
[pulsar-timer-4-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - [persistent://public/default/key-shared-test] [sub0] [820f0] Prefetched messages: 499 --- Consume throughput received: 0.02 msgs/s --- 0.00 Mbit/s --- Ack sent rate: 0.00 ack/s --- Failed messages: 0 --- batch messages: 0 ---Failed acks: 0
Received: my-message-0 PublishTime: 1594687006203 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-1 PublishTime: 1594687006243 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-2 PublishTime: 1594687006247 Date: Tue Jul 14 09:37:46 JST 2020
...
Received: my-message-498 PublishTime: 1594687008727 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-499 PublishTime: 1594687008731 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-500 PublishTime: 1594687038742 Date: Tue Jul 14 09:37:46 JST 2020
...
Received: my-message-990 PublishTime: 1594687040094 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-994 PublishTime: 1594687040103 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-995 PublishTime: 1594687040105 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-997 PublishTime: 1594687040113 Date: Tue Jul 14 09:37:46 JST 2020
```

Consumer2:
```
Connected: Tue Jul 14 09:37:03 JST 2020
[pulsar-client-io-1-1] WARN com.scurrilous.circe.checksum.Crc32cIntChecksum - Failed to load Circe JNI library. Falling back to Java based CRC32c provider
Received: my-message-501 MessageId: 4:1501:-1 PublishTime: 1594687038753 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-502 MessageId: 4:1502:-1 PublishTime: 1594687038755 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-503 MessageId: 4:1503:-1 PublishTime: 1594687038759 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-506 MessageId: 4:1506:-1 PublishTime: 1594687038785 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-508 MessageId: 4:1508:-1 PublishTime: 1594687038812 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-901 MessageId: 4:1901:-1 PublishTime: 1594687039871 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-509 MessageId: 4:1509:-1 PublishTime: 1594687038815 Date: Tue Jul 14 09:37:46 JST 2020
ordering was broken, key: 1 oldNum: 901 newNum: 511
Received: my-message-511 MessageId: 4:1511:-1 PublishTime: 1594687038826 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-512 MessageId: 4:1512:-1 PublishTime: 1594687038830 Date: Tue Jul 14 09:37:46 JST 2020
...
```

I think this issue is caused by apache#7105.
Here is an example.
1. dispatch messages
2. Consumer2 was stuck and `totalMessagesSent=0`
   - Consumer2 availablePermits was 0
3. skip redeliver messages temporally
   - Consumer2 availablePermits was back to 1
4. dispatch new messages
   - new message was dispatched to Consumer2
5. back to redeliver messages
4. dispatch messages
   - ordering was broken

### Modifications
Stop to dispatch when skip message temporally since Key_Shared consumer stuck on delivery.
lbenc135 pushed a commit to lbenc135/pulsar that referenced this pull request Sep 5, 2020
…ey_Shared consumer stuck on delivery (apache#7553)

### Motivation
In some case of Key_Shared consumer, messages ordering was broken.
Here is how to reproduce(I think it is one of case to reproduce this issue).

1. Connect Consumer1 to Key_Shared subscription `sub` and stop to receive
   - receiverQueueSize: 500
2. Connect Producer and publish 500 messages with key `(i % 10)`
3. Connect Consumer2 to same subscription and start to receive
   - receiverQueueSize: 1
   - since apache#7106 , Consumer2 can't receive (expected)
4. Producer publish more 500 messages with same key generation algorithm
5. After that, Consumer1 start to receive
6. Check Consumer2 message ordering
   - sometimes message ordering was broken in same key

Consumer1:
```
Connected: Tue Jul 14 09:36:39 JST 2020
[pulsar-client-io-1-1] WARN com.scurrilous.circe.checksum.Crc32cIntChecksum - Failed to load Circe JNI library. Falling back to Java based CRC32c provider
[pulsar-timer-4-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - [persistent://public/default/key-shared-test] [sub0] [820f0] Prefetched messages: 499 --- Consume throughput received: 0.02 msgs/s --- 0.00 Mbit/s --- Ack sent rate: 0.00 ack/s --- Failed messages: 0 --- batch messages: 0 ---Failed acks: 0
Received: my-message-0 PublishTime: 1594687006203 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-1 PublishTime: 1594687006243 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-2 PublishTime: 1594687006247 Date: Tue Jul 14 09:37:46 JST 2020
...
Received: my-message-498 PublishTime: 1594687008727 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-499 PublishTime: 1594687008731 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-500 PublishTime: 1594687038742 Date: Tue Jul 14 09:37:46 JST 2020
...
Received: my-message-990 PublishTime: 1594687040094 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-994 PublishTime: 1594687040103 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-995 PublishTime: 1594687040105 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-997 PublishTime: 1594687040113 Date: Tue Jul 14 09:37:46 JST 2020
```

Consumer2:
```
Connected: Tue Jul 14 09:37:03 JST 2020
[pulsar-client-io-1-1] WARN com.scurrilous.circe.checksum.Crc32cIntChecksum - Failed to load Circe JNI library. Falling back to Java based CRC32c provider
Received: my-message-501 MessageId: 4:1501:-1 PublishTime: 1594687038753 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-502 MessageId: 4:1502:-1 PublishTime: 1594687038755 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-503 MessageId: 4:1503:-1 PublishTime: 1594687038759 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-506 MessageId: 4:1506:-1 PublishTime: 1594687038785 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-508 MessageId: 4:1508:-1 PublishTime: 1594687038812 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-901 MessageId: 4:1901:-1 PublishTime: 1594687039871 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-509 MessageId: 4:1509:-1 PublishTime: 1594687038815 Date: Tue Jul 14 09:37:46 JST 2020
ordering was broken, key: 1 oldNum: 901 newNum: 511
Received: my-message-511 MessageId: 4:1511:-1 PublishTime: 1594687038826 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-512 MessageId: 4:1512:-1 PublishTime: 1594687038830 Date: Tue Jul 14 09:37:46 JST 2020
...
```

I think this issue is caused by apache#7105.
Here is an example.
1. dispatch messages
2. Consumer2 was stuck and `totalMessagesSent=0`
   - Consumer2 availablePermits was 0
3. skip redeliver messages temporally
   - Consumer2 availablePermits was back to 1
4. dispatch new messages
   - new message was dispatched to Consumer2
5. back to redeliver messages
4. dispatch messages
   - ordering was broken

### Modifications
Stop to dispatch when skip message temporally since Key_Shared consumer stuck on delivery.
lbenc135 pushed a commit to lbenc135/pulsar that referenced this pull request Sep 5, 2020
…ey_Shared consumer stuck on delivery (apache#7553)

### Motivation
In some case of Key_Shared consumer, messages ordering was broken.
Here is how to reproduce(I think it is one of case to reproduce this issue).

1. Connect Consumer1 to Key_Shared subscription `sub` and stop to receive
   - receiverQueueSize: 500
2. Connect Producer and publish 500 messages with key `(i % 10)`
3. Connect Consumer2 to same subscription and start to receive
   - receiverQueueSize: 1
   - since apache#7106 , Consumer2 can't receive (expected)
4. Producer publish more 500 messages with same key generation algorithm
5. After that, Consumer1 start to receive
6. Check Consumer2 message ordering
   - sometimes message ordering was broken in same key

Consumer1:
```
Connected: Tue Jul 14 09:36:39 JST 2020
[pulsar-client-io-1-1] WARN com.scurrilous.circe.checksum.Crc32cIntChecksum - Failed to load Circe JNI library. Falling back to Java based CRC32c provider
[pulsar-timer-4-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - [persistent://public/default/key-shared-test] [sub0] [820f0] Prefetched messages: 499 --- Consume throughput received: 0.02 msgs/s --- 0.00 Mbit/s --- Ack sent rate: 0.00 ack/s --- Failed messages: 0 --- batch messages: 0 ---Failed acks: 0
Received: my-message-0 PublishTime: 1594687006203 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-1 PublishTime: 1594687006243 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-2 PublishTime: 1594687006247 Date: Tue Jul 14 09:37:46 JST 2020
...
Received: my-message-498 PublishTime: 1594687008727 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-499 PublishTime: 1594687008731 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-500 PublishTime: 1594687038742 Date: Tue Jul 14 09:37:46 JST 2020
...
Received: my-message-990 PublishTime: 1594687040094 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-994 PublishTime: 1594687040103 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-995 PublishTime: 1594687040105 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-997 PublishTime: 1594687040113 Date: Tue Jul 14 09:37:46 JST 2020
```

Consumer2:
```
Connected: Tue Jul 14 09:37:03 JST 2020
[pulsar-client-io-1-1] WARN com.scurrilous.circe.checksum.Crc32cIntChecksum - Failed to load Circe JNI library. Falling back to Java based CRC32c provider
Received: my-message-501 MessageId: 4:1501:-1 PublishTime: 1594687038753 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-502 MessageId: 4:1502:-1 PublishTime: 1594687038755 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-503 MessageId: 4:1503:-1 PublishTime: 1594687038759 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-506 MessageId: 4:1506:-1 PublishTime: 1594687038785 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-508 MessageId: 4:1508:-1 PublishTime: 1594687038812 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-901 MessageId: 4:1901:-1 PublishTime: 1594687039871 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-509 MessageId: 4:1509:-1 PublishTime: 1594687038815 Date: Tue Jul 14 09:37:46 JST 2020
ordering was broken, key: 1 oldNum: 901 newNum: 511
Received: my-message-511 MessageId: 4:1511:-1 PublishTime: 1594687038826 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-512 MessageId: 4:1512:-1 PublishTime: 1594687038830 Date: Tue Jul 14 09:37:46 JST 2020
...
```

I think this issue is caused by apache#7105.
Here is an example.
1. dispatch messages
2. Consumer2 was stuck and `totalMessagesSent=0`
   - Consumer2 availablePermits was 0
3. skip redeliver messages temporally
   - Consumer2 availablePermits was back to 1
4. dispatch new messages
   - new message was dispatched to Consumer2
5. back to redeliver messages
4. dispatch messages
   - ordering was broken

### Modifications
Stop to dispatch when skip message temporally since Key_Shared consumer stuck on delivery.
wolfstudy pushed a commit that referenced this pull request Oct 30, 2020
…ey_Shared consumer stuck on delivery (#7553)

### Motivation
In some case of Key_Shared consumer, messages ordering was broken.
Here is how to reproduce(I think it is one of case to reproduce this issue).

1. Connect Consumer1 to Key_Shared subscription `sub` and stop to receive
   - receiverQueueSize: 500
2. Connect Producer and publish 500 messages with key `(i % 10)`
3. Connect Consumer2 to same subscription and start to receive
   - receiverQueueSize: 1
   - since #7106 , Consumer2 can't receive (expected)
4. Producer publish more 500 messages with same key generation algorithm
5. After that, Consumer1 start to receive
6. Check Consumer2 message ordering
   - sometimes message ordering was broken in same key

Consumer1:
```
Connected: Tue Jul 14 09:36:39 JST 2020
[pulsar-client-io-1-1] WARN com.scurrilous.circe.checksum.Crc32cIntChecksum - Failed to load Circe JNI library. Falling back to Java based CRC32c provider
[pulsar-timer-4-1] INFO org.apache.pulsar.client.impl.ConsumerStatsRecorderImpl - [persistent://public/default/key-shared-test] [sub0] [820f0] Prefetched messages: 499 --- Consume throughput received: 0.02 msgs/s --- 0.00 Mbit/s --- Ack sent rate: 0.00 ack/s --- Failed messages: 0 --- batch messages: 0 ---Failed acks: 0
Received: my-message-0 PublishTime: 1594687006203 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-1 PublishTime: 1594687006243 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-2 PublishTime: 1594687006247 Date: Tue Jul 14 09:37:46 JST 2020
...
Received: my-message-498 PublishTime: 1594687008727 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-499 PublishTime: 1594687008731 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-500 PublishTime: 1594687038742 Date: Tue Jul 14 09:37:46 JST 2020
...
Received: my-message-990 PublishTime: 1594687040094 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-994 PublishTime: 1594687040103 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-995 PublishTime: 1594687040105 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-997 PublishTime: 1594687040113 Date: Tue Jul 14 09:37:46 JST 2020
```

Consumer2:
```
Connected: Tue Jul 14 09:37:03 JST 2020
[pulsar-client-io-1-1] WARN com.scurrilous.circe.checksum.Crc32cIntChecksum - Failed to load Circe JNI library. Falling back to Java based CRC32c provider
Received: my-message-501 MessageId: 4:1501:-1 PublishTime: 1594687038753 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-502 MessageId: 4:1502:-1 PublishTime: 1594687038755 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-503 MessageId: 4:1503:-1 PublishTime: 1594687038759 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-506 MessageId: 4:1506:-1 PublishTime: 1594687038785 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-508 MessageId: 4:1508:-1 PublishTime: 1594687038812 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-901 MessageId: 4:1901:-1 PublishTime: 1594687039871 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-509 MessageId: 4:1509:-1 PublishTime: 1594687038815 Date: Tue Jul 14 09:37:46 JST 2020
ordering was broken, key: 1 oldNum: 901 newNum: 511
Received: my-message-511 MessageId: 4:1511:-1 PublishTime: 1594687038826 Date: Tue Jul 14 09:37:46 JST 2020
Received: my-message-512 MessageId: 4:1512:-1 PublishTime: 1594687038830 Date: Tue Jul 14 09:37:46 JST 2020
...
```

I think this issue is caused by #7105.
Here is an example.
1. dispatch messages
2. Consumer2 was stuck and `totalMessagesSent=0`
   - Consumer2 availablePermits was 0
3. skip redeliver messages temporally
   - Consumer2 availablePermits was back to 1
4. dispatch new messages
   - new message was dispatched to Consumer2
5. back to redeliver messages
4. dispatch messages
   - ordering was broken

### Modifications
Stop to dispatch when skip message temporally since Key_Shared consumer stuck on delivery.

(cherry picked from commit c7ac08b)
poorbarcode added a commit that referenced this pull request May 19, 2023
…essage skip to avoid unnecessary consumption stuck (#20335)

### Motivation
- #7105 provide a mechanism to avoid a stuck consumer affecting the consumption of other consumers: 
  - if all consumers can not accept more messages, stop delivering messages to the client.
  - if one consumer can not accept more messages, just read new messages and deliver them to other consumers.
- #7553 provide a mechanism to fix the issue of lost order of consumption: If the consumer cannot accept any more messages, skip the consumer for the next round of message delivery because there may be messages with the same key in the replay queue.
- #10762 provide a mechanism to fix the issue of lost order of consumption: If there have any messages with the same key in the replay queue, do not deliver the new messages to this consumer.

#10762 and #7553 do the same thing and #10762 is better than #7553 , so #7553 is unnecessary. 

### Modifications
remove the mechanism provided by #7553 to avoid unnecessary consumption stuck.
poorbarcode added a commit that referenced this pull request May 19, 2023
…essage skip to avoid unnecessary consumption stuck (#20335)

- #7105 provide a mechanism to avoid a stuck consumer affecting the consumption of other consumers:
  - if all consumers can not accept more messages, stop delivering messages to the client.
  - if one consumer can not accept more messages, just read new messages and deliver them to other consumers.
- #7553 provide a mechanism to fix the issue of lost order of consumption: If the consumer cannot accept any more messages, skip the consumer for the next round of message delivery because there may be messages with the same key in the replay queue.
- #10762 provide a mechanism to fix the issue of lost order of consumption: If there have any messages with the same key in the replay queue, do not deliver the new messages to this consumer.

#10762 and #7553 do the same thing and #10762 is better than #7553 , so #7553 is unnecessary.

remove the mechanism provided by #7553 to avoid unnecessary consumption stuck.

(cherry picked from commit 1e664b7)
Technoboy- pushed a commit that referenced this pull request May 24, 2023
…essage skip to avoid unnecessary consumption stuck (#20335)

### Motivation
- #7105 provide a mechanism to avoid a stuck consumer affecting the consumption of other consumers: 
  - if all consumers can not accept more messages, stop delivering messages to the client.
  - if one consumer can not accept more messages, just read new messages and deliver them to other consumers.
- #7553 provide a mechanism to fix the issue of lost order of consumption: If the consumer cannot accept any more messages, skip the consumer for the next round of message delivery because there may be messages with the same key in the replay queue.
- #10762 provide a mechanism to fix the issue of lost order of consumption: If there have any messages with the same key in the replay queue, do not deliver the new messages to this consumer.

#10762 and #7553 do the same thing and #10762 is better than #7553 , so #7553 is unnecessary. 

### Modifications
remove the mechanism provided by #7553 to avoid unnecessary consumption stuck.
lhotari pushed a commit to datastax/pulsar that referenced this pull request May 29, 2023
…essage skip to avoid unnecessary consumption stuck (apache#20335)

- apache#7105 provide a mechanism to avoid a stuck consumer affecting the consumption of other consumers:
  - if all consumers can not accept more messages, stop delivering messages to the client.
  - if one consumer can not accept more messages, just read new messages and deliver them to other consumers.
- apache#7553 provide a mechanism to fix the issue of lost order of consumption: If the consumer cannot accept any more messages, skip the consumer for the next round of message delivery because there may be messages with the same key in the replay queue.
- apache#10762 provide a mechanism to fix the issue of lost order of consumption: If there have any messages with the same key in the replay queue, do not deliver the new messages to this consumer.

apache#10762 and apache#7553 do the same thing and apache#10762 is better than apache#7553 , so apache#7553 is unnecessary.

remove the mechanism provided by apache#7553 to avoid unnecessary consumption stuck.

(cherry picked from commit 1e664b7)
(cherry picked from commit c973603)
poorbarcode added a commit that referenced this pull request May 30, 2023
…essage skip to avoid unnecessary consumption stuck (#20335)

### Motivation
- #7105 provide a mechanism to avoid a stuck consumer affecting the consumption of other consumers:
  - if all consumers can not accept more messages, stop delivering messages to the client.
  - if one consumer can not accept more messages, just read new messages and deliver them to other consumers.
- #7553 provide a mechanism to fix the issue of lost order of consumption: If the consumer cannot accept any more messages, skip the consumer for the next round of message delivery because there may be messages with the same key in the replay queue.
- #10762 provide a mechanism to fix the issue of lost order of consumption: If there have any messages with the same key in the replay queue, do not deliver the new messages to this consumer.

#10762 and #7553 do the same thing and #10762 is better than #7553 , so #7553 is unnecessary.

### Modifications
remove the mechanism provided by #7553 to avoid unnecessary consumption stuck.

(cherry picked from commit 1e664b7)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/broker release/2.6.2 type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants