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/04/07 11:46:36 UTC

[GitHub] [pulsar] lhotari opened a new pull request, #15067: [ML

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

   <!--
   ### Contribution Checklist
     
     - PR title format should be *[type][component] summary*. For details, see *[Guideline - Pulsar PR Naming Convention](https://docs.google.com/document/d/1d8Pw6ZbWk-_pCKdOmdvx9rnhPiyuxwq60_TrD68d7BA/edit#heading=h.trs9rsex3xom)*. 
   
     - 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.
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   *(If this PR fixes a github issue, please add `Fixes #<xyz>`.)*
   
   Fixes #<xyz>
   
   *(or if this PR is one task of a github issue, please add `Master Issue: #<xyz>` to link to the master issue.)*
   
   Master Issue: #<xyz>
   
   ### Motivation
   
   
   *Explain here the context, and why you're making that change. What is the problem you're trying to solve.*
   
   ### Modifications
   
   *Describe the modifications you've done.*
   
   ### Verifying this change
   
   - [ ] Make sure that the change passes the CI checks.
   
   *(Please pick either of the following options)*
   
   This change is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This change is already covered by existing tests, such as *(please describe tests)*.
   
   *(or)*
   
   This change added tests and can be verified as follows:
   
   *(example:)*
     - *Added integration tests for end-to-end deployment with large payloads (10MB)*
     - *Extended integration test for recovery after broker failure*
   
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
     - Dependencies (does it add or upgrade a dependency): (yes / no)
     - The public API: (yes / no)
     - The schema: (yes / no / don't know)
     - The default values of configurations: (yes / no)
     - The wire protocol: (yes / no)
     - The rest endpoints: (yes / no)
     - The admin cli options: (yes / no)
     - Anything that affects deployment: (yes / no / don't know)
   
   ### Documentation
   
   Check the box below or label this PR directly.
   
   Need to update docs? 
   
   - [ ] `doc-required` 
   (Your PR needs to update docs and you will update later)
     
   - [ ] `no-need-doc` 
   (Please explain why)
     
   - [ ] `doc` 
   (Your PR contains doc changes)
   
   - [ ] `doc-added`
   (Docs have been already added)


-- 
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] eolivelli commented on pull request #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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

   This patch fixes this problem
   https://github.com/apache/pulsar/issues/15151


-- 
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 #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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

   @lhotari Sorry for the late response.
   
   Looks like this PR will not fix the persistentMarkDeletePosition consistency issue?
   
   Assume 2 threads, thread 0 and thread 1, thread 0 reach [here](https://github.com/apache/pulsar/pull/15067/files#diff-fad355f91bd15cc041161f9a46fce62b7fee87fbfb8f0ff8a8b724a1bd1f29eeR1883)  and thread 1 reach [here](https://github.com/apache/pulsar/pull/15067/files#diff-fad355f91bd15cc041161f9a46fce62b7fee87fbfb8f0ff8a8b724a1bd1f29eeR1852), the persistentMarkDeletePosition might apply the value from thread 0 or thread 1, it's uncertain, I think we should provide consistent behavior here.


-- 
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] congbobo184 commented on a diff in pull request #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java:
##########
@@ -1814,6 +1876,9 @@ public void operationComplete() {
                             mdEntry.newPosition);
                 }
 
+                INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER.compareAndSet(ManagedCursorImpl.this,
+                        mdEntry.newPosition, null);
+

Review Comment:
   we should make sure only one `persistPositionToLedger ` op in the same time. we should lock update persistentMarkDeletePosition and update INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER like : 
   ```
           try {
               lock.writeLock().lock();
               if((persistentMarkDeletePosition == null ||
                       mdEntry.newPosition.compareTo(persistentMarkDeletePosition) > 0) && PENDING_MARK_DELETED_SUBMITTED_COUNT_UPDATER.compareAndSet(this, null, mdEntry.newPosition)) {
                   persistPositionToLedger(cursorLedger, mdEntry, new VoidCallback() {
                       public void operationComplete() {
                           try {
                               lock.writeLock().lock();
                               INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER.compareAndSet(ManagedCursorImpl.this,
                                       mdEntry.newPosition, null);
                               persistentMarkDeletePosition = mdEntry.newPosition;
                           } finally {
                               lock.writeLock().unlock();
                           }
                       }
                   }
               }
           } finally {
               lock.writeLock().unlock();
           }
   ```
   of cause you only use a flag not use INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER is also can implement



-- 
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] Technoboy- commented on pull request #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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

   > > , the persistentMarkDeletePosition might apply the value from thread 0 or thread 1, it's uncertain, I think we should provide consistent behavior here.
   > > And `inProgressLatest` is a local variable, `mdEntry` will not share between threads, I think they are not related to race conditions.
   > 
   > @codelipenghui That's true. However, I don't think that the problem has become worse than before. It would be good to address the problems with a single threaded approach with a actor like model where there would be no concurrency.
   
   I don't agree with this. This will add the complexity of the logic.  We should find a way to fix this completely, not merge with other ideas


