apache / pulsar

Apache Pulsar - distributed pub-sub messaging system
https://pulsar.apache.org/
Apache License 2.0
13.94k stars 3.53k forks source link

[fix] Make operations on `individualDeletedMessages` in lock scope #22966

Open dao-jun opened 1 week ago

dao-jun commented 1 week ago

Motivation

In https://github.com/apache/pulsar/pull/22908 we introduced ConcurrentRoaringBitSet which is based on StampLock and RoaringBitmap to optimize the memory usage and GC pause on BitSet.

However, there is a concurrency issue on ConcurrentRoaringBitSet.

It will throw NPE when calling ConcurrentRoaringBitSet#get and ConcurrentRoaringBitSet#set in multiple threads, the situation is a little similar with https://github.com/apache/pulsar/issues/18388.

see: RoaringBitmap#add RoaringBitmap#get

It will throw NPE if use StampLock, the situation is a little similar with https://github.com/apache/pulsar/issues/18388

Modifications

  1. Remove ConcurrentBitSet
  2. Rename ConcurrentOpenLongPairRangeSet to OpenLongPairRangeSet and mark it as NotThreadSafe.
  3. Make all the operations on ManageCursorImpl#individualDeletedMessages in ReadWriteLock scope.

Verifying this change

(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:)

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

If the box was checked, please highlight the changes

Documentation

Matching PR in forked repository

PR in forked repository:

dao-jun commented 1 week ago

I think that we need to find another solution. ReadWriteLock adds a lot more overhead than StampedLock.

Yes, but RoaringBitmap is not designed for Concurrency at all, and the PR is a quick fix, we can make further improvements in the future.

dao-jun commented 1 week ago

I wonder if it would be a viable option to catch exceptions and retry with a read lock if that happens?

Then we may catch a lot of exceptions when a broker is in a large throughput, I'm not sure if the cost is less than RWLock or not.

lhotari commented 1 week ago

I wonder if it would be a viable option to catch exceptions and retry with a read lock if that happens?

Then we may catch a lot of exceptions when a broker is in a large throughput, I'm not sure if the cost is less than RWLock or not.

That's a valid concern, we should investigate the different choices and experiment.

lhotari commented 1 week ago

I think that we should revert the migration to RoaringBitSet in branch-3.0, branch-3.2 and branch-3.3 so that we don't need to rush with the solution.

lhotari commented 1 week ago

I reverted the changes in branch-3.0, branch-3.2 and branch-3.3. Here's the PR to revert the change in master branch: #22968 . It's better to have a fresh start with a proper fix that is validated so that it doesn't cause performance regressions and also addresses the concurrency issues. The concern about switching to ReadWriteLock is about it causing a performance regression. It's possible that it's not a valid concern, but let's validate that before applying the solution.

dao-jun commented 1 week ago

I did a less rigorous test:

    @Test
    public void test() {
        long start = System.currentTimeMillis();
        CountDownLatch latch = new CountDownLatch(2);
        ConcurrentRoaringBitSet bitSet = new ConcurrentRoaringBitSet();
        new Thread(() -> {
            for (int i = 0; i < 100000000; i++) {
                bitSet.set(1);
            }
            latch.countDown();
        }).start();
        new Thread(() -> {
            for (int i = 0; i < 100000000; i++) {
                bitSet.get(1);
            }
            latch.countDown();
        }).start();

        try {
            latch.await();
        } catch (InterruptedException e) {
            e.printStackTrace();
        }
        System.out.println("Time: " + (System.currentTimeMillis() - start));
    }

I started 2 threads to call get/set methods on ReadWriteLock/StampLock based ConcurrentRoaringBitSet, each thread looping 100 million times. For ReadWriteLock based ConcurrentRoaringBitSet, the total durations are around 9.5s For StampLock base ConcurrentRoaringBitSet, the total durations are around 8.5s.

Maybe we don't need to worry about the performance regression?

dao-jun commented 1 week ago

