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

[improve] Handle PositionInfo that's too large to serialize as a single entry #22799

Open
wants to merge 16 commits into
base: master
Choose a base branch
from

Conversation

dlg99
Copy link
Contributor

@dlg99 dlg99 commented May 29, 2024

Motivation

In some cases cursor position info can be too large to serialize as a single entry, e.g. in case of too many deleted ranges. Also the serialization can be too slow.

cherry-picks of changes by @eolivelli @nicoloboschi and I.

Modifications

Cursor PositionInfo serialization is reworked to produce less garbage/serialize faster; serialized data can be compressed.
In case the serialized data too large it is chunked and saved as a sequence of entries.

Verifying this change

  • Make sure that the change passes the CI checks.

This change added tests.

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

NO

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
  • The metrics
  • Anything that affects deployment

Documentation

  • doc
  • doc-required
  • doc-not-needed
  • doc-complete

Matching PR in forked repository

PR in forked repository: dlg99#17

@dlg99 dlg99 requested a review from eolivelli May 29, 2024 23:08
@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label May 29, 2024
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.

LGTM

Copy link
Contributor

@merlimat merlimat left a comment

Choose a reason for hiding this comment

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

Since this is touching on-disk formats it would be good to have a PIP discussion.

@lhotari
Copy link
Member

lhotari commented Aug 16, 2024

I wonder if this is related to PIP-81 which doesn't seem to have an implementation: https://github.com/apache/pulsar/wiki/PIP-81%3A-Split-the-individual-acknowledgments-into-multiple-entries

There was a larger PR for PIP-81 that was closed: #10729
Some parts of it were split, such as #15425 and #15607.
@codelipenghui @315157973 any details about future PIP-81 plans to share?

@lhotari
Copy link
Member

lhotari commented Aug 16, 2024

Btw. I'm currently investigating a Key_Shared subscription type issue where ordinary consumption of message leads to a very large number of "ack holes". The WIP test app where this is reproduced is https://github.com/lhotari/pulsar-playground/blob/master/src/main/java/com/github/lhotari/pulsar/playground/TestScenarioIssueKeyShared.java .
The test class is not yet simplified to contain the relevant parts. I started with a very complex test case and it seems that the "ack hole" problem shows up in all possible cases.

No messages get lost. It's just that some messages don't get delivered until all other messages have been processed.

@lhotari
Copy link
Member

lhotari commented Aug 16, 2024

In the 1M message experiment, the number of ack holes goes down from about 150k ack holes to <500 with this experiment: lhotari@a3b0639

@lhotari
Copy link
Member

lhotari commented Aug 27, 2024

It's possible that the root cause of this issue of large PositionInfo is #23200 and it is addressed with PRs #23231 and #23226. There's #23224 for observability. Large msgInReplay counts would confirm the root cause.

@rdhabalia
Copy link
Contributor

this is a real problem and it has been solved with a simple and fundamentally proven solution with perf numbers : #9292

But again I am not sure some folks blocked this PR without saying the reason even after asking multiple times and blocked the progress on this PR.

@315157973
Copy link
Contributor

I wonder if this is related to PIP-81 which doesn't seem to have an implementation: https://github.com/apache/pulsar/wiki/PIP-81%3A-Split-the-individual-acknowledgments-into-multiple-entries

There was a larger PR for PIP-81 that was closed: #10729 Some parts of it were split, such as #15425 and #15607. @codelipenghui @315157973 any details about future PIP-81 plans to share?

Since a PR implemented the compression of PositionInfo, the size of PositionInfo can be greatly reduced, and the problem of Entry size exceeding the threshold will no longer occur, so this PIP was not further promoted.

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.

Great work Andrey!

If there's way to refactor the logic to avoid byte[] and use Netty ByteBuf when possible, the solution would be more aligned with the "no garbage" style in Pulsar.