-- 
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] github-actions[bot] commented on pull request #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #15067:
URL: https://github.com/apache/pulsar/pull/15067#issuecomment-1091649088

   @lhotari:Thanks for providing doc info!


-- 
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 pull request #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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

   > **This is wrong since the meaning of the field is not what one would expect it to be.**
   
   @lhotari - are you able to describe the meaning of the field more?


-- 
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 #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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

   > > This patch fixes this problem #15151
   > 
   > Could you explain why this patch has fixed this issue ?
   
   This line was missing:
   https://github.com/apache/pulsar/blob/8c534dbcb8daab8a249501604a9312ff4b5a1166/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java#L1616
   
   Since #15031 added a solution to prevent race conditions in updating the `lastMarkDeleteEntry` field, the solution prevented the value to move backwards. The `lastMarkDeleteEntry` field should be resetted in the `initializeCursorPosition` method for this logic to work correctly when using earliest initialposition:
   https://github.com/apache/pulsar/blob/b083e9a72227a3360d1ec33b5f239d82f0804e65/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java#L935-L942
   
   What ended up happening that #15031 prevented updating the `lastMarkDeleteEntry` field and when the scheduled flush call happened, that updated the markDelete position to the latest position.
   https://github.com/apache/pulsar/blob/8c534dbcb8daab8a249501604a9312ff4b5a1166/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java#L3123-L3129
   
   That caused the behavior that the cursor would move the readPosition to the last entry when the flush call happened and the resulted in the entries (messages) to be skipped.
   
   I'm planning to add a test case to reproduce this and I'll use it also for refactoring the solution to fix the remaining ordering issue  / race condition that remains.
   
   


-- 
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 #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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

   > , the persistentMarkDeletePosition might apply the value from thread 0 or thread 1, it's uncertain, I think we should provide consistent behavior here.
   > And `inProgressLatest` is a local variable, `mdEntry` will not share between threads, I think they are not related to race conditions.
   
   @codelipenghui That's true. However, I don't think that the problem has become worse than before.
   It would be good to address the problems with a single threaded approach with a actor like model where there would be no concurrency.


