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

fix(sink): do compaction in chunk for upsert sink #17174

Merged
merged 1 commit into from
Jun 11, 2024
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
29 changes: 19 additions & 10 deletions src/stream/src/executor/sink.rs
Original file line number Diff line number Diff line change
Expand Up @@ -368,16 +368,25 @@ impl<F: LogStoreFactory> SinkExecutor<F> {
// Compact the chunk to eliminate any useless intermediate result (e.g. UPDATE
// V->V).
let chunk = merge_chunk_row(chunk, &stream_key);
let chunk = if sink_type == SinkType::ForceAppendOnly {
// Force append-only by dropping UPDATE/DELETE messages. We do this when the
// user forces the sink to be append-only while it is actually not based on
// the frontend derivation result.
force_append_only(chunk)
} else {
chunk
};

yield Message::Chunk(chunk);
match sink_type {
SinkType::AppendOnly => yield Message::Chunk(chunk),
SinkType::ForceAppendOnly => {
// Force append-only by dropping UPDATE/DELETE messages. We do this when the
// user forces the sink to be append-only while it is actually not based on
// the frontend derivation result.
yield Message::Chunk(force_append_only(chunk))
}
SinkType::Upsert => {
// Making sure the optimization in https://github.com/risingwavelabs/risingwave/pull/12250 is correct,
// it is needed to do the compaction here.
for chunk in
StreamChunkCompactor::new(stream_key.clone(), vec![chunk])
Copy link
Contributor

Choose a reason for hiding this comment

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

Why we don't compact all chunks of an epoch for this case?

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 a damaging operation that leads to a decrease in the latency of the sink. Therefore we do not do it by default.

.into_compacted_chunks()
{
yield Message::Chunk(chunk)
}
}
}
}
Message::Barrier(barrier) => {
yield Message::Barrier(barrier);
Expand Down
Loading