if (log.isDebugEnabled()) {
log.debug("[{}} readComplete rc={} entryId={}", ledger.getName(), rc1, lh1.getLastAddConfirmed());
LedgerEntry entry = seq.nextElement();
byte[] data = entry.getEntry();
Copy link
Member

Choose a reason for hiding this comment

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

Please replace this with the use of getEntryBuffer so that Netty ByteBuf is used instead of byte[].
Large arrays adds significant GC overhead and that's why Netty ByteBuf is preferred.
It's better to refactor the logic to operate with Netty ByteBufs.

lh.asyncReadEntries(startPos, endPos, new AsyncCallback.ReadCallback() {
@Override
public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> entries, Object ctx) {
ByteArrayOutputStream buffer = new ByteArrayOutputStream();
Copy link
Member

Choose a reason for hiding this comment

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

ByteArrayOutputStream adds a lot of GC overhead compared to the usage of Netty ByteBufs.
Please refactor this to use Netty ByteBufs instead of using ByteArrayOutputStream.

(btw. There are also a lot of gotchas when using Netty ByteBufs, I happened to learn quite a few when working on optimizing https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGenerator.java . The gotchas mainly apply when generating very huge response buffers like it's the case with metrics. The prometheus metrics results might be 500MB of text in certain worst cases with topic level metrics enabled in brokers. In this case we wouldn't have to be concerned about those challenges with Netty ByteBufs since I guess the size of the output isn't in that range.)

}
}

static byte[] decompressDataIfNeeded(byte[] data, LedgerHandle lh) {
Copy link
Member

Choose a reason for hiding this comment

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

Refactor this to use Netty ByteBufs instead of byte[]

@lhotari
Copy link
Member

lhotari commented Sep 23, 2024

I wonder if this is related to PIP-81 which doesn't seem to have an implementation: https://github.com/apache/pulsar/wiki/PIP-81%3A-Split-the-individual-acknowledgments-into-multiple-entries
There was a larger PR for PIP-81 that was closed: #10729 Some parts of it were split, such as #15425 and #15607. @codelipenghui @315157973 any details about future PIP-81 plans to share?

Since a PR implemented the compression of PositionInfo, the size of PositionInfo can be greatly reduced, and the problem of Entry size exceeding the threshold will no longer occur, so this PIP was not further promoted.

@315157973 Are you referring to PIP-146: ManagedCursorInfo compression
or ManagedLedgerInfo compression (does that contain a PIP?) ? What if the size exceeds the threshold after compression?

eolivelli and others added 14 commits September 23, 2024 15:53
* serialize/compress without intermediate byte arrays
* use lightproto for cursor serialization to the ledger
* Reuse PositionInfo

(cherry picked from commit 1887c44)
(cherry picked from commit 98a3d25)
* ManagedCursor: manually serialise PositionInfo
* Add tests and save last serialized side to prevent reallocations

(cherry picked from commit 8a365d0)
(cherry picked from commit 44ba614)
(cherry picked from commit c3fe80e)
…footer of the chunked data (apache#282)

(cherry picked from commit 6e72ecb)
@codecov-commenter
Copy link

Codecov Report

Attention: Patch coverage is 38.98757% with 687 lines in your changes missing coverage. Please review.

Project coverage is 74.28%. Comparing base (bbc6224) to head (1397faf).
Report is 599 commits behind head on master.

Files with missing lines Patch % Lines
...che/bookkeeper/mledger/impl/PositionInfoUtils.java 26.32% 597 Missing and 16 partials ⚠️
...che/bookkeeper/mledger/impl/ManagedCursorImpl.java 74.00% 56 Missing and 16 partials ⚠️
...e/bookkeeper/mledger/impl/LedgerMetadataUtils.java 50.00% 1 Missing and 1 partial ⚠️
Additional details and impacted files

Impacted file tree graph

@@             Coverage Diff              @@
##             master   #22799      +/-   ##
============================================
+ Coverage     73.57%   74.28%   +0.70%     
- Complexity    32624    34457    +1833     
============================================
  Files          1877     1935      +58     
  Lines        139502   146033    +6531     
  Branches      15299    15998     +699     
============================================
+ Hits         102638   108477    +5839     
- Misses        28908    29247     +339     
- Partials       7956     8309     +353     
Flag Coverage Δ
inttests 27.56% <20.78%> (+2.98%) ⬆️
systests 24.59% <19.53%> (+0.26%) ⬆️
unittests 73.62% <38.98%> (+0.78%) ⬆️

Flags with carried forward coverage won't be shown. Click here to find out more.

Files with missing lines Coverage Δ
.../apache/bookkeeper/mledger/impl/MetaStoreImpl.java 86.34% <100.00%> (+0.43%) ⬆️
...e/bookkeeper/mledger/impl/LedgerMetadataUtils.java 91.66% <50.00%> (-8.34%) ⬇️
...che/bookkeeper/mledger/impl/ManagedCursorImpl.java 79.05% <74.00%> (-0.25%) ⬇️
...che/bookkeeper/mledger/impl/PositionInfoUtils.java 26.32% <26.32%> (ø)

... and 602 files with indirect coverage changes

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
doc-not-needed Your PR changes do not impact docs ready-to-test
Projects
None yet
Development

Successfully merging this pull request may close these issues.

8 participants