risingwave_common/array/stream_chunk_builder.rs
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use crate::array::stream_record::Record;
use crate::array::{ArrayBuilderImpl, Op, StreamChunk};
use crate::bitmap::BitmapBuilder;
use crate::row::Row;
use crate::types::{DataType, DatumRef};
use crate::util::iter_util::ZipEqFast;
/// Build stream chunks with fixed chunk size from rows or records.
pub struct StreamChunkBuilder {
/// operations in the data chunk to build
ops: Vec<Op>,
/// arrays in the data chunk to build
column_builders: Vec<ArrayBuilderImpl>,
/// Visibility
vis_builder: BitmapBuilder,
/// Data types of columns
data_types: Vec<DataType>,
/// Max number of rows in a chunk. When it's `Some(n)`, the chunk builder will, if necessary,
/// yield a chunk of which the size is strictly less than or equal to `n` when appending records.
/// When it's `None`, the chunk builder will yield chunks only when `take` is called.
max_chunk_size: Option<usize>,
/// The initial capacity of `ops` and `ArrayBuilder`s.
initial_capacity: usize,
/// Number of currently pending rows.
size: usize,
}
impl Drop for StreamChunkBuilder {
fn drop(&mut self) {
// Possible to fail when async task gets cancelled.
if self.size != 0 {
tracing::warn!(
remaining = self.size,
"dropping non-empty stream chunk builder"
);
}
}
}
const MAX_INITIAL_CAPACITY: usize = 4096;
const DEFAULT_INITIAL_CAPACITY: usize = 64;
impl StreamChunkBuilder {
/// Create a new `StreamChunkBuilder` with a fixed max chunk size.
/// Note that in the case of ending with `Update`, the builder may yield a chunk with size
/// `max_chunk_size + 1`.
pub fn new(max_chunk_size: usize, data_types: Vec<DataType>) -> Self {
assert!(max_chunk_size > 0);
let initial_capacity = max_chunk_size.min(MAX_INITIAL_CAPACITY);
let ops = Vec::with_capacity(initial_capacity);
let column_builders = data_types
.iter()
.map(|datatype| datatype.create_array_builder(initial_capacity))
.collect();
let vis_builder = BitmapBuilder::with_capacity(initial_capacity);
Self {
ops,
column_builders,
data_types,
vis_builder,
max_chunk_size: Some(max_chunk_size),
initial_capacity,
size: 0,
}
}
/// Create a new `StreamChunkBuilder` with unlimited chunk size.
/// The builder will only yield chunks when `take` is called.
pub fn unlimited(data_types: Vec<DataType>, initial_capacity: Option<usize>) -> Self {
let initial_capacity = initial_capacity.unwrap_or(DEFAULT_INITIAL_CAPACITY);
Self {
ops: Vec::with_capacity(initial_capacity),
column_builders: data_types
.iter()
.map(|datatype| datatype.create_array_builder(initial_capacity))
.collect(),
data_types,
vis_builder: BitmapBuilder::default(),
max_chunk_size: None,
initial_capacity,
size: 0,
}
}
/// Get the current number of rows in the builder.
pub fn size(&self) -> usize {
self.size
}
/// Append an iterator of output index and datum to the builder, return a chunk if the builder
/// is full.
///
/// Note: the caller must ensure that each column occurs exactly once in `iter`.
#[must_use]
pub fn append_iter<'a>(
&mut self,
op: Op,
iter: impl IntoIterator<Item = (usize, DatumRef<'a>)>,
) -> Option<StreamChunk> {
self.append_iter_inner::<true>(op, iter)
}
/// Append a row to the builder, return a chunk if the builder is full.
#[must_use]
pub fn append_row(&mut self, op: Op, row: impl Row) -> Option<StreamChunk> {
self.append_iter_inner::<true>(op, row.iter().enumerate())
}
/// Append an invisible row to the builder, return a chunk if the builder is full.
#[must_use]
pub fn append_row_invisible(&mut self, op: Op, row: impl Row) -> Option<StreamChunk> {
self.append_iter_inner::<false>(op, row.iter().enumerate())
}
/// Append a record to the builder, return a chunk if the builder is full.
#[must_use]
pub fn append_record(&mut self, record: Record<impl Row>) -> Option<StreamChunk> {
match record {
Record::Insert { new_row } => {
self.append_iter_inner::<true>(Op::Insert, new_row.iter().enumerate())
}
Record::Delete { old_row } => {
self.append_iter_inner::<true>(Op::Delete, old_row.iter().enumerate())
}
Record::Update { old_row, new_row } => {
let none =
self.append_iter_inner::<true>(Op::UpdateDelete, old_row.iter().enumerate());
assert!(none.is_none());
self.append_iter_inner::<true>(Op::UpdateInsert, new_row.iter().enumerate())
}
}
}
/// Take all the pending data and return a chunk. If there is no pending data, return `None`.
/// Note that if this is an unlimited chunk builder, the only way to get a chunk is to call
/// `take`.
#[must_use]
pub fn take(&mut self) -> Option<StreamChunk> {
if self.size == 0 {
return None;
}
self.size = 0;
let ops = std::mem::replace(&mut self.ops, Vec::with_capacity(self.initial_capacity));
let columns = self
.column_builders
.iter_mut()
.zip_eq_fast(&self.data_types)
.map(|(builder, datatype)| {
std::mem::replace(
builder,
datatype.create_array_builder(self.initial_capacity),
)
.finish()
})
.map(Into::into)
.collect::<Vec<_>>();
let vis = std::mem::take(&mut self.vis_builder).finish();
Some(StreamChunk::with_visibility(ops, columns, vis))
}
#[must_use]
fn append_iter_inner<'a, const VIS: bool>(
&mut self,
op: Op,
iter: impl IntoIterator<Item = (usize, DatumRef<'a>)>,
) -> Option<StreamChunk> {
self.ops.push(op);
for (i, datum) in iter {
self.column_builders[i].append(datum);
}
self.vis_builder.append(VIS);
self.size += 1;
if let Some(max_chunk_size) = self.max_chunk_size {
if self.size == max_chunk_size && !op.is_update_delete() || self.size > max_chunk_size {
// Two situations here:
// 1. `self.size == max_chunk_size && op == Op::UpdateDelete`
// We should wait for next `UpdateInsert` to join the chunk.
// 2. `self.size > max_chunk_size`
// Here we assert that `self.size == max_chunk_size + 1`. It's possible that
// the `Op` after `UpdateDelete` is not `UpdateInsert`, if something inconsistent
// happens, we should still take the existing data.
self.take()
} else {
None
}
} else {
// unlimited
None
}
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::array::{Datum, StreamChunkTestExt};
use crate::row::OwnedRow;
#[test]
fn test_stream_chunk_builder() {
let row = OwnedRow::new(vec![Datum::None, Datum::None]);
let mut builder = StreamChunkBuilder::new(3, vec![DataType::Int32, DataType::Int32]);
let res = builder.append_row(Op::Delete, row.clone());
assert!(res.is_none());
let res = builder.append_row(Op::Insert, row.clone());
assert!(res.is_none());
let res = builder.append_row(Op::Insert, row.clone());
assert_eq!(
res,
Some(StreamChunk::from_pretty(
" i i
- . .
+ . .
+ . ."
))
);
let res = builder.take();
assert!(res.is_none());
let res = builder.append_row_invisible(Op::Delete, row.clone());
assert!(res.is_none());
let res = builder.append_iter(Op::Delete, row.clone().iter().enumerate());
assert!(res.is_none());
let res = builder.append_record(Record::Insert {
new_row: row.clone(),
});
assert_eq!(
res,
Some(StreamChunk::from_pretty(
" i i
- . . D
- . .
+ . ."
))
);
let res = builder.append_row(Op::UpdateDelete, row.clone());
assert!(res.is_none());
let res = builder.append_row(Op::UpdateInsert, row.clone());
assert!(res.is_none());
let res = builder.append_record(Record::Update {
old_row: row.clone(),
new_row: row.clone(),
});
assert_eq!(
res,
Some(StreamChunk::from_pretty(
" i i
U- . .
U+ . .
U- . .
U+ . ."
))
);
let res = builder.take();
assert!(res.is_none());
}
#[test]
fn test_stream_chunk_builder_with_max_size_1() {
let row = OwnedRow::new(vec![Datum::None, Datum::None]);
let mut builder = StreamChunkBuilder::new(1, vec![DataType::Int32, DataType::Int32]);
let res = builder.append_row(Op::Delete, row.clone());
assert_eq!(
res,
Some(StreamChunk::from_pretty(
" i i
- . ."
))
);
let res = builder.append_row(Op::Insert, row.clone());
assert_eq!(
res,
Some(StreamChunk::from_pretty(
" i i
+ . ."
))
);
let res = builder.append_record(Record::Update {
old_row: row.clone(),
new_row: row.clone(),
});
assert_eq!(
res,
Some(StreamChunk::from_pretty(
" i i
U- . .
U+ . ."
))
);
let res = builder.append_row(Op::UpdateDelete, row.clone());
assert!(res.is_none());
let res = builder.append_row(Op::UpdateDelete, row.clone()); // note this is an inconsistency
assert_eq!(
res,
Some(StreamChunk::from_pretty(
" i i
U- . .
U- . ."
))
);
}
#[test]
fn test_unlimited_stream_chunk_builder() {
let row = OwnedRow::new(vec![Datum::None, Datum::None]);
let mut builder =
StreamChunkBuilder::unlimited(vec![DataType::Int32, DataType::Int32], None);
let res = builder.append_row(Op::Delete, row.clone());
assert!(res.is_none());
let res = builder.append_row(Op::Insert, row.clone());
assert!(res.is_none());
let res = builder.append_row(Op::UpdateDelete, row.clone());
assert!(res.is_none());
let res = builder.append_row(Op::UpdateInsert, row.clone());
assert!(res.is_none());
for _ in 0..2048 {
let res = builder.append_record(Record::Update {
old_row: row.clone(),
new_row: row.clone(),
});
assert!(res.is_none());
}
let res = builder.take();
assert_eq!(res.unwrap().capacity(), 2048 * 2 + 4);
}
}