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

   <!--
   ### Contribution Checklist
   
     - PR title format should be *[type][component] summary*. For details, see 
*[Guideline - Pulsar PR Naming 
Convention](https://pulsar.apache.org/contribute/develop-semantic-title/)*.
     - Fill out the template below to describe the changes contributed by the 
pull request. That will give reviewers the context they need to do the review.
   
     - Each pull request should address only one issue, not mix up code from 
multiple issues.
   
     - Each commit in the pull request has a meaningful commit message
     - Once all items of the checklist are addressed, remove the above text and 
this checklist, leaving only the filled out template below.
   -->
   
   ### Motivation
   
   The Key_Shared subscription type has the following issues.
   
   1. Key_Shared subscription has out-of-order cases because of the race 
condition of [the recently joined consumers 
feature](https://github.com/apache/pulsar/blob/35e9897742b7db4bd29349940075a819b2ad6999/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java#L400-L408).
      Consider the following flow.
   
      1. Assume that the current read position is `1:6` and the recently joined 
consumers is empty.
      2. Called 
[OpReadEntry#internalReadEntriesComplete](https://github.com/apache/pulsar/blob/35e9897742b7db4bd29349940075a819b2ad6999/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java#L92-L95)
 from thread-1.
         Then, the current read position is updated to `1:11` (Messages from 
`1:6` to `1:10` have yet to be dispatched to consumers).
      3. Called 
[PersistentStickyKeyDispatcherMultipleConsumers#addConsumer](https://github.com/apache/pulsar/blob/35e9897742b7db4bd29349940075a819b2ad6999/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java#L130-L131)
 from thread-2.
         Then, the new consumer is stored to recently joined consumers with 
read position `1:11`.
      4. Called 
[PersistentDispatcherMultipleConsumers#trySendMessagesToConsumers](https://github.com/apache/pulsar/blob/35e9897742b7db4bd29349940075a819b2ad6999/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java#L174)
 from thread-5.
         Then, messages from `1:6` to `1:10` are dispatched to consumers. From 
the recently joined consumers feature, the new consumer can receive messages 
from `1:6` to `1:10`. **However, it is not expected.**
         For example, if existing consumers have some unacked messages and 
disconnect, it causes out of order in some cases.
   
   2. Key_Shared subscription has a redundant process.
      [The stuckConsumers 
feature](https://github.com/apache/pulsar/blob/35e9897742b7db4bd29349940075a819b2ad6999/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java#L369-L375)
 was introduced from https://github.com/apache/pulsar/pull/7553 .
      However, it can't fix the issue entirely because it doesn't consider the 
range changes.
      After this commit, https://github.com/apache/pulsar/pull/10762 was 
introduced. It fixes the issue.
   
   ### Modifications
   
   1. Store last sent position instead of read position to recently joined 
consumers.
      Updating read position, then dispatching messages, and adding new 
consumer are not exclusive.
      I used the last send position to get a position without any new exclusion 
features.
   
   2. Keep the last sent position per key.
      When I introduced the last sent position, I noticed a new concern. 
Consider the following flow.
   
      1. Assume that the 
[entries](https://github.com/apache/pulsar/blob/35e9897742b7db4bd29349940075a819b2ad6999/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java#L174)
 has the following messages,
         - `msg-1`, key: `key-a`, position: `1:1`
         - `msg-2`, key: `key-a`, position: `1:2`
         - `msg-3`, key: `key-a`, position: `1:3`
         - `msg-4`, key: `key-b`, position: `1:4`
         - `msg-5`, key: `key-b`, position: `1:5`
         - `msg-6`, key: `key-b`, position: `1:6`
         the dispatcher has two consumers (`c1` `messagesForC` is 1, `c2` 
`messageForC` is 1000), and the selector will return `c1` if `key-a` and `c2` 
if `key-b`.
      2. Send `msg-1` to `c1` and `msg-4` - `msg-6` to `c2`.
         - So, the current last sent position is `1:6`.
         - `c1` never acknowledge `msg-1`.
      3. Add new consumer `c3`, and the selector will return `c3` if `key-a`.
      4. Send `msg-2` - `msg-3` to `c3` because `1:2` and `1:3` are less than 
the last sent position, `1:6`.
      5. Disconnect `c1`.
      6. Send `msg-1` to `c3`.
         Now, `c3` receives messages without expected order about `key-a`.
   
      To avoid this issue, I introduce the last sent positions feature.
   
   3. Remove redundant features due to the addition of the last sent positions 
feature.
   
      1. Remove this condition because the last sent positions feature 
restricts part of messages.
         New consumers can receive any new messages with the message key that 
is not in the last sent position.
         
https://github.com/apache/pulsar/blob/35e9897742b7db4bd29349940075a819b2ad6999/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java#L323-L336
   
      2. Remove this behavior because the last sent positions has positions per 
key.
         If some key is stuck at a certain point in time, from that point, new 
consumers store the same information about the key to the 
`recentlyJoinedConsumers`.
         
https://github.com/apache/pulsar/blob/35e9897742b7db4bd29349940075a819b2ad6999/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java#L365-L399
   
      3. Remove this behavior because this calculation is moved to 
LastSentPositions#compareToLastSentPosition.
         
https://github.com/apache/pulsar/blob/35e9897742b7db4bd29349940075a819b2ad6999/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java#L400-L408
   
   4. Remove the stuckConsumers feature.
   5. Reconstruct some consumer stats fields related to Key_Shared.
      **This is a breaking change.** However, if this PR is merged, the 
existing field which is removed in this PR is no longer needed.
   
   ### Verifying this change
   
   - [ ] Make sure that the change passes the CI checks.
   
   This change added tests and can be verified as follows:
   - Added unit test to ensure the behavior of last sent positions feature 
works properly
   
   ### Does this pull request potentially affect one of the following parts:
   
   <!-- DO NOT REMOVE THIS SECTION. CHECK THE PROPER BOX ONLY. -->
   
   *If the box was checked, please highlight the changes*
   
   - [ ] Dependencies (add or upgrade a dependency)
   - [ ] The public API
   - [ ] The schema
   - [ ] The default values of configurations
   - [ ] The threading model
   - [ ] The binary protocol
   - [ ] The REST endpoints
   - [ ] The admin CLI options
   - [x] The metrics
   - [ ] Anything that affects deployment
   
   ### Documentation
   
   <!-- DO NOT REMOVE THIS SECTION. CHECK THE PROPER BOX ONLY. -->
   
   - [ ] `doc` <!-- Your PR contains doc changes. -->
   - [x] `doc-required` <!-- Your PR changes impact docs and you will update 
later -->
   - [ ] `doc-not-needed` <!-- Your PR changes do not impact docs -->
   - [ ] `doc-complete` <!-- Docs have been already added -->
   
   ### Matching PR in forked repository
   
   PR in forked repository: 
https://github.com/equanz/pulsar/tree/change_definition_of_recently_joined_consumers_position
   
   <!--
   After opening this PR, the build in apache/pulsar will fail and instructions 
will
   be provided for opening a PR in the PR author's forked repository.
   apache/pulsar pull requests should be first tested in your own fork since the
   apache/pulsar CI based on GitHub Actions has constrained resources and quota.
   GitHub Actions provides separate quota for pull requests that are executed in
   a forked repository.
   The tests will be run in the forked repository until all PR review comments 
have
   been handled, the tests pass and the PR is approved by a reviewer.
   -->
   


-- 
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-unsubscr...@pulsar.apache.org

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

Reply via email to