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

[managed-ledger] Do not send duplicate reads to BK/offloaders #17241

Merged

Conversation

eolivelli
Copy link
Contributor

@eolivelli eolivelli commented Aug 23, 2022

Motivation

When you have many subscriptions on the same topic and they are catching up (for instance after some downtime of the consumer application) Pulsar will start to perform many reads from BK for the same entries.

Modifications

Prevent concurrent reads of the same entries.

Verifying this change

This change is already covered by existing tests.

I also tested manually that with this fix we are reducing a lot the pressure on the bookies. This is something we cannot code in a integration test as it will use too many resources on CI.

With some tests with only 64 subscription we saw a reduction from 100K reads/s from BK to 250 reads/s.

@eolivelli eolivelli added this to the 2.12.0 milestone Aug 23, 2022
@eolivelli eolivelli self-assigned this Aug 23, 2022
@eolivelli eolivelli requested a review from lhotari August 23, 2022 15:53
@lhotari lhotari requested a review from hangc0276 August 24, 2022 04:57
Copy link
Member

@lhotari lhotari left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, Good work @eolivelli

Copy link
Contributor

@nicoloboschi nicoloboschi left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

@eolivelli eolivelli changed the title [managed-ledger] prevent sending duplicate reads to BK/offloaders [managed-ledger] Do not send duplicate reads to BK/offloaders Aug 24, 2022
Copy link
Contributor

@codelipenghui codelipenghui left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I understand the problem that the PR wants to resolve for now. And I think it's a good point of the reads improvement.

I have some questions

  1. We are using the PendingReadKey as the key to determining whether the new one is duplicated read or not. It looks can only cover the case that all the subscriptions with the same read position. It will be great if we can have a more general solution for the duplicated reads which able to handle the case like [0, 20], [2, 22]
  2. After this change, even if the subscription with different read position e.g. sub-a 1:10, sub-b 3:20. We will always introduce the new heap memory overhead.

I haven't thought of a specific way yet, but I think it's very necessary to discuss looking for a general solution for duplicated reads.

I think we can start the discussion on the mailing list. So that many contributors can share their ideas.

Copy link
Member

@zymap zymap left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Great work!

@zymap
Copy link
Member

zymap commented Aug 25, 2022

  1. We are using the PendingReadKey as the key to determining whether the new one is duplicated read or not. It looks can only cover the case that all the subscriptions with the same read position. It will be great if we can have a more general solution for the duplicated reads which able to handle the case like [0, 20], [2, 22]

I have an idea about this that is to split the read range into small pieces, then different read can share the pieces to construct their read. There will have a tradeoff, if the pieces are smaller, the different reads will share more.
Something like this:
image

Copy link
Contributor

@Jason918 Jason918 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@eolivelli
Copy link
Contributor Author

@zymap thank you for your idea.
I am also thinking to the same think in order to allow more chances to match the same range.
There is an open discussion on the ML, maybe you can post your idea there.

@eolivelli
Copy link
Contributor Author

@codelipenghui @Jason918 @zymap @lhotari @nicoloboschi
I have updated this patch.

We did more testing and actually trying to find pending reads with ranges that "include" the requested range allows us to have many more "hits".

image

in the picture:

  • green line is for "perfect matches" (previous version of the patch)
  • the blue line is the rate of "partial matches" (excluding the perfect matches)
  • the yellow line is "no pending read to attach to"

the rate of "partial matches" is usually higher or equals to the rate of "misses" both during tailing reads and catch up reads.

Comment on lines 449 to 450
AtomicBoolean createdByThisThread = new AtomicBoolean();
CachedPendingRead cachedPendingRead = findBestCandidate(key,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we can only make the findBestCandidate() return null if no candidate and then create a new one to continue the read operation.

So that we don't need AtomicBoolean createdByThisThread = new AtomicBoolean(); and the code is easy to read.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

findBestCandidate is kind of a computeIfAbsent method (when I was not looking for "includes" it was actually computeIfAbsent on a ConcurrentHashMap).
So I have to create the object and put it into the map inside the "lock"

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I just want to avoid creating AtomicBoolean here.
Can we use the callback size of the cachedPendingRead or a boolean inside CachedPendingRead?

return result;
}

public void attach(CompletableFuture<List<EntryImpl>> handle) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to remove the CachedPendingRead from the cachedPendingReads?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

}
}

