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

check chunk inconsistency #20341

Draft
wants to merge 9 commits into
base: release-2.1
Choose a base branch
from
Draft
Show file tree
Hide file tree
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
69 changes: 58 additions & 11 deletions src/common/src/array/stream_chunk.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ use rand::prelude::SmallRng;
use rand::{Rng, SeedableRng};
use risingwave_common_estimate_size::EstimateSize;
use risingwave_pb::data::{PbOp, PbStreamChunk};
use rw_iter_util::ZipEqFast;

use super::stream_chunk_builder::StreamChunkBuilder;
use super::{ArrayImpl, ArrayRef, ArrayResult, DataChunkTestExt, RowRef};
Expand Down Expand Up @@ -139,10 +140,19 @@ impl StreamChunk {
) -> Self {
let ops = ops.into();
for col in &columns {
assert_eq!(col.len(), ops.len());
assert_eq!(
col.len(),
ops.len(),
"column length mismatch column: {:#?}, ops: {:#?}, vis: {:#?}",
col,
ops,
visibility
);
}
let data = DataChunk::new(columns, visibility);
StreamChunk { ops, data }
let chunk = StreamChunk { ops, data };
chunk.check_consistency();
chunk
}

/// Build a `StreamChunk` from rows.
Expand All @@ -159,7 +169,9 @@ impl StreamChunk {
debug_assert!(none.is_none());
}

builder.take().expect("chunk should not be empty")
let chunk = builder.take().expect("chunk should not be empty");
chunk.check_consistency();
chunk
}

/// Get the reference of the underlying data chunk.
Expand All @@ -170,6 +182,7 @@ impl StreamChunk {
/// compact the `StreamChunk` with its visibility map
pub fn compact(self) -> Self {
if self.is_compacted() {
self.check_consistency();
return self;
}

Expand All @@ -186,7 +199,9 @@ impl StreamChunk {
for idx in visibility.iter_ones() {
new_ops.push(ops[idx]);
}
StreamChunk::new(new_ops, columns)
let chunk = StreamChunk::new(new_ops, columns);
chunk.check_consistency();
chunk
}

/// Split the `StreamChunk` into multiple chunks with the given size at most.
Expand All @@ -210,6 +225,9 @@ impl StreamChunk {
outputs.push(output);
}

for output in &outputs {
output.check_consistency();
}
outputs
}

Expand Down Expand Up @@ -248,7 +266,9 @@ impl StreamChunk {
for column in prost.get_columns() {
columns.push(ArrayImpl::from_protobuf(column, cardinality)?.into());
}
Ok(StreamChunk::new(ops, columns))
let chunk = StreamChunk::new(ops, columns);
chunk.check_consistency();
Ok(chunk)
}

pub fn ops(&self) -> &[Op] {
Expand Down Expand Up @@ -313,10 +333,12 @@ impl StreamChunk {
/// will be `[c, b, a]`. If `indices` is [2, 0], then the output will be `[c, a]`.
/// If the input mapping is identity mapping, no reorder will be performed.
pub fn project(&self, indices: &[usize]) -> Self {
Self {
let new = Self {
ops: self.ops.clone(),
data: self.data.project(indices),
}
};
new.check_consistency();
new
}

/// Remove the adjacent delete-insert if their row value are the same.
Expand All @@ -343,23 +365,48 @@ impl StreamChunk {
prev_r = Some(curr);
}
}
c.into()
let new: StreamChunk = c.into();
new.check_consistency();
new
}

/// Reorder columns and set visibility.
pub fn project_with_vis(&self, indices: &[usize], vis: Bitmap) -> Self {
Self {
let new = Self {
ops: self.ops.clone(),
data: self.data.project_with_vis(indices, vis),
}
};
new.check_consistency();
new
}

/// Clone the `StreamChunk` with a new visibility.
pub fn clone_with_vis(&self, vis: Bitmap) -> Self {
Self {
let new = Self {
ops: self.ops.clone(),
data: self.data.with_visibility(vis),
};
new.check_consistency();
new
}

pub fn check_consistency(&self) {
let ops_len = self.ops.len();
for col in self.data.columns() {
assert_eq!(col.len(), ops_len);
}
let mut has_hanging_update_delete = false;
for (op, vis) in self.ops.iter().zip_eq_fast(self.data.visibility().iter()) {
if vis {
if matches!(op, Op::UpdateDelete) {
has_hanging_update_delete = true;
} else if matches!(op, Op::UpdateInsert) {
assert!(has_hanging_update_delete, "unmatched update insert");
has_hanging_update_delete = false;
}
}
}
assert!(!has_hanging_update_delete, "unmatched update delete");
}
}

Expand Down
1 change: 1 addition & 0 deletions src/stream/src/executor/backfill/utils.rs
Original file line number Diff line number Diff line change
Expand Up @@ -383,6 +383,7 @@ fn mark_chunk_inner(
pk_in_output_indices: PkIndicesRef<'_>,
pk_order: &[OrderType],
) -> StreamChunk {
chunk.check_consistency();
let (data, ops) = chunk.into_parts();
let mut new_visibility = BitmapBuilder::with_capacity(ops.len());
// Use project to avoid allocation.
Expand Down
6 changes: 6 additions & 0 deletions src/stream/src/executor/dispatch.rs
Original file line number Diff line number Diff line change
Expand Up @@ -749,6 +749,9 @@ impl Dispatcher for HashDataDispatcher {
}

async fn dispatch_data(&mut self, chunk: StreamChunk) -> StreamResult<()> {
// Check chunk consistency.
chunk.check_consistency();

// A chunk can be shuffled into multiple output chunks that to be sent to downstreams.
// In these output chunks, the only difference are visibility map, which is calculated
// by the hash value of each line in the input chunk.
Expand All @@ -775,6 +778,9 @@ impl Dispatcher for HashDataDispatcher {
chunk.project(&self.output_indices)
};

// Check chunk consistency.
chunk.check_consistency();

for ((vnode, &op), visible) in vnodes
.iter()
.copied()
Expand Down