When we do Readonly operations on StampLock based ConcurrentRoaringBitSet, it does faster than ReadWriteLock(about 5 times faster), but in the case we use ConcurrentRoaringBitSet is Read and Write(about 1:1).

lhotari commented 1 week ago

When we do Readonly operations on StampLock based ConcurrentRoaringBitSet, it does faster than ReadWriteLock(about 5 times faster), but in the case we use ConcurrentRoaringBitSet is Read and Write(about 1:1).

In Pulsar we have https://github.com/apache/pulsar/tree/master/microbench module with JMH. I think JMH is better for comparisons. For Pulsar, the efficiency also matters so the comparison might not be that simple.

btw. In Pulsar ConcurrentOpenLongPairRangeSet is only used in RangeSetWrapper and the only usage of that is in ManagedCursorImpl for individualDeletedMessages. In many cases, the operations on individualDeletedMessages are already protected by the ReadWriteLock field lock in ManagedCursorImpl. It might be better to make the lock usage consistent. We wouldn't need ConcurrentRoaringBitSet in the Pulsar code base in that case as long as we document that ConcurrentOpenLongPairRangeSet isn't really thread safe. The thread safe solution could use the old solution.

dao-jun commented 1 week ago

btw. In Pulsar ConcurrentOpenLongPairRangeSet is only used in RangeSetWrapper and the only usage of that is in ManagedCursorImpl for individualDeletedMessages. In many cases, the operations on individualDeletedMessages are already protected by the ReadWriteLock field lock in ManagedCursorImpl. It might be better to make the lock usage consistent. We wouldn't need ConcurrentRoaringBitSet in the Pulsar code base in that case as long as we document that ConcurrentOpenLongPairRangeSet isn't really thread safe. The thread safe solution could use the old solution.

It makes sense, I addressed this, PTAL

lhotari commented 1 week ago

It makes sense, I addressed this, PTAL

@dao-jun Looks good, I'll soon review in more detail. Please update the PR title and description so that it describes the motivation and modifications of this PR more accurately.

lhotari commented 1 week ago

Since the previous change #22908 was rollbacked by #22968, please rebase the changes.

lhotari commented 1 week ago

Please use write lock for individualDeletedMessages.resetDirtyKeys(); call in buildIndividualDeletedMessageRanges method.

This is actually a real bug in the current implementation and needs to be fixed even if we wouldn't switch to use RoaringBitMap's RoaringBitSet.

lhotari commented 1 week ago

Rename ConcurrentOpenLongPairRangeSet to OpenLongPairRangeSet and mark it as NotThreadSafe.

I guess this change and the switch to use RoaringBitSet (in version 1.1.0) was lost in rebasing?

One possibility would be to complete this PR by switching to the non-thread version of ConcurrentOpenLongPairRangeSet using ordinary BitSet in this PR and then switch to use RoaringBitSet in a follow up PR.

It's possible that using StampedLock in ConcurrentBitSet results in similar problems as we had with StampedLock in ConcurrentRoaringBitSet.

By looking at the code of BitSet, it seems that assertions in this method could fail in ConcurrentBitSet:

   private void checkInvariants() {
        assert(wordsInUse == 0 || words[wordsInUse - 1] != 0);
        assert(wordsInUse >= 0 && wordsInUse <= words.length);
        assert(wordsInUse == words.length || words[wordsInUse] == 0);
    }

However the problems are hidden since assertions aren't commonly enabled in production.

dao-jun commented 1 week ago

Please use write lock for individualDeletedMessages.resetDirtyKeys(); call in buildIndividualDeletedMessageRanges method.

This is actually a real bug in the current implementation and needs to be fixed even if we wouldn't switch to use RoaringBitMap's RoaringBitSet.

Yes, individualDeletedMessages.resetDirtyKeys() is a WRITE operation, but it just requires a READ lock.

codecov-commenter commented 1 week ago

Codecov Report

Attention: Patch coverage is 91.48936% with 4 lines in your changes missing coverage. Please review.

Project coverage is 73.43%. Comparing base (bbc6224) to head (66b228c). Report is 424 commits behind head on master.

