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 stream chunk with sink key when sink key mismatch #15345

Merged
merged 15 commits into from
Mar 22, 2024

Conversation

st1page
Copy link
Contributor

@st1page st1page commented Feb 28, 2024

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

What's changed and what's your intention?

In sink reconstruct the stream chunk and reorder the RowOp to make delete and insert on the same key adjacent. It is to make sinkImpl identity the update operation and omit the delete record for some downstream system.
Because it is expensive, currently only do it when sink pk and stream key mismatch.

close #13138

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 assigned st1page and unassigned st1page Feb 28, 2024
@st1page st1page marked this pull request as draft February 28, 2024 13:47
@st1page st1page requested a review from wenym1 March 4, 2024 04:00
@st1page st1page marked this pull request as ready for review March 4, 2024 04:01
@st1page st1page requested a review from hzxa21 March 4, 2024 04:01
@st1page st1page marked this pull request as draft March 13, 2024 06:53
@st1page st1page marked this pull request as ready for review March 14, 2024 09:30
@st1page
Copy link
Contributor Author

st1page commented Mar 18, 2024

have fixed to use sink's pk to compact, would you please please review later? @hzxa21 @wenym1

@@ -114,8 +114,13 @@ impl<'a, 'b> RowOpMap<'a, 'b> {
RowOp::Delete(ref old_v) => {
e.insert(RowOp::Update((*old_v, v)));
}
RowOp::Insert(_) | RowOp::Update(_) => {
tracing::warn!("double insert");
RowOp::Insert(_) => {
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we log more information in the pk to help debug?

@@ -88,6 +91,8 @@ impl<F: LogStoreFactory> SinkExecutor<F> {
sink_param: SinkParam,
columns: Vec<ColumnCatalog>,
log_store_factory: F,
chunk_size: usize,
input_data_types: Vec<DataType>,
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe we can get the input_data_types from executor.schema() instead of passing it separatedly?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

after #15167 I think it is banned

stream_key_sink_pk_mismatch && self.sink_param.sink_type != SinkType::AppendOnly;
// NOTE(st1page): reconstruct with sink pk need extra cost to buffer a barrier's data, so currently we bind it with mismatch case.
let re_construct_with_sink_pk = need_advance_delete
&& self.sink_param.sink_type == SinkType::Upsert
Copy link
Contributor

Choose a reason for hiding this comment

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

Is this == SinkType::Upsert necessary? Because if need_advance_delete, it must != SinkType::AppendOnly.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It is for the forceAppendOnly case.

// NOTE(st1page): reconstruct with sink pk need extra cost to buffer a barrier's data, so currently we bind it with mismatch case.
let re_construct_with_sink_pk = need_advance_delete
&& self.sink_param.sink_type == SinkType::Upsert
&& !self.sink_param.downstream_pk.is_empty();
Copy link
Contributor

Choose a reason for hiding this comment

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

Should we assert that downstream_pk must be non-empty here? IIRC, for upsert sink we require the users to set downstream pk.

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 do not like to assert it in such a code branch and I think there should be some assert in other places. here is just a defensive programming to prevent too strange behavior here.

@hzxa21
Copy link
Collaborator

hzxa21 commented Mar 19, 2024

I think we are still mixing two independent concepts served for different purposes here. Let me leave a note here to clarify the behavior:

Compaction by stream_key (i.e. need_advance_delete)

This is a correctness requirement, not an optimization . In other words, compaction by stream_key is a must and cannot be skipped when sink_pk != stream_key.

Compaction by stream_pk will advance the DELETE messages before INSERT messages for the same stream_key within the same epoch. This is the only way to ensure correctness.

Compaction by sink_pk (i.e. re_construct_with_sink_pk)

This is an optimization, not a correctness requirement. In other words, regardless of what the relationship is between sink_pk and stream_key, compaction by sink_pk is optional. Given that it is an optimization with the buffering and compaction overhead, I think eventually it is better to let user choose whether to enable it. I am okay with not providing this option and enabling it forcedly when sink_pk != stream_key for now though.

Compaction by sink_pk has two potential implementations:

  1. Ensure there is only one change emitted per sink_pk in one epoch. Buffering all messages within an epoch is required and this is what this PR does.
  2. Ensure there is no duplicate updates per sink_pk. This can be done by buffering delete messages on demand (ideas from @wenym1). If most duplicates are caused by delete + insert, this approach just need to maintain a very small buffer.

@st1page
Copy link
Contributor Author

st1page commented Mar 19, 2024

I think we are still mixing two independent concepts served for different purposes here. Let me leave a note here to clarify the behavior:

Compaction by stream_key (i.e. need_advance_delete)

This is a correctness requirement, not an optimization . In other words, compaction by stream_key is a must and cannot be skipped when sink_pk != stream_key.

Compaction by stream_pk will advance the DELETE messages before INSERT messages for the same stream_key within the same epoch. This is the only way to ensure correctness.

Compaction by sink_pk (i.e. re_construct_with_sink_pk)

This is an optimization, not a correctness requirement. In other words, regardless of what the relationship is between sink_pk and stream_key, compaction by sink_pk is optional. Given that it is an optimization with the buffering and compaction overhead, I think eventually it is better to let user choose whether to enable it. I am okay with not providing this option and enabling it forcedly when sink_pk != stream_key for now though.

Compaction by sink_pk has two potential implementations:

  1. Ensure there is only one change emitted per sink_pk in one epoch. Buffering all messages within an epoch is required and this is what this PR does.
  2. Ensure there is no duplicate updates per sink_pk. This can be done by buffering delete messages on demand (ideas from @wenym1). If most duplicates are caused by delete + insert, this approach just need to maintain a very small buffer.

Agree, so there are remaining works for Compaction by sink_pk, I will create issues

  1. add a session varible to force enable sink pk compaction
  2. optimze it with only buffering delete message. That's reasonable because there are less DELETE operations.

I am okay with not providing this option and enabling it forcedly when sink_pk != stream_key for now though

And I also think it is ok to merge this PR first because the main side effect of "Compaction by stream_key" and "Compaction by sink_pk" are the same, buffering all message during the epoch.

Copy link
Collaborator

@hzxa21 hzxa21 left a comment

Choose a reason for hiding this comment

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

I think we are still mixing two independent concepts served for different purposes here. Let me leave a note here to clarify the behavior:

Compaction by stream_key (i.e. need_advance_delete)

This is a correctness requirement, not an optimization . In other words, compaction by stream_key is a must and cannot be skipped when sink_pk != stream_key.
Compaction by stream_pk will advance the DELETE messages before INSERT messages for the same stream_key within the same epoch. This is the only way to ensure correctness.

Compaction by sink_pk (i.e. re_construct_with_sink_pk)

This is an optimization, not a correctness requirement. In other words, regardless of what the relationship is between sink_pk and stream_key, compaction by sink_pk is optional. Given that it is an optimization with the buffering and compaction overhead, I think eventually it is better to let user choose whether to enable it. I am okay with not providing this option and enabling it forcedly when sink_pk != stream_key for now though.
Compaction by sink_pk has two potential implementations:

  1. Ensure there is only one change emitted per sink_pk in one epoch. Buffering all messages within an epoch is required and this is what this PR does.
  2. Ensure there is no duplicate updates per sink_pk. This can be done by buffering delete messages on demand (ideas from @wenym1). If most duplicates are caused by delete + insert, this approach just need to maintain a very small buffer.

Agree, so there are remaining works for Compaction by sink_pk, I will create issues

  1. add a session varible to force enable sink pk compaction
  2. optimze it with only buffering delete message. That's reasonable because there are less DELETE operations.

I am okay with not providing this option and enabling it forcedly when sink_pk != stream_key for now though

And I also think it is ok to merge this PR first because the main side effect of "Compaction by stream_key" and "Compaction by sink_pk" are the same, buffering all message during the epoch.

+1. Rest LGTM!

src/common/src/array/compact_chunk.rs Outdated Show resolved Hide resolved
src/common/src/array/compact_chunk.rs Outdated Show resolved Hide resolved
src/common/src/array/compact_chunk.rs Outdated Show resolved Hide resolved
@st1page st1page added this pull request to the merge queue Mar 22, 2024
Merged via the queue into main with commit c8be2be Mar 22, 2024
26 of 27 checks passed
@st1page st1page deleted the sts/compact_sink_with_sink_key branch March 22, 2024 05:42
github-actions bot pushed a commit that referenced this pull request Mar 22, 2024
st1page added a commit that referenced this pull request Mar 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.

feat(sink): omit old value in FORMAT UPSERT
4 participants