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

perf(topn): compact the (Group)TopN output to avoid amplification #19451

Merged
merged 6 commits into from
Nov 21, 2024

Conversation

stdrc
Copy link
Member

@stdrc stdrc commented Nov 19, 2024

I hereby agree to the terms of the RisingWave Labs, Inc. Contributor License Agreement.

What's changed and what's your intention?

Previously we produced intermediate result containing redundant rows in GroupTopN, this PR uses a newly introduced TopNStaging structure to compact the result.

For instance, with offset = 0, limit = 1, group by col 0, 1, order by col 2, this input chunk

image.png

will produce this output

image.png

but it actually should produce only one row:

+ 0 0 4

Benchmark Results

Nexmark Throughtput
q9 +42.9%
q17 +25.4%
q18 +9.8%
q19 +15.9%
q105 +32.3%

Checklist

  • I have written necessary rustdoc comments
  • I have added necessary unit tests and integration tests
  • I have added test labels as necessary. See details.
  • I have added fuzzing tests or opened an issue to track them. (Optional, recommended for new SQL features Sqlsmith: Sql feature generation #7934).
  • My PR contains breaking changes. (If it deprecates some features, please create a tracking issue to remove them in the future).
  • All checks passed in ./risedev check (or alias, ./risedev c)
  • My PR changes performance-critical code. (Please run macro/micro-benchmarks and show the results.)
  • My PR contains critical fixes that are necessary to be merged into the latest release. (Please check out the details)

Documentation

  • My PR needs documentation updates. (Please use the Release note section below to summarize the impact on users)

Release note

If this PR includes changes that directly affect users or other significant modifications relevant to the community, kindly draft a release note to provide a concise summary of these changes. Please prioritize highlighting the impact these changes will have on users.

Copy link
Member Author

stdrc commented Nov 19, 2024

This stack of pull requests is managed by Graphite. Learn more about stacking.

@stdrc stdrc changed the title introduce TopNStaging to compact the topn results perf(topn): compact the (Group)TopN output to avoid amplification Nov 19, 2024
@stdrc stdrc marked this pull request as ready for review November 19, 2024 09:53
@stdrc stdrc added need-cherry-pick-release-1.10 Open a cherry-pick PR to branch release-1.10 after the current PR is merged need-cherry-pick-release-2.0 need-cherry-pick-release-2.1 labels Nov 19, 2024
@kwannoel
Copy link
Contributor

Can we trigger a nexmark benchmark on this?

@stdrc
Copy link
Member Author

stdrc commented Nov 19, 2024

A bug is found by e2e test. Fixing...

@stdrc stdrc marked this pull request as draft November 19, 2024 17:13
@stdrc stdrc marked this pull request as ready for review November 20, 2024 08:15
@stdrc stdrc requested a review from yezizp2012 November 20, 2024 08:15
Signed-off-by: Richard Chien <[email protected]>
/// Used to manage staging changes for one `TopNCache` (for one group).
/// It should be maintained when an entry is inserted or deleted from the `middle` cache.
#[derive(Debug, Default)]
pub struct TopNStaging {
Copy link
Member

Choose a reason for hiding this comment

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

Is it possible to reuse StreamChunkCompactor?

Copy link
Member Author

Choose a reason for hiding this comment

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

Since we can achieve it with ease, I want to avoid deserialize those rows and construct StreamChunk with wastes.

Copy link
Member

Choose a reason for hiding this comment

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

avoid deserialize those rows and construct StreamChunk with wastes

This sounds reasonable. Does RowOpMap work?

with ease

Haven't read the code, but you also made a mistake. So I'm thinking it's not trivial 😄

Copy link
Member Author

Choose a reason for hiding this comment

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

It's pretty straightforward actually. I made a mistake because I'm stupid😄

@stdrc
Copy link
Member Author

stdrc commented Nov 20, 2024

benchmarking...

@xxchan xxchan requested a review from st1page November 20, 2024 09:31
@stdrc
Copy link
Member Author

stdrc commented Nov 20, 2024

benchmarking...

Done. Results are added to PR description.

@stdrc stdrc requested a review from xxchan November 21, 2024 03:39
@stdrc stdrc enabled auto-merge November 21, 2024 03:40
Copy link
Member

@BugenZhao BugenZhao left a comment

Choose a reason for hiding this comment

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

NP

Comment on lines 831 to 833
debug_assert!(!self.to_update.contains_key(&cache_key));
debug_assert!(!self.to_insert.contains_key(&cache_key));
self.to_insert.insert(cache_key, row);
Copy link
Member

Choose a reason for hiding this comment

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

Shall we enable assertion in production as well, just like mem-table conflict?

Copy link
Member Author

Choose a reason for hiding this comment

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

Actually I realized that we shouldn't do any assertion in this structure. We should leave the responsibility to TopNCache, and the latter, instead of simply assert, will do consistency_error/consistency_panic.

if self.to_insert.remove(&cache_key).is_some() {
// do nothing more
} else if let Some((old_row, _)) = self.to_update.remove(&cache_key) {
self.to_delete.insert(cache_key, old_row);
Copy link
Member

Choose a reason for hiding this comment

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

Shall we assert_eq about the new row?

@stdrc stdrc added this pull request to the merge queue Nov 21, 2024
@BugenZhao BugenZhao removed this pull request from the merge queue due to a manual request Nov 21, 2024
Signed-off-by: Richard Chien <[email protected]>
@stdrc stdrc enabled auto-merge November 21, 2024 07:14
@stdrc stdrc added this pull request to the merge queue Nov 21, 2024
@xxchan
Copy link
Member

xxchan commented Nov 21, 2024

q9 +42.9%

Really so niubi?

Merged via the queue into main with commit ca38177 Nov 21, 2024
29 of 30 checks passed
@stdrc stdrc deleted the rc/topn-perf branch November 21, 2024 08:32
@stdrc
Copy link
Member Author

stdrc commented Nov 21, 2024

q9 +42.9%

Really so niubi?

I think this is because of reduced redundant deserialization.

github-actions bot pushed a commit that referenced this pull request Nov 21, 2024
github-actions bot pushed a commit that referenced this pull request Nov 21, 2024
github-merge-queue bot pushed a commit that referenced this pull request Nov 21, 2024
stdrc added a commit that referenced this pull request Nov 21, 2024
github-merge-queue bot pushed a commit that referenced this pull request Nov 21, 2024
@stdrc stdrc removed the need-cherry-pick-release-1.10 Open a cherry-pick PR to branch release-1.10 after the current PR is merged label Nov 21, 2024
stdrc added a commit that referenced this pull request Nov 22, 2024
stdrc added a commit that referenced this pull request Nov 22, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants