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

feat: compact_noop_update_after_outer_join #17568

Merged
merged 8 commits into from
Jul 9, 2024

Conversation

st1page
Copy link
Contributor

@st1page st1page commented Jul 4, 2024

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

What's changed and what's your intention?

to solve #17450

use the SQL to test if is takes effect.

create table t1(v int, k int primary key);
create table d(vv int, k int primary key);
INSERT into t1 VALUES (1,1);
INSERT into d values (1,1);

create materialized view mv as 
select t1.v, t1.k 
from t1 
full outer join d d1 using(k)
full outer join d d2 using(k)
full outer join d d3 using(k)
full outer join d d4 using(k)
full outer join d d5 using(k)
full outer join d d6 using(k)
full outer join d d7 using(k)
full outer join d d8 using(k)
full outer join d d9 using(k)
full outer join d d10 using(k);

UPDATE t1 set v = 2; 

before
image
after
image

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.

@st1page st1page requested a review from fuyufjh July 4, 2024 08:16
@fuyufjh
Copy link
Member

fuyufjh commented Jul 4, 2024

When need_compact is true, the overhead will be buffering all output data in memory. Considering that we have introduced MemTable spilling to avoid OOM of Join, this overhead will bring OOM back.

Also, the side effect of making U-/U+ into -/+ also looks bad to me.

@st1page
Copy link
Contributor Author

st1page commented Jul 4, 2024

When need_compact is true, the overhead will be buffering all output data in memory. Considering that we have introduced MemTable spilling to avoid OOM of Join, this overhead will bring OOM back.

This PR will not buffer the chunks, instead, it will only do compact in each chunk.

Also, the side effect of making U-/U+ into -/+ also looks bad to me.

I still think the U-/U+ here is not dependable because it does not take into account the stream key, which can easily lead to different stream keys for a pair of U- and U+. This has already caused some bugs in the previous implementation of the over window. #12539

@fuyufjh
Copy link
Member

fuyufjh commented Jul 4, 2024

When need_compact is true, the overhead will be buffering all output data in memory. Considering that we have introduced MemTable spilling to avoid OOM of Join, this overhead will bring OOM back.

This PR will not buffer the chunks, instead, it will only do compact in each chunk.

Hmmm, I think the problem #17450 never happen in one chunk. It's shown in your example: https://github.com/risingwavelabs/risingwave/pull/17449/files

@st1page
Copy link
Contributor Author

st1page commented Jul 4, 2024

When need_compact is true, the overhead will be buffering all output data in memory. Considering that we have introduced MemTable spilling to avoid OOM of Join, this overhead will bring OOM back.

This PR will not buffer the chunks, instead, it will only do compact in each chunk.

Hmmm, I think the problem #17450 never happen in one chunk. It's shown in your example: https://github.com/risingwavelabs/risingwave/pull/17449/files

I am just realize that we can just compact the output chunk to solve the issue.

                U- 1 1 1 1
                U+ . . 1 1
                U- . . 1 1
                U+ 1 2 1 1

will be

                - 1 1 1 1
                + 1 2 1 1

@st1page st1page marked this pull request as draft July 5, 2024 08:39
Comment on lines 160 to 181
pub fn post_process(c: StreamChunk) -> StreamChunk {
let mut c = StreamChunkMut::from(c);

// NOTE(st1page): remove the pattern `UpdateDel(k, old), UpdateIns(k, NULL), UpdateDel(k, NULL), UpdateIns(k, new)`
// to avoid this issue <https://github.com/risingwavelabs/risingwave/issues/17450>
let mut i = 1;
while i < c.capacity() {
if c.op(i - 1) == Op::UpdateInsert
&& c.op(i) == Op::UpdateDelete
&& c.row_ref(i) == c.row_ref(i - 1)
{
c.set_op(i - 2, Op::Delete);
c.set_vis(i - 1, false);
c.set_vis(i, false);
c.set_op(i + 1, Op::Insert);
i += 3;
} else {
i += 1;
}
}
c.into()
}
Copy link
Contributor Author

Choose a reason for hiding this comment

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

A less intrusive and lower overhead approach, PTAL again when you have time @fuyufjh .

@st1page st1page marked this pull request as ready for review July 8, 2024 07:49
Comment on lines +160 to +161
pub fn post_process(c: StreamChunk) -> StreamChunk {
let mut c = StreamChunkMut::from(c);
Copy link
Contributor

Choose a reason for hiding this comment

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

Please add some comment about this.

Also is it possible to add this in append_row_matched etc.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

only do it on the JoinChunkBuilder but not for the JoinStreamChunkBuilder

Copy link
Member

Choose a reason for hiding this comment

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

Why not adding comments :(

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 forgot. 😶‍🌫️

@st1page st1page enabled auto-merge July 9, 2024 06:26
@graphite-app graphite-app bot requested a review from a team July 9, 2024 06:38
@st1page st1page added this pull request to the merge queue Jul 9, 2024
Merged via the queue into main with commit 9da0627 Jul 9, 2024
29 of 30 checks passed
@st1page st1page deleted the sts/compact_noop_update_after_outer_join branch July 9, 2024 07:21
Kyry pushed a commit to Kyry/risingwave that referenced this pull request Jul 12, 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