Additional details and impacted files [![Impacted file tree graph](https://app.codecov.io/gh/apache/pulsar/pull/22966/graphs/tree.svg?width=650&height=150&src=pr&token=acYqCpsK9J&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)](https://app.codecov.io/gh/apache/pulsar/pull/22966?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) ```diff @@ Coverage Diff @@ ## master #22966 +/- ## ============================================ - Coverage 73.57% 73.43% -0.15% - Complexity 32624 33219 +595 ============================================ Files 1877 1903 +26 Lines 139502 142680 +3178 Branches 15299 15574 +275 ============================================ + Hits 102638 104771 +2133 - Misses 28908 29891 +983 - Partials 7956 8018 +62 ``` | [Flag](https://app.codecov.io/gh/apache/pulsar/pull/22966/flags?src=pr&el=flags&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | | |---|---|---| | [inttests](https://app.codecov.io/gh/apache/pulsar/pull/22966/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `27.79% <38.29%> (+3.21%)` | :arrow_up: | | [systests](https://app.codecov.io/gh/apache/pulsar/pull/22966/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `24.76% <36.17%> (+0.44%)` | :arrow_up: | | [unittests](https://app.codecov.io/gh/apache/pulsar/pull/22966/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `72.46% <91.48%> (-0.39%)` | :arrow_down: | Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#carryforward-flags-in-the-pull-request-comment) to find out more. | [Files](https://app.codecov.io/gh/apache/pulsar/pull/22966?dropdown=coverage&src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | | |---|---|---| | [...apache/bookkeeper/mledger/ManagedLedgerConfig.java](https://app.codecov.io/gh/apache/pulsar/pull/22966?src=pr&el=tree&filepath=managed-ledger%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fbookkeeper%2Fmledger%2FManagedLedgerConfig.java&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-bWFuYWdlZC1sZWRnZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2Jvb2trZWVwZXIvbWxlZGdlci9NYW5hZ2VkTGVkZ2VyQ29uZmlnLmphdmE=) | `96.38% <ø> (+0.08%)` | :arrow_up: | | [.../common/util/collections/OpenLongPairRangeSet.java](https://app.codecov.io/gh/apache/pulsar/pull/22966?src=pr&el=tree&filepath=pulsar-common%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fpulsar%2Fcommon%2Futil%2Fcollections%2FOpenLongPairRangeSet.java&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cHVsc2FyLWNvbW1vbi9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcHVsc2FyL2NvbW1vbi91dGlsL2NvbGxlY3Rpb25zL09wZW5Mb25nUGFpclJhbmdlU2V0LmphdmE=) | `90.00% <100.00%> (ø)` | | | [...pache/bookkeeper/mledger/impl/RangeSetWrapper.java](https://app.codecov.io/gh/apache/pulsar/pull/22966?src=pr&el=tree&filepath=managed-ledger%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fbookkeeper%2Fmledger%2Fimpl%2FRangeSetWrapper.java&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-bWFuYWdlZC1sZWRnZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2Jvb2trZWVwZXIvbWxlZGdlci9pbXBsL1JhbmdlU2V0V3JhcHBlci5qYXZh) | `94.33% <80.00%> (ø)` | | | [...che/bookkeeper/mledger/impl/ManagedCursorImpl.java](https://app.codecov.io/gh/apache/pulsar/pull/22966?src=pr&el=tree&filepath=managed-ledger%2Fsrc%2Fmain%2Fjava%2Forg%2Fapache%2Fbookkeeper%2Fmledger%2Fimpl%2FManagedCursorImpl.java&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-bWFuYWdlZC1sZWRnZXIvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2Jvb2trZWVwZXIvbWxlZGdlci9pbXBsL01hbmFnZWRDdXJzb3JJbXBsLmphdmE=) | `80.00% <92.68%> (+0.70%)` | :arrow_up: | ... and [469 files with indirect coverage changes](https://app.codecov.io/gh/apache/pulsar/pull/22966/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
lhotari commented 6 days ago

LGTM, good work @dao-jun