@Override
public void clear() {
Pair<Integer, Long> removedPair = entries.clear();
manager.entriesRemoved(removedPair.getRight(), removedPair.getLeft());
cachedPendingReads.clear();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I noticed the element of the cachedPendingReads only be removed here. This one will be called if all the cached data should be clean up

It's better to add a unit test for this one to make sure we will not introduce any heap memory leak.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the elements here are one entry per ledger.
the per ledger maps are evicted here
https://github.com/apache/pulsar/pull/17241/files#diff-c55509e3ab1389d89a58fd564f2e318dbb95f50121ab33c729a7ca4a21d02ef1R340

We could remove the entry for a ledger in case of rollover.
but I thought it requires more coordination.
I will try to improve this

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We could remove the entry for a ledger in case of rollover.
but I thought it requires more coordination.
I will try to improve this

Yes, we are on the same page.


private CachedPendingRead findBestCandidate(PendingReadKey key, Map<PendingReadKey, CachedPendingRead> ledgerCache,
AtomicBoolean created) {
synchronized (ledgerCache) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can move the synchronized to method

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I want to synchronize only on the Map that is for the specific "ledger", not on the whole RangeEntryCacheImpl
this way the lock is more fine grained

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh, sorry. I read it wrong, I thought it was equal to synchronized(this).

Comment on lines 361 to 363
List<EntryImpl> copy = new ArrayList<>(entriesToReturn.size());
long callbackStartEntry = callback.startEntry;
long callbackEndEntry = callback.endEntry;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like we can change to

long callbackStartEntry = callback.startEntry;
long callbackEndEntry = callback.endEntry
List copy = new ArrayList<>(callbackEndEntry - callbackStartEntry + 1);

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

good point

}, ml.getExecutor().chooseThread(ml.getName())).exceptionally(exception -> {
synchronized (CachedPendingRead.this) {
for (ReadEntriesCallbackWithContext callback : callbacks) {
if (exception instanceof BKException
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should unwrap the exception

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

And looks like we can just use createManagedLedgerException directly without the if check here.

    public static ManagedLedgerException createManagedLedgerException(Throwable t) {
        if (t instanceof org.apache.bookkeeper.client.api.BKException) {
            return createManagedLedgerException(((org.apache.bookkeeper.client.api.BKException) t).getCode());
        } else if (t instanceof CompletionException
                && !(t.getCause() instanceof CompletionException) /* check to avoid stackoverlflow */) {
            return createManagedLedgerException(t.getCause());
        } else {
            log.error("Unknown exception for ManagedLedgerException.", t);
            return new ManagedLedgerException("Other exception", t);
        }
    }

It already handled the CompletionException and TooManyRequestsException

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure, thanks. previous code had a "invalidateLedger" that has been moved

@eolivelli eolivelli force-pushed the impl/master-skip-duplicate-bk-reads branch from 5d07b94 to 64b8101 Compare August 30, 2022 12:33
@eolivelli
Copy link
Contributor Author

@zymap @codelipenghui @Jason918 @lhotari
I have added one last step to the patch and now it is really ready:

  • added the ability to reuse pending reads that partially overlap with the requested range
  • refactored the new code to a dedicated class, in order to reduce the complexity of the code (and to ease testability)

I tried to do something more sophisticated, like reusing only overlapping pending reads only if the amount of overlapping entries is big enough, but in my testing I have seen that it seems always a good idea to reuse pending reads.

@eolivelli
Copy link
Contributor Author

@Jason918 thanks for your suggestions. I have applied them

Copy link
Member

@zymap zymap left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice improve! Looks good to me.

Do we need to add a unit test for the PendingReadsManager?

@eolivelli
Copy link
Contributor Author

eolivelli commented Sep 1, 2022

@zymap

Do we need to add a unit test for the PendingReadsManager?

I haven't pushed this part of my patch. I will update this PR soon. Thanks for the reminder.

@eolivelli
Copy link
Contributor Author

@zymap tests added

@eolivelli eolivelli merged commit 3a3a993 into apache:master Sep 2, 2022
@eolivelli eolivelli deleted the impl/master-skip-duplicate-bk-reads branch September 2, 2022 13:09
@Technoboy- Technoboy- modified the milestones: 2.12.0, 2.11.0 Oct 13, 2022
@Technoboy- Technoboy- added type/enhancement The enhancements for the existing features or docs. e.g. reduce memory usage of the delayed messages area/broker labels Oct 13, 2022
nodece added a commit to nodece/pulsar that referenced this pull request Sep 10, 2024
…#17241)

(cherry picked from commit 3a3a993)
Signed-off-by: Zixuan Liu <nodeces@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/broker type/enhancement The enhancements for the existing features or docs. e.g. reduce memory usage of the delayed messages
Projects
None yet
Development

Successfully merging this pull request may close these issues.

8 participants