-- 
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] github-actions[bot] commented on pull request #15067: [ML

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #15067:
URL: https://github.com/apache/pulsar/pull/15067#issuecomment-1091637520

   @lhotari:Thanks for your contribution. For this PR, do we need to update docs?
   (The [PR template contains info about doc](https://github.com/apache/pulsar/blob/master/.github/PULL_REQUEST_TEMPLATE.md#documentation), which helps others know more about the changes. Can you provide doc-related info in this and future PR descriptions? Thanks)


-- 
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] Technoboy- commented on pull request #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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

   > This patch fixes this problem #15151
   
   Could you explain why this patch has fixed this issue ?


-- 
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 merged pull request #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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


-- 
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 a diff in pull request #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java:
##########
@@ -1793,6 +1830,34 @@ protected void internalAsyncMarkDelete(final PositionImpl newPosition, Map<Strin
     }
 
     void internalMarkDelete(final MarkDeleteEntry mdEntry) {
+        if (persistentMarkDeletePosition != null
+                && mdEntry.newPosition.compareTo(persistentMarkDeletePosition) < 0) {
+            if (log.isInfoEnabled()) {
+                log.info("Skipping updating mark delete position to {}. The persisted mark delete position {} "
+                        + "is latest.", mdEntry.newPosition, persistentMarkDeletePosition);
+            }
+            mdEntry.triggerComplete();
+            return;
+        }
+
+        PositionImpl inProgressLatest = INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER.updateAndGet(this, current -> {
+            if (current != null && current.compareTo(mdEntry.newPosition) > 0) {
+                return current;
+            } else {
+                return mdEntry.newPosition;
+            }
+        });
+
+        // if there's a newer or equal mark delete update in progress, skip it.
+        if (inProgressLatest != mdEntry.newPosition) {
+            if (log.isInfoEnabled()) {
+                log.info("Skipping updating mark delete position to {}. The mark delete position update "
+                        + "in progress {} is latest.", mdEntry.newPosition, inProgressLatest);

Review Comment:
   done



##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java:
##########
@@ -1793,6 +1830,34 @@ protected void internalAsyncMarkDelete(final PositionImpl newPosition, Map<Strin
     }
 
     void internalMarkDelete(final MarkDeleteEntry mdEntry) {
+        if (persistentMarkDeletePosition != null
+                && mdEntry.newPosition.compareTo(persistentMarkDeletePosition) < 0) {
+            if (log.isInfoEnabled()) {
+                log.info("Skipping updating mark delete position to {}. The persisted mark delete position {} "
+                        + "is latest.", mdEntry.newPosition, persistentMarkDeletePosition);

Review Comment:
   done



-- 
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 #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java:
##########
@@ -1793,6 +1830,34 @@ protected void internalAsyncMarkDelete(final PositionImpl newPosition, Map<Strin
     }
 
     void internalMarkDelete(final MarkDeleteEntry mdEntry) {
+        if (persistentMarkDeletePosition != null
+                && mdEntry.newPosition.compareTo(persistentMarkDeletePosition) < 0) {
+            if (log.isInfoEnabled()) {
+                log.info("Skipping updating mark delete position to {}. The persisted mark delete position {} "
+                        + "is latest.", mdEntry.newPosition, persistentMarkDeletePosition);
+            }
+            mdEntry.triggerComplete();
+            return;
+        }
+
+        PositionImpl inProgressLatest = INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER.updateAndGet(this, current -> {
+            if (current != null && current.compareTo(mdEntry.newPosition) > 0) {
+                return current;
+            } else {
+                return mdEntry.newPosition;
+            }
+        });
+
+        // if there's a newer or equal mark delete update in progress, skip it.
+        if (inProgressLatest != mdEntry.newPosition) {
+            if (log.isInfoEnabled()) {
+                log.info("Skipping updating mark delete position to {}. The mark delete position update "
+                        + "in progress {} is latest.", mdEntry.newPosition, inProgressLatest);

Review Comment:
   Nit: I think we might want to replace `latest` with `later` because `latest` already has a meaning for a topic.



##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java:
##########
@@ -1793,6 +1830,34 @@ protected void internalAsyncMarkDelete(final PositionImpl newPosition, Map<Strin
     }
 
     void internalMarkDelete(final MarkDeleteEntry mdEntry) {
+        if (persistentMarkDeletePosition != null
+                && mdEntry.newPosition.compareTo(persistentMarkDeletePosition) < 0) {
+            if (log.isInfoEnabled()) {
+                log.info("Skipping updating mark delete position to {}. The persisted mark delete position {} "
+                        + "is latest.", mdEntry.newPosition, persistentMarkDeletePosition);

Review Comment:
   Nit: same comment about replacing `latest` with `later`.
   



##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java:
##########
@@ -1793,6 +1830,34 @@ protected void internalAsyncMarkDelete(final PositionImpl newPosition, Map<Strin
     }
 
     void internalMarkDelete(final MarkDeleteEntry mdEntry) {
+        if (persistentMarkDeletePosition != null
+                && mdEntry.newPosition.compareTo(persistentMarkDeletePosition) < 0) {
+            if (log.isInfoEnabled()) {
+                log.info("Skipping updating mark delete position to {}. The persisted mark delete position {} "
+                        + "is latest.", mdEntry.newPosition, persistentMarkDeletePosition);
+            }
+            mdEntry.triggerComplete();
+            return;
+        }
+
+        PositionImpl inProgressLatest = INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER.updateAndGet(this, current -> {
+            if (current != null && current.compareTo(mdEntry.newPosition) > 0) {
+                return current;
+            } else {
+                return mdEntry.newPosition;
+            }
+        });
+
+        // if there's a newer or equal mark delete update in progress, skip it.
+        if (inProgressLatest != mdEntry.newPosition) {
+            if (log.isInfoEnabled()) {
+                log.info("Skipping updating mark delete position to {}. The mark delete position update "
+                        + "in progress {} is latest.", mdEntry.newPosition, inProgressLatest);
+            }
+            mdEntry.triggerComplete();

Review Comment:
   Is it correct to complete this `MarkDeleteEntry`? It seems to me that it's ambiguous at this point in the code because the current `inProgressLatest` might be in the process of persisting that position as mark deleted. I'd think we should comply complete that `MarkDeleteEntry` once the in progress mark delete is completed.



-- 
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] congbobo184 commented on a diff in pull request #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java:
##########
@@ -1814,6 +1876,9 @@ public void operationComplete() {
                             mdEntry.newPosition);
                 }
 
+                INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER.compareAndSet(ManagedCursorImpl.this,
+                        mdEntry.newPosition, null);
+

Review Comment:
   internalMarkDelete is not a sync method, so persistPositionToLedger may persist 2 then persist 1



-- 
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 a diff in pull request #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java:
##########
@@ -1814,6 +1876,9 @@ public void operationComplete() {
                             mdEntry.newPosition);
                 }
 
+                INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER.compareAndSet(ManagedCursorImpl.this,
+                        mdEntry.newPosition, null);
+

Review Comment:
   @congbobo184 Locks could have surprising side effects, such as deadlocks (example #11965). 
   
   Please explain the reason why `persistentMarkDeletePosition` exists and the consistency guarantees for the value.
   
   I think that using a single threaded execution model would be a better way to ensure consistency where ordering matters if that's the goal.
   
   The changes in this PR already ensure eventual consistency at some level, but there's no ordering of the persisting operations, so there are possible gaps. The question is more about the impact if `persistentMarkDeletePosition` would be off by one etc.
   
   
   



-- 
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 #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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

   > @lhotari Hi, do you have a plan to fix my comment? If not I will take over the task
   
   Yes, I'm planning to address it.


-- 
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 #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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

   > > > , the persistentMarkDeletePosition might apply the value from thread 0 or thread 1, it's uncertain, I think we should provide consistent behavior here.
   > > > And `inProgressLatest` is a local variable, `mdEntry` will not share between threads, I think they are not related to race conditions.
   > > 
   > > 
   > > @codelipenghui That's true. However, I don't think that the problem has become worse than before. It would be good to address the problems with a single threaded approach with a actor like model where there would be no concurrency.
   > 
   > I don't agree with this. This will add the complexity of the logic. We should find a way to fix this completely, not merge with other ideas
   
   Please be more specific about your feedback. I don't really get the point of your sentence. Please rephrase.
   


-- 
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] congbobo184 commented on pull request #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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

   @lhotari Hi, do you have a plan to fix my comment? If not I will take over the task


-- 
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 #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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

   the periodic flushing added in #8634 will cause problems unless this fix is applied.


-- 
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 a diff in pull request #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java:
##########
@@ -1814,6 +1876,9 @@ public void operationComplete() {
                             mdEntry.newPosition);
                 }
 
+                INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER.compareAndSet(ManagedCursorImpl.this,
+                        mdEntry.newPosition, null);
+

Review Comment:
   @congbobo184 do you mean to say that there's a problem that should be resolved in some way? What do you suggest?



-- 
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 a diff in pull request #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java:
##########
@@ -1814,6 +1876,9 @@ public void operationComplete() {
                             mdEntry.newPosition);
                 }
 
+                INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER.compareAndSet(ManagedCursorImpl.this,
+                        mdEntry.newPosition, null);
+

Review Comment:
   @congbobo184 Locks could have surprising side effects, such as deadlocks (example #11965). 
   
   Please explain the reason why `persistentMarkDeletePosition` exists and the consistency guarantees for the value.
   
   I think that using a single threaded execution model (by using an OrderedExecutor with the ledger name as the key) would be a better way to ensure consistency where ordering matters, if that's the goal.
   
   The changes in this PR already ensure eventual consistency at some level, but there's no ordering of the persisting operations, so there are possible gaps. The question is more about the impact if `persistentMarkDeletePosition` would be off by one etc.
   
   
   



-- 
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] congbobo184 commented on a diff in pull request #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java:
##########
@@ -1814,6 +1876,9 @@ public void operationComplete() {
                             mdEntry.newPosition);
                 }
 
+                INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER.compareAndSet(ManagedCursorImpl.this,
+                        mdEntry.newPosition, null);
+

Review Comment:
   can use OrderedExecutor, if we don't ensure the order of persistentMarkDeletePosition, the pendingAck log will delete more then it will not recover to a correct state



-- 
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 #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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

   @lhotari got it. Thanks.


-- 
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 #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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

   > > **This is wrong since the meaning of the field is not what one would expect it to be.**
   > 
   > @lhotari - are you able to describe the meaning of the field more?
   
   I was making this conclusion based on the name which I assume that it means "persisted mark delete position".


-- 
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 a diff in pull request #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java:
##########
@@ -1793,6 +1830,34 @@ protected void internalAsyncMarkDelete(final PositionImpl newPosition, Map<Strin
     }
 
     void internalMarkDelete(final MarkDeleteEntry mdEntry) {
+        if (persistentMarkDeletePosition != null
+                && mdEntry.newPosition.compareTo(persistentMarkDeletePosition) < 0) {
+            if (log.isInfoEnabled()) {
+                log.info("Skipping updating mark delete position to {}. The persisted mark delete position {} "
+                        + "is latest.", mdEntry.newPosition, persistentMarkDeletePosition);
+            }
+            mdEntry.triggerComplete();
+            return;
+        }
+
+        PositionImpl inProgressLatest = INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER.updateAndGet(this, current -> {
+            if (current != null && current.compareTo(mdEntry.newPosition) > 0) {
+                return current;
+            } else {
+                return mdEntry.newPosition;
+            }
+        });
+
+        // if there's a newer or equal mark delete update in progress, skip it.
+        if (inProgressLatest != mdEntry.newPosition) {
+            if (log.isInfoEnabled()) {
+                log.info("Skipping updating mark delete position to {}. The mark delete position update "
+                        + "in progress {} is latest.", mdEntry.newPosition, inProgressLatest);
+            }
+            mdEntry.triggerComplete();

Review Comment:
   You are right that it would seem like the correct solution. It would add more complexity to solve it in that way. 
   I doubt that there would be an actual benefit. I'd rather solve this later in a way where concurrency is eliminated by using a single threaded model similar to an actor model.



-- 
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 #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java:
##########
@@ -1793,6 +1830,34 @@ protected void internalAsyncMarkDelete(final PositionImpl newPosition, Map<Strin
     }
 
     void internalMarkDelete(final MarkDeleteEntry mdEntry) {
+        if (persistentMarkDeletePosition != null
+                && mdEntry.newPosition.compareTo(persistentMarkDeletePosition) < 0) {
+            if (log.isInfoEnabled()) {
+                log.info("Skipping updating mark delete position to {}. The persisted mark delete position {} "
+                        + "is latest.", mdEntry.newPosition, persistentMarkDeletePosition);
+            }
+            mdEntry.triggerComplete();
+            return;
+        }
+
+        PositionImpl inProgressLatest = INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER.updateAndGet(this, current -> {
+            if (current != null && current.compareTo(mdEntry.newPosition) > 0) {
+                return current;
+            } else {
+                return mdEntry.newPosition;
+            }
+        });
+
+        // if there's a newer or equal mark delete update in progress, skip it.
+        if (inProgressLatest != mdEntry.newPosition) {
+            if (log.isInfoEnabled()) {
+                log.info("Skipping updating mark delete position to {}. The mark delete position update "
+                        + "in progress {} is latest.", mdEntry.newPosition, inProgressLatest);

Review Comment:
   Nit: I think we might want to replace `latest` with `later` because `latest` already has a meaning for a subscription.



-- 
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 a diff in pull request #15067: [ML] Follow up on race condition fixes in ManagedCursorImpl #15031

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


##########
managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java:
##########
@@ -1814,6 +1876,9 @@ public void operationComplete() {
                             mdEntry.newPosition);
                 }
 
+                INPROGRESS_MARKDELETE_PERSIST_POSITION_UPDATER.compareAndSet(ManagedCursorImpl.this,
+                        mdEntry.newPosition, null);
+

Review Comment:
   The change to use a single threaded model can be done in the future. This PR fixes the immediate problems.



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