risingwave_connector/parser/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 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513 514 515 516 517 518 519 520 521 522 523 524 525 526 527 528 529 530 531 532 533 534 535 536 537 538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566 567 568 569 570 571 572
// Copyright 2025 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 std::sync::LazyLock;
use risingwave_common::array::stream_record::RecordType;
use risingwave_common::array::{ArrayBuilderImpl, Op, StreamChunk};
use risingwave_common::bitmap::BitmapBuilder;
use risingwave_common::log::LogSuppresser;
use risingwave_common::types::{Datum, DatumCow, ScalarRefImpl};
use risingwave_common::util::iter_util::ZipEqFast;
use risingwave_connector_codec::decoder::{AccessError, AccessResult};
use risingwave_pb::plan_common::additional_column::ColumnType as AdditionalColumnType;
use smallvec::SmallVec;
use thiserror_ext::AsReport;
use super::MessageMeta;
use crate::parser::utils::{
extract_cdc_meta_column, extract_header_inner_from_meta, extract_headers_from_meta,
extract_subject_from_meta, extract_timestamp_from_meta,
};
use crate::source::{SourceColumnDesc, SourceColumnType, SourceCtrlOpts, SourceMeta};
/// Maximum number of rows in a transaction. If a transaction is larger than this, it will be force
/// committed to avoid potential OOM.
const MAX_TRANSACTION_SIZE: usize = 4096;
/// Represents an ongoing transaction.
struct Transaction {
id: Box<str>,
len: usize,
}
/// A builder for building a [`StreamChunk`] from [`SourceColumnDesc`].
///
/// Output chunk size is controlled by `source_ctrl_opts.chunk_size` and `source_ctrl_opts.split_txn`.
/// During building process, it's possible that multiple chunks are built even without any explicit
/// call to `finish_current_chunk`. This mainly happens when we find more than one records in one
/// `SourceMessage` when parsing it. User of this builder should call `consume_ready_chunks` to consume
/// the built chunks from time to time, to avoid the buffer from growing too large.
pub struct SourceStreamChunkBuilder {
column_descs: Vec<SourceColumnDesc>,
source_ctrl_opts: SourceCtrlOpts,
builders: Vec<ArrayBuilderImpl>,
op_builder: Vec<Op>,
vis_builder: BitmapBuilder,
ongoing_txn: Option<Transaction>,
ready_chunks: SmallVec<[StreamChunk; 1]>,
}
impl SourceStreamChunkBuilder {
pub fn new(column_descs: Vec<SourceColumnDesc>, source_ctrl_opts: SourceCtrlOpts) -> Self {
let (builders, op_builder, vis_builder) =
Self::create_builders(&column_descs, source_ctrl_opts.chunk_size);
Self {
column_descs,
source_ctrl_opts,
builders,
op_builder,
vis_builder,
ongoing_txn: None,
ready_chunks: SmallVec::new(),
}
}
fn create_builders(
column_descs: &[SourceColumnDesc],
chunk_size: usize,
) -> (Vec<ArrayBuilderImpl>, Vec<Op>, BitmapBuilder) {
let reserved_capacity = chunk_size + 1; // it's possible to have an additional `U-` at the end
let builders = column_descs
.iter()
.map(|desc| desc.data_type.create_array_builder(reserved_capacity))
.collect();
let op_builder = Vec::with_capacity(reserved_capacity);
let vis_builder = BitmapBuilder::with_capacity(reserved_capacity);
(builders, op_builder, vis_builder)
}
/// Begin a (CDC) transaction with the given `txn_id`.
pub fn begin_transaction(&mut self, txn_id: Box<str>) {
if let Some(ref txn) = self.ongoing_txn {
tracing::warn!(
ongoing_txn_id = txn.id,
new_txn_id = txn_id,
"already in a transaction"
);
}
tracing::debug!(txn_id, "begin upstream transaction");
self.ongoing_txn = Some(Transaction { id: txn_id, len: 0 });
}
/// Commit the ongoing transaction with the given `txn_id`.
pub fn commit_transaction(&mut self, txn_id: Box<str>) {
if let Some(txn) = self.ongoing_txn.take() {
if txn.id != txn_id {
tracing::warn!(
expected_txn_id = txn.id,
actual_txn_id = txn_id,
"unexpected transaction id"
);
}
tracing::debug!(txn_id, "commit upstream transaction");
if self.current_chunk_len() >= self.source_ctrl_opts.chunk_size {
// if `split_txn` is on, we should've finished the chunk already
assert!(!self.source_ctrl_opts.split_txn);
self.finish_current_chunk();
}
} else {
tracing::warn!(txn_id, "no ongoing transaction to commit");
}
}
/// Check if the builder is in an ongoing transaction.
pub fn is_in_transaction(&self) -> bool {
self.ongoing_txn.is_some()
}
/// Get a row writer for parser to write records to the builder.
pub fn row_writer(&mut self) -> SourceStreamChunkRowWriter<'_> {
SourceStreamChunkRowWriter {
builder: self,
visible: true, // write visible rows by default
row_meta: None,
}
}
/// Write a heartbeat record to the builder. The builder will decide whether to finish the
/// current chunk or not. Currently it ensures that heartbeats are always in separate chunks.
pub fn heartbeat(&mut self, meta: MessageMeta<'_>) {
if self.current_chunk_len() > 0 {
// If there are records in the chunk, finish it first.
// If there's an ongoing transaction, `finish_current_chunk` will handle it properly.
// Note this
self.finish_current_chunk();
}
_ = self
.row_writer()
.invisible()
.with_meta(meta)
.do_insert(|_| Ok(Datum::None));
self.finish_current_chunk(); // each heartbeat should be a separate chunk
}
/// Finish and build a [`StreamChunk`] from the current pending records in the builder,
/// no matter whether the builder is in a transaction or not, `split_txn` or not. The
/// built chunk will be appended to the `ready_chunks` and the builder will be reset.
pub fn finish_current_chunk(&mut self) {
if self.op_builder.is_empty() {
return;
}
let (builders, op_builder, vis_builder) =
Self::create_builders(&self.column_descs, self.source_ctrl_opts.chunk_size);
let chunk = StreamChunk::with_visibility(
std::mem::replace(&mut self.op_builder, op_builder),
std::mem::replace(&mut self.builders, builders)
.into_iter()
.map(|builder| builder.finish().into())
.collect(),
std::mem::replace(&mut self.vis_builder, vis_builder).finish(),
);
self.ready_chunks.push(chunk);
if let Some(ref mut txn) = self.ongoing_txn {
tracing::warn!(
txn_id = txn.id,
len = txn.len,
"splitting an ongoing transaction"
);
txn.len = 0;
}
}
/// Consumes and returns the ready [`StreamChunk`]s.
pub fn consume_ready_chunks(&mut self) -> impl ExactSizeIterator<Item = StreamChunk> + '_ {
self.ready_chunks.drain(..)
}
fn current_chunk_len(&self) -> usize {
self.op_builder.len()
}
/// Commit a newly-written record by appending `op` and `vis` to the corresponding builders.
/// This is supposed to be called via the `row_writer` only.
fn commit_record(&mut self, op: Op, vis: bool) {
self.op_builder.push(op);
self.vis_builder.append(vis);
let curr_chunk_size = self.current_chunk_len();
let max_chunk_size = self.source_ctrl_opts.chunk_size;
if let Some(ref mut txn) = self.ongoing_txn {
txn.len += 1;
if txn.len >= MAX_TRANSACTION_SIZE
|| (self.source_ctrl_opts.split_txn && curr_chunk_size >= max_chunk_size)
{
self.finish_current_chunk();
}
} else if curr_chunk_size >= max_chunk_size {
self.finish_current_chunk();
}
}
}
/// `SourceStreamChunkRowWriter` is responsible to write one or more records to the [`StreamChunk`],
/// where each contains either one row (Insert/Delete) or two rows (Update) that can be written atomically.
///
/// Callers are supposed to call one of the `insert`, `delete` or `update` methods to write a record,
/// providing a closure that produces one or two [`Datum`]s by corresponding [`SourceColumnDesc`].
/// Specifically,
/// - only columns with [`SourceColumnType::Normal`] need to be handled;
/// - errors for non-primary key columns will be ignored and filled with default value instead;
/// - other errors will be propagated.
pub struct SourceStreamChunkRowWriter<'a> {
builder: &'a mut SourceStreamChunkBuilder,
/// Whether the rows written by this writer should be visible in output `StreamChunk`.
visible: bool,
/// An optional meta data of the original message.
///
/// When this is set by `with_meta`, it'll be used to fill the columns of types other than [`SourceColumnType::Normal`].
row_meta: Option<MessageMeta<'a>>,
}
impl<'a> SourceStreamChunkRowWriter<'a> {
/// Set the meta data of the original message for this row writer.
///
/// This should always be called except for tests.
pub fn with_meta(mut self, row_meta: MessageMeta<'a>) -> Self {
self.row_meta = Some(row_meta);
self
}
pub fn row_meta(&self) -> Option<MessageMeta<'a>> {
self.row_meta
}
/// Convert the row writer to invisible row writer.
pub fn invisible(mut self) -> Self {
self.visible = false;
self
}
}
impl SourceStreamChunkRowWriter<'_> {
fn do_action<'a, A: RowWriterAction>(
&'a mut self,
mut f: impl FnMut(&SourceColumnDesc) -> AccessResult<A::Output<'a>>,
) -> AccessResult<()> {
let mut parse_field = |desc: &SourceColumnDesc| {
match f(desc) {
Ok(output) => Ok(output),
// Throw error for failed access to primary key columns.
Err(e) if desc.is_pk => Err(e),
// Ignore error for other columns and fill in `NULL` instead.
Err(error) => {
// TODO: figure out a way to fill in not-null default value if user specifies one
// TODO: decide whether the error should not be ignored (e.g., even not a valid Debezium message)
// TODO: not using tracing span to provide `split_id` and `offset` due to performance concern,
// see #13105
static LOG_SUPPERSSER: LazyLock<LogSuppresser> =
LazyLock::new(LogSuppresser::default);
if let Ok(suppressed_count) = LOG_SUPPERSSER.check() {
tracing::warn!(
error = %error.as_report(),
split_id = self.row_meta.as_ref().map(|m| m.split_id),
offset = self.row_meta.as_ref().map(|m| m.offset),
column = desc.name,
suppressed_count,
"failed to parse non-pk column, padding with `NULL`"
);
}
Ok(A::output_for(Datum::None))
}
}
};
let mut wrapped_f = |desc: &SourceColumnDesc| {
match (&desc.column_type, &desc.additional_column.column_type) {
(&SourceColumnType::Offset | &SourceColumnType::RowId, _) => {
// SourceColumnType is for CDC source only.
Ok(A::output_for(
self.row_meta
.as_ref()
.and_then(|row_meta| row_meta.value_for_column(desc)),
))
}
(&SourceColumnType::Meta, _)
if matches!(
&self.row_meta.map(|ele| ele.source_meta),
&Some(SourceMeta::Kafka(_) | SourceMeta::DebeziumCdc(_))
) =>
{
// SourceColumnType is for CDC source only.
Ok(A::output_for(
self.row_meta
.as_ref()
.and_then(|row_meta| row_meta.value_for_column(desc)),
))
}
(
_, // for cdc tables
&Some(ref col @ AdditionalColumnType::DatabaseName(_))
| &Some(ref col @ AdditionalColumnType::TableName(_)),
) => {
match self.row_meta {
Some(row_meta) => {
if let SourceMeta::DebeziumCdc(cdc_meta) = row_meta.source_meta {
Ok(A::output_for(extract_cdc_meta_column(
cdc_meta,
col,
desc.name.as_str(),
)?))
} else {
Err(AccessError::Uncategorized {
message: "CDC metadata not found in the message".to_owned(),
})
}
}
None => parse_field(desc), // parse from payload
}
}
(_, &Some(AdditionalColumnType::Timestamp(_))) => match self.row_meta {
Some(row_meta) => Ok(A::output_for(extract_timestamp_from_meta(
row_meta.source_meta,
))),
None => parse_field(desc), // parse from payload
},
(_, &Some(AdditionalColumnType::CollectionName(_))) => {
// collection name for `mongodb-cdc` should be parsed from the message payload
parse_field(desc)
}
(_, &Some(AdditionalColumnType::Subject(_))) => Ok(A::output_for(
self.row_meta
.as_ref()
.and_then(|ele| extract_subject_from_meta(ele.source_meta))
.unwrap_or(None),
)),
(_, &Some(AdditionalColumnType::Partition(_))) => {
// the meta info does not involve spec connector
Ok(A::output_for(
self.row_meta
.as_ref()
.map(|ele| ScalarRefImpl::Utf8(ele.split_id)),
))
}
(_, &Some(AdditionalColumnType::Offset(_))) => {
// the meta info does not involve spec connector
Ok(A::output_for(
self.row_meta
.as_ref()
.map(|ele| ScalarRefImpl::Utf8(ele.offset)),
))
}
(_, &Some(AdditionalColumnType::HeaderInner(ref header_inner))) => {
Ok(A::output_for(
self.row_meta
.as_ref()
.and_then(|ele| {
extract_header_inner_from_meta(
ele.source_meta,
header_inner.inner_field.as_ref(),
header_inner.data_type.as_ref(),
)
})
.unwrap_or(Datum::None.into()),
))
}
(_, &Some(AdditionalColumnType::Headers(_))) => Ok(A::output_for(
self.row_meta
.as_ref()
.and_then(|ele| extract_headers_from_meta(ele.source_meta))
.unwrap_or(None),
)),
(_, &Some(AdditionalColumnType::Filename(_))) => {
// Filename is used as partition in FS connectors
Ok(A::output_for(
self.row_meta
.as_ref()
.map(|ele| ScalarRefImpl::Utf8(ele.split_id)),
))
}
(_, &Some(AdditionalColumnType::Payload(_))) => {
// ingest the whole payload as a single column
// do special logic in `KvEvent::access_field`
parse_field(desc)
}
(_, _) => {
// For normal columns, call the user provided closure.
parse_field(desc)
}
}
};
// Columns that changes have been applied to. Used to rollback when an error occurs.
let mut applied_columns = 0;
let result = (self.builder.column_descs.iter())
.zip_eq_fast(self.builder.builders.iter_mut())
.try_for_each(|(desc, builder)| {
wrapped_f(desc).map(|output| {
A::apply(builder, output);
applied_columns += 1;
})
});
match result {
Ok(_) => {
// commit the action by appending `Op`s and visibility
for op in A::RECORD_TYPE.ops() {
self.builder.commit_record(*op, self.visible);
}
Ok(())
}
Err(e) => {
for i in 0..applied_columns {
A::rollback(&mut self.builder.builders[i]);
}
Err(e)
}
}
}
/// Write an `Insert` record to the [`StreamChunk`], with the given fallible closure that
/// produces one [`Datum`] by corresponding [`SourceColumnDesc`].
///
/// See the [struct-level documentation](SourceStreamChunkRowWriter) for more details.
#[inline(always)]
pub fn do_insert<'a, D>(
&mut self,
mut f: impl FnMut(&SourceColumnDesc) -> AccessResult<D>,
) -> AccessResult<()>
where
D: Into<DatumCow<'a>>,
{
self.do_action::<InsertAction>(|desc| f(desc).map(Into::into))
}
/// Write a `Delete` record to the [`StreamChunk`], with the given fallible closure that
/// produces one [`Datum`] by corresponding [`SourceColumnDesc`].
///
/// See the [struct-level documentation](SourceStreamChunkRowWriter) for more details.
#[inline(always)]
pub fn do_delete<'a, D>(
&mut self,
mut f: impl FnMut(&SourceColumnDesc) -> AccessResult<D>,
) -> AccessResult<()>
where
D: Into<DatumCow<'a>>,
{
self.do_action::<DeleteAction>(|desc| f(desc).map(Into::into))
}
/// Write a `Update` record to the [`StreamChunk`], with the given fallible closure that
/// produces two [`Datum`]s as old and new value by corresponding [`SourceColumnDesc`].
///
/// See the [struct-level documentation](SourceStreamChunkRowWriter) for more details.
#[inline(always)]
pub fn do_update<'a, D1, D2>(
&mut self,
mut f: impl FnMut(&SourceColumnDesc) -> AccessResult<(D1, D2)>,
) -> AccessResult<()>
where
D1: Into<DatumCow<'a>>,
D2: Into<DatumCow<'a>>,
{
self.do_action::<UpdateAction>(|desc| f(desc).map(|(old, new)| (old.into(), new.into())))
}
}
trait RowWriterAction {
type Output<'a>;
const RECORD_TYPE: RecordType;
fn output_for<'a>(datum: impl Into<DatumCow<'a>>) -> Self::Output<'a>;
fn apply(builder: &mut ArrayBuilderImpl, output: Self::Output<'_>);
fn rollback(builder: &mut ArrayBuilderImpl);
}
struct InsertAction;
impl RowWriterAction for InsertAction {
type Output<'a> = DatumCow<'a>;
const RECORD_TYPE: RecordType = RecordType::Insert;
#[inline(always)]
fn output_for<'a>(datum: impl Into<DatumCow<'a>>) -> Self::Output<'a> {
datum.into()
}
#[inline(always)]
fn apply(builder: &mut ArrayBuilderImpl, output: DatumCow<'_>) {
builder.append(output)
}
#[inline(always)]
fn rollback(builder: &mut ArrayBuilderImpl) {
builder.pop().unwrap()
}
}
struct DeleteAction;
impl RowWriterAction for DeleteAction {
type Output<'a> = DatumCow<'a>;
const RECORD_TYPE: RecordType = RecordType::Delete;
#[inline(always)]
fn output_for<'a>(datum: impl Into<DatumCow<'a>>) -> Self::Output<'a> {
datum.into()
}
#[inline(always)]
fn apply(builder: &mut ArrayBuilderImpl, output: DatumCow<'_>) {
builder.append(output)
}
#[inline(always)]
fn rollback(builder: &mut ArrayBuilderImpl) {
builder.pop().unwrap()
}
}
struct UpdateAction;
impl RowWriterAction for UpdateAction {
type Output<'a> = (DatumCow<'a>, DatumCow<'a>);
const RECORD_TYPE: RecordType = RecordType::Update;
#[inline(always)]
fn output_for<'a>(datum: impl Into<DatumCow<'a>>) -> Self::Output<'a> {
let datum = datum.into();
(datum.clone(), datum)
}
#[inline(always)]
fn apply(builder: &mut ArrayBuilderImpl, output: (DatumCow<'_>, DatumCow<'_>)) {
builder.append(output.0);
builder.append(output.1);
}
#[inline(always)]
fn rollback(builder: &mut ArrayBuilderImpl) {
builder.pop().unwrap();
builder.pop().unwrap();
}
}