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
1 change: 1 addition & 0 deletions e2e_test/batch/aggregate/two_phase_agg.slt.part
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@ SET RW_IMPLICIT_FLUSH TO true;

statement ok
SET RW_ENABLE_TWO_PHASE_AGG=false;

# This should override `RW_ENABLE_TWO_PHASE_AGG`, enabling it.
statement ok
SET RW_FORCE_TWO_PHASE_AGG=true;
Expand Down
12 changes: 9 additions & 3 deletions src/common/src/array/compact_chunk.rs
Original file line number Diff line number Diff line change
Expand Up @@ -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?

tracing::warn!("double insert for the same pk");
e.insert(RowOp::Insert(v));
}
RowOp::Update((ref old_v, _)) => {
tracing::warn!("double insert for the same pk");
e.insert(RowOp::Update((*old_v, v)));
}
},
}
Expand All @@ -135,7 +140,8 @@ impl<'a, 'b> RowOpMap<'a, 'b> {
e.insert(RowOp::Delete(*prev));
}
RowOp::Delete(_) => {
tracing::warn!("double delete");
tracing::warn!("double delete for the same pk");
e.insert(RowOp::Delete(v));
}
},
}
Expand Down
16 changes: 10 additions & 6 deletions src/stream/src/executor/sink.rs
Original file line number Diff line number Diff line change
Expand Up @@ -187,16 +187,19 @@ impl<F: LogStoreFactory> SinkExecutor<F> {
// So the delete event is not to delete the inserted record in our internal streaming SQL semantic.
let need_advance_delete =
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.

&& !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.

let processed_input = Self::process_msg(
input,
self.sink_param.sink_type,
stream_key,
need_advance_delete,
// NOTE(st1page): reconstruct with sink pk need extra cost to buffer a barrier's data, so currently we bind it with mismatch case.
need_advance_delete,
re_construct_with_sink_pk,
self.chunk_size,
self.input_data_types,
self.sink_param.downstream_pk.clone(),
);

if self.sink.is_sink_into_table() {
Expand Down Expand Up @@ -287,6 +290,7 @@ impl<F: LogStoreFactory> SinkExecutor<F> {
}
}

#[allow(clippy::too_many_arguments)]
#[try_stream(ok = Message, error = StreamExecutorError)]
async fn process_msg(
input: impl MessageStream,
Expand All @@ -296,6 +300,7 @@ impl<F: LogStoreFactory> SinkExecutor<F> {
re_construct_with_sink_pk: bool,
chunk_size: usize,
input_data_types: Vec<DataType>,
down_stream_pk: Vec<usize>,
) {
// need to buffer chunks during one barrier
if need_advance_delete || re_construct_with_sink_pk {
Expand Down Expand Up @@ -333,7 +338,7 @@ impl<F: LogStoreFactory> SinkExecutor<F> {
chunks
};
let chunks = if re_construct_with_sink_pk {
st1page marked this conversation as resolved.
Show resolved Hide resolved
StreamChunkCompactor::new(stream_key.clone(), chunks)
StreamChunkCompactor::new(down_stream_pk.clone(), chunks)
.reconstructed_compacted_chunks(
chunk_size,
input_data_types.clone(),
Expand Down Expand Up @@ -712,8 +717,7 @@ mod test {
StreamChunk::from_pretty(
" I I I
U- 1 1 10
U+ 1 1 40
+ 1 3 30",
U+ 1 1 40",
)
);

Expand Down
Loading