1use std::future::Future;
16use std::pin::Pin;
17
18use either::Either;
19use futures::stream;
20use futures::stream::select_with_strategy;
21use itertools::Itertools;
22use risingwave_common::array::DataChunk;
23use risingwave_common::bail;
24use risingwave_common::catalog::ColumnDesc;
25use risingwave_connector::parser::{
26 ByteStreamSourceParser, DebeziumParser, DebeziumProps, EncodingProperties, JsonProperties,
27 ProtocolProperties, SourceStreamChunkBuilder, SpecificParserConfig,
28};
29use risingwave_connector::source::cdc::external::{CdcOffset, ExternalTableReaderImpl};
30use risingwave_connector::source::{SourceColumnDesc, SourceContext, SourceCtrlOpts};
31use rw_futures_util::pausable;
32use thiserror_ext::AsReport;
33use tracing::Instrument;
34
35use crate::executor::UpdateMutation;
36use crate::executor::backfill::CdcScanOptions;
37use crate::executor::backfill::cdc::state::CdcBackfillState;
38use crate::executor::backfill::cdc::upstream_table::external::ExternalStorageTable;
39use crate::executor::backfill::cdc::upstream_table::snapshot::{
40 SnapshotReadArgs, UpstreamTableRead, UpstreamTableReader,
41};
42use crate::executor::backfill::utils::{
43 get_cdc_chunk_last_offset, get_new_pos, mapping_chunk, mapping_message, mark_cdc_chunk,
44};
45use crate::executor::monitor::CdcBackfillMetrics;
46use crate::executor::prelude::*;
47use crate::executor::source::get_infinite_backoff_strategy;
48use crate::task::CreateMviewProgressReporter;
49
50const METADATA_STATE_LEN: usize = 4;
52
53pub struct CdcBackfillExecutor<S: StateStore> {
54 actor_ctx: ActorContextRef,
55
56 external_table: ExternalStorageTable,
58
59 upstream: Executor,
61
62 output_indices: Vec<usize>,
64
65 output_columns: Vec<ColumnDesc>,
67
68 state_impl: CdcBackfillState<S>,
70
71 progress: Option<CreateMviewProgressReporter>,
74
75 metrics: CdcBackfillMetrics,
76
77 rate_limit_rps: Option<u32>,
79
80 options: CdcScanOptions,
81}
82
83impl<S: StateStore> CdcBackfillExecutor<S> {
84 #[allow(clippy::too_many_arguments)]
85 pub fn new(
86 actor_ctx: ActorContextRef,
87 external_table: ExternalStorageTable,
88 upstream: Executor,
89 output_indices: Vec<usize>,
90 output_columns: Vec<ColumnDesc>,
91 progress: Option<CreateMviewProgressReporter>,
92 metrics: Arc<StreamingMetrics>,
93 state_table: StateTable<S>,
94 rate_limit_rps: Option<u32>,
95 options: CdcScanOptions,
96 ) -> Self {
97 let pk_indices = external_table.pk_indices();
98 let upstream_table_id = external_table.table_id().table_id;
99 let state_impl = CdcBackfillState::new(
100 upstream_table_id,
101 state_table,
102 pk_indices.len() + METADATA_STATE_LEN,
103 );
104
105 let metrics = metrics.new_cdc_backfill_metrics(external_table.table_id(), actor_ctx.id);
106
107 Self {
108 actor_ctx,
109 external_table,
110 upstream,
111 output_indices,
112 output_columns,
113 state_impl,
114 progress,
115 metrics,
116 rate_limit_rps,
117 options,
118 }
119 }
120
121 fn report_metrics(
122 metrics: &CdcBackfillMetrics,
123 snapshot_processed_row_count: u64,
124 upstream_processed_row_count: u64,
125 ) {
126 metrics
127 .cdc_backfill_snapshot_read_row_count
128 .inc_by(snapshot_processed_row_count);
129
130 metrics
131 .cdc_backfill_upstream_output_row_count
132 .inc_by(upstream_processed_row_count);
133 }
134
135 #[try_stream(ok = Message, error = StreamExecutorError)]
136 async fn execute_inner(mut self) {
137 let pk_indices = self.external_table.pk_indices().to_vec();
139 let pk_order = self.external_table.pk_order_types().to_vec();
140
141 let table_id = self.external_table.table_id().table_id;
142 let upstream_table_name = self.external_table.qualified_table_name();
143 let schema_table_name = self.external_table.schema_table_name().clone();
144 let external_database_name = self.external_table.database_name().to_owned();
145
146 let additional_columns = self
147 .output_columns
148 .iter()
149 .filter(|col| col.additional_column.column_type.is_some())
150 .cloned()
151 .collect_vec();
152
153 let mut upstream = self.upstream.execute();
154
155 let mut current_pk_pos: Option<OwnedRow>;
158
159 let first_barrier = expect_first_barrier(&mut upstream).await?;
161
162 let mut is_snapshot_paused = first_barrier.is_pause_on_startup();
163 let first_barrier_epoch = first_barrier.epoch;
164 yield Message::Barrier(first_barrier);
166 let mut rate_limit_to_zero = self.rate_limit_rps.is_some_and(|val| val == 0);
167
168 let mut state_impl = self.state_impl;
171
172 state_impl.init_epoch(first_barrier_epoch).await?;
173
174 let state = state_impl.restore_state().await?;
176 current_pk_pos = state.current_pk_pos.clone();
177
178 let need_backfill = !self.options.disable_backfill && !state.is_finished;
179
180 let mut total_snapshot_row_count = state.row_count as u64;
182
183 let mut table_reader: Option<ExternalTableReaderImpl> = None;
188 let external_table = self.external_table.clone();
189 let mut future = Box::pin(async move {
190 let backoff = get_infinite_backoff_strategy();
191 tokio_retry::Retry::spawn(backoff, || async {
192 match external_table.create_table_reader().await {
193 Ok(reader) => Ok(reader),
194 Err(e) => {
195 tracing::warn!(error = %e.as_report(), "failed to create cdc table reader, retrying...");
196 Err(e)
197 }
198 }
199 })
200 .instrument(tracing::info_span!("create_cdc_table_reader_with_retry"))
201 .await
202 .expect("Retry create cdc table reader until success.")
203 });
204
205 let mut upstream = transform_upstream(upstream, self.output_columns.clone()).boxed();
207 loop {
208 if let Some(msg) =
209 build_reader_and_poll_upstream(&mut upstream, &mut table_reader, &mut future)
210 .await?
211 {
212 if let Some(msg) = mapping_message(msg, &self.output_indices) {
213 match msg {
214 Message::Barrier(barrier) => {
215 state_impl.commit_state(barrier.epoch).await?;
217 yield Message::Barrier(barrier);
218 }
219 Message::Chunk(chunk) => {
220 if need_backfill {
221 } else {
223 yield Message::Chunk(chunk);
225 }
226 }
227 Message::Watermark(_) => {
228 }
230 }
231 }
232 } else {
233 assert!(table_reader.is_some(), "table reader must created");
234 tracing::info!(
235 table_id,
236 upstream_table_name,
237 "table reader created successfully"
238 );
239 break;
240 }
241 }
242
243 let upstream_table_reader = UpstreamTableReader::new(
244 self.external_table.clone(),
245 table_reader.expect("table reader must created"),
246 );
247
248 let mut upstream = upstream.peekable();
249 let mut last_binlog_offset: Option<CdcOffset> = state
250 .last_cdc_offset
251 .map_or(upstream_table_reader.current_cdc_offset().await?, Some);
252
253 let offset_parse_func = upstream_table_reader.reader.get_cdc_offset_parser();
254 let mut consumed_binlog_offset: Option<CdcOffset> = None;
255
256 tracing::info!(
257 table_id,
258 upstream_table_name,
259 initial_binlog_offset = ?last_binlog_offset,
260 ?current_pk_pos,
261 is_finished = state.is_finished,
262 is_snapshot_paused,
263 snapshot_row_count = total_snapshot_row_count,
264 rate_limit = self.rate_limit_rps,
265 disable_backfill = self.options.disable_backfill,
266 snapshot_interval = self.options.snapshot_interval,
267 snapshot_batch_size = self.options.snapshot_batch_size,
268 "start cdc backfill",
269 );
270
271 if need_backfill {
291 let _ = Pin::new(&mut upstream).peek().await;
294
295 #[for_await]
297 for msg in upstream.by_ref() {
298 match msg? {
299 Message::Barrier(barrier) => {
300 match barrier.mutation.as_deref() {
301 Some(crate::executor::Mutation::Pause) => {
302 is_snapshot_paused = true;
303 tracing::info!(
304 table_id,
305 upstream_table_name,
306 "snapshot is paused by barrier"
307 );
308 }
309 Some(crate::executor::Mutation::Resume) => {
310 is_snapshot_paused = false;
311 tracing::info!(
312 table_id,
313 upstream_table_name,
314 "snapshot is resumed by barrier"
315 );
316 }
317 _ => {
318 }
320 }
321 state_impl.commit_state(barrier.epoch).await?;
323 yield Message::Barrier(barrier);
324 break;
325 }
326 Message::Chunk(ref chunk) => {
327 last_binlog_offset = get_cdc_chunk_last_offset(&offset_parse_func, chunk)?;
328 }
329 Message::Watermark(_) => {
330 }
332 }
333 }
334
335 tracing::info!(table_id,
336 upstream_table_name,
337 initial_binlog_offset = ?last_binlog_offset,
338 ?current_pk_pos,
339 is_snapshot_paused,
340 "start cdc backfill loop");
341
342 let mut upstream_chunk_buffer: Vec<StreamChunk> = vec![];
344
345 'backfill_loop: loop {
346 let left_upstream = upstream.by_ref().map(Either::Left);
347
348 let mut snapshot_read_row_cnt: usize = 0;
349 let read_args = SnapshotReadArgs::new(
350 current_pk_pos.clone(),
351 self.rate_limit_rps,
352 pk_indices.clone(),
353 additional_columns.clone(),
354 schema_table_name.clone(),
355 external_database_name.clone(),
356 );
357
358 let right_snapshot = pin!(
359 upstream_table_reader
360 .snapshot_read_full_table(read_args, self.options.snapshot_batch_size)
361 .map(Either::Right)
362 );
363
364 let (right_snapshot, snapshot_valve) = pausable(right_snapshot);
365 if is_snapshot_paused {
366 snapshot_valve.pause();
367 }
368
369 let mut backfill_stream =
371 select_with_strategy(left_upstream, right_snapshot, |_: &mut ()| {
372 stream::PollNext::Left
373 });
374
375 let mut cur_barrier_snapshot_processed_rows: u64 = 0;
376 let mut cur_barrier_upstream_processed_rows: u64 = 0;
377 let mut barrier_count: u32 = 0;
378 let mut pending_barrier = None;
379
380 #[for_await]
381 for either in &mut backfill_stream {
382 match either {
383 Either::Left(msg) => {
385 match msg? {
386 Message::Barrier(barrier) => {
387 barrier_count += 1;
389 let can_start_new_snapshot =
390 barrier_count == self.options.snapshot_interval;
391
392 if let Some(mutation) = barrier.mutation.as_deref() {
393 use crate::executor::Mutation;
394 match mutation {
395 Mutation::Pause => {
396 is_snapshot_paused = true;
397 snapshot_valve.pause();
398 }
399 Mutation::Resume => {
400 is_snapshot_paused = false;
401 snapshot_valve.resume();
402 }
403 Mutation::Throttle(some) => {
404 if let Some(new_rate_limit) =
405 some.get(&self.actor_ctx.id)
406 && *new_rate_limit != self.rate_limit_rps
407 {
408 self.rate_limit_rps = *new_rate_limit;
409 rate_limit_to_zero = self
410 .rate_limit_rps
411 .is_some_and(|val| val == 0);
412
413 state_impl
415 .mutate_state(
416 current_pk_pos.clone(),
417 last_binlog_offset.clone(),
418 total_snapshot_row_count,
419 false,
420 )
421 .await?;
422 state_impl.commit_state(barrier.epoch).await?;
423 yield Message::Barrier(barrier);
424
425 continue 'backfill_loop;
427 }
428 }
429 Mutation::Update(UpdateMutation {
430 dropped_actors,
431 ..
432 }) => {
433 if dropped_actors.contains(&self.actor_ctx.id) {
434 tracing::info!(
436 table_id,
437 upstream_table_name,
438 "CdcBackfill has been dropped due to config change"
439 );
440 yield Message::Barrier(barrier);
441 break 'backfill_loop;
442 }
443 }
444 _ => (),
445 }
446 }
447
448 Self::report_metrics(
449 &self.metrics,
450 cur_barrier_snapshot_processed_rows,
451 cur_barrier_upstream_processed_rows,
452 );
453
454 if can_start_new_snapshot {
457 pending_barrier = Some(barrier);
459 tracing::debug!(
460 table_id,
461 ?current_pk_pos,
462 ?snapshot_read_row_cnt,
463 "Prepare to start a new snapshot"
464 );
465 break;
467 } else {
468 state_impl
470 .mutate_state(
471 current_pk_pos.clone(),
472 last_binlog_offset.clone(),
473 total_snapshot_row_count,
474 false,
475 )
476 .await?;
477
478 state_impl.commit_state(barrier.epoch).await?;
479
480 yield Message::Barrier(barrier);
482 }
483 }
484 Message::Chunk(chunk) => {
485 if chunk.cardinality() == 0 {
487 continue;
488 }
489
490 let chunk_binlog_offset =
491 get_cdc_chunk_last_offset(&offset_parse_func, &chunk)?;
492
493 tracing::trace!(
494 "recv changelog chunk: chunk_offset {:?}, capactiy {}",
495 chunk_binlog_offset,
496 chunk.capacity()
497 );
498
499 if let Some(last_binlog_offset) = last_binlog_offset.as_ref() {
503 if let Some(chunk_offset) = chunk_binlog_offset
504 && chunk_offset < *last_binlog_offset
505 {
506 tracing::trace!(
507 "skip changelog chunk: chunk_offset {:?}, capacity {}",
508 chunk_offset,
509 chunk.capacity()
510 );
511 continue;
512 }
513 }
514 upstream_chunk_buffer.push(chunk.compact());
516 }
517 Message::Watermark(_) => {
518 }
520 }
521 }
522 Either::Right(msg) => {
524 match msg? {
525 None => {
526 tracing::info!(
527 table_id,
528 ?last_binlog_offset,
529 ?current_pk_pos,
530 "snapshot read stream ends"
531 );
532 for chunk in upstream_chunk_buffer.drain(..) {
537 yield Message::Chunk(mapping_chunk(
538 chunk,
539 &self.output_indices,
540 ));
541 }
542
543 break 'backfill_loop;
545 }
546 Some(chunk) => {
547 current_pk_pos = Some(get_new_pos(&chunk, &pk_indices));
551
552 tracing::trace!(
553 "got a snapshot chunk: len {}, current_pk_pos {:?}",
554 chunk.cardinality(),
555 current_pk_pos
556 );
557 let chunk_cardinality = chunk.cardinality() as u64;
558 cur_barrier_snapshot_processed_rows += chunk_cardinality;
559 total_snapshot_row_count += chunk_cardinality;
560 yield Message::Chunk(mapping_chunk(
561 chunk,
562 &self.output_indices,
563 ));
564 }
565 }
566 }
567 }
568 }
569
570 assert!(pending_barrier.is_some(), "pending_barrier must exist");
571 let pending_barrier = pending_barrier.unwrap();
572
573 let (_, mut snapshot_stream) = backfill_stream.into_inner();
578
579 if !is_snapshot_paused
581 && !rate_limit_to_zero
582 && let Some(msg) = snapshot_stream
583 .next()
584 .instrument_await("consume_snapshot_stream_once")
585 .await
586 {
587 let Either::Right(msg) = msg else {
588 bail!("BUG: snapshot_read contains upstream messages");
589 };
590 match msg? {
591 None => {
592 tracing::info!(
593 table_id,
594 ?last_binlog_offset,
595 ?current_pk_pos,
596 "snapshot read stream ends in the force emit branch"
597 );
598 for chunk in upstream_chunk_buffer.drain(..) {
601 yield Message::Chunk(mapping_chunk(chunk, &self.output_indices));
602 }
603
604 state_impl
606 .mutate_state(
607 current_pk_pos.clone(),
608 last_binlog_offset.clone(),
609 total_snapshot_row_count,
610 true,
611 )
612 .await?;
613
614 state_impl.commit_state(pending_barrier.epoch).await?;
616 yield Message::Barrier(pending_barrier);
617 break 'backfill_loop;
619 }
620 Some(chunk) => {
621 current_pk_pos = Some(get_new_pos(&chunk, &pk_indices));
623
624 let row_count = chunk.cardinality() as u64;
625 cur_barrier_snapshot_processed_rows += row_count;
626 total_snapshot_row_count += row_count;
627 snapshot_read_row_cnt += row_count as usize;
628
629 tracing::debug!(
630 table_id,
631 ?current_pk_pos,
632 ?snapshot_read_row_cnt,
633 "force emit a snapshot chunk"
634 );
635 yield Message::Chunk(mapping_chunk(chunk, &self.output_indices));
636 }
637 }
638 }
639
640 if let Some(current_pos) = ¤t_pk_pos {
643 for chunk in upstream_chunk_buffer.drain(..) {
644 cur_barrier_upstream_processed_rows += chunk.cardinality() as u64;
645
646 consumed_binlog_offset =
649 get_cdc_chunk_last_offset(&offset_parse_func, &chunk)?;
650
651 yield Message::Chunk(mapping_chunk(
652 mark_cdc_chunk(
653 &offset_parse_func,
654 chunk,
655 current_pos,
656 &pk_indices,
657 &pk_order,
658 last_binlog_offset.clone(),
659 )?,
660 &self.output_indices,
661 ));
662 }
663 } else {
664 upstream_chunk_buffer.clear();
667 }
668
669 if consumed_binlog_offset.is_some() {
671 last_binlog_offset.clone_from(&consumed_binlog_offset);
672 }
673
674 Self::report_metrics(
675 &self.metrics,
676 cur_barrier_snapshot_processed_rows,
677 cur_barrier_upstream_processed_rows,
678 );
679
680 state_impl
682 .mutate_state(
683 current_pk_pos.clone(),
684 last_binlog_offset.clone(),
685 total_snapshot_row_count,
686 false,
687 )
688 .await?;
689
690 state_impl.commit_state(pending_barrier.epoch).await?;
691 yield Message::Barrier(pending_barrier);
692 }
693 } else if self.options.disable_backfill {
694 tracing::info!(
696 table_id,
697 upstream_table_name,
698 "CdcBackfill has been disabled"
699 );
700 state_impl
701 .mutate_state(
702 current_pk_pos.clone(),
703 last_binlog_offset.clone(),
704 total_snapshot_row_count,
705 true,
706 )
707 .await?;
708 }
709
710 drop(upstream_table_reader);
712
713 tracing::info!(
714 table_id,
715 upstream_table_name,
716 "CdcBackfill has already finished and will forward messages directly to the downstream"
717 );
718
719 while let Some(Ok(msg)) = upstream.next().await {
722 if let Some(msg) = mapping_message(msg, &self.output_indices) {
723 if let Message::Barrier(barrier) = &msg {
725 state_impl
728 .mutate_state(
729 current_pk_pos.clone(),
730 last_binlog_offset.clone(),
731 total_snapshot_row_count,
732 true,
733 )
734 .await?;
735 state_impl.commit_state(barrier.epoch).await?;
736
737 if let Some(progress) = self.progress.as_mut() {
739 progress.finish(barrier.epoch, total_snapshot_row_count);
740 }
741 yield msg;
742 break;
744 }
745 yield msg;
746 }
747 }
748
749 #[for_await]
753 for msg in upstream {
754 if let Some(msg) = mapping_message(msg?, &self.output_indices) {
757 if let Message::Barrier(barrier) = &msg {
758 state_impl.commit_state(barrier.epoch).await?;
760 }
761 yield msg;
762 }
763 }
764 }
765}
766
767async fn build_reader_and_poll_upstream(
768 upstream: &mut BoxedMessageStream,
769 table_reader: &mut Option<ExternalTableReaderImpl>,
770 future: &mut Pin<Box<impl Future<Output = ExternalTableReaderImpl>>>,
771) -> StreamExecutorResult<Option<Message>> {
772 if table_reader.is_some() {
773 return Ok(None);
774 }
775 tokio::select! {
776 biased;
777 reader = &mut *future => {
778 *table_reader = Some(reader);
779 Ok(None)
780 }
781 msg = upstream.next() => {
782 msg.transpose()
783 }
784 }
785}
786
787#[try_stream(ok = Message, error = StreamExecutorError)]
788pub async fn transform_upstream(upstream: BoxedMessageStream, output_columns: Vec<ColumnDesc>) {
789 let props = SpecificParserConfig {
790 encoding_config: EncodingProperties::Json(JsonProperties {
791 use_schema_registry: false,
792 timestamptz_handling: None,
793 }),
794 protocol_config: ProtocolProperties::Debezium(DebeziumProps::default()),
796 };
797
798 let columns_with_meta = output_columns
800 .iter()
801 .map(SourceColumnDesc::from)
802 .collect_vec();
803
804 let mut parser = DebeziumParser::new(
805 props,
806 columns_with_meta.clone(),
807 Arc::new(SourceContext::dummy()),
808 )
809 .await
810 .map_err(StreamExecutorError::connector_error)?;
811
812 pin_mut!(upstream);
813 #[for_await]
814 for msg in upstream {
815 let mut msg = msg?;
816 if let Message::Chunk(chunk) = &mut msg {
817 let parsed_chunk = parse_debezium_chunk(&mut parser, chunk).await?;
818 let _ = std::mem::replace(chunk, parsed_chunk);
819 }
820 yield msg;
821 }
822}
823
824async fn parse_debezium_chunk(
825 parser: &mut DebeziumParser,
826 chunk: &StreamChunk,
827) -> StreamExecutorResult<StreamChunk> {
828 let mut builder = SourceStreamChunkBuilder::new(
835 parser.columns().to_vec(),
836 SourceCtrlOpts {
837 chunk_size: chunk.capacity(),
838 split_txn: false,
839 },
840 );
841
842 let payloads = chunk.data_chunk().project(&[0]);
846 let offsets = chunk.data_chunk().project(&[1]).compact();
847
848 for payload in payloads.rows() {
850 let ScalarRefImpl::Jsonb(jsonb_ref) = payload.datum_at(0).expect("payload must exist")
851 else {
852 panic!("payload must be jsonb");
853 };
854
855 parser
856 .parse_inner(
857 None,
858 Some(jsonb_ref.to_string().as_bytes().to_vec()),
859 builder.row_writer(),
860 )
861 .await
862 .unwrap();
863 }
864 builder.finish_current_chunk();
865
866 let parsed_chunk = {
867 let mut iter = builder.consume_ready_chunks();
868 assert_eq!(1, iter.len());
869 iter.next().unwrap()
870 };
871 assert_eq!(parsed_chunk.capacity(), chunk.capacity()); let (ops, mut columns, vis) = parsed_chunk.into_inner();
873 columns.extend(offsets.into_parts().0);
877
878 Ok(StreamChunk::from_parts(
879 ops,
880 DataChunk::from_parts(columns.into(), vis),
881 ))
882}
883
884impl<S: StateStore> Execute for CdcBackfillExecutor<S> {
885 fn execute(self: Box<Self>) -> BoxedMessageStream {
886 self.execute_inner().boxed()
887 }
888}
889
890#[cfg(test)]
891mod tests {
892 use std::str::FromStr;
893
894 use futures::{StreamExt, pin_mut};
895 use risingwave_common::array::{Array, DataChunk, Op, StreamChunk};
896 use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, Schema};
897 use risingwave_common::types::{DataType, Datum, JsonbVal};
898 use risingwave_common::util::epoch::test_epoch;
899 use risingwave_common::util::iter_util::ZipEqFast;
900 use risingwave_storage::memory::MemoryStateStore;
901
902 use crate::executor::backfill::cdc::cdc_backfill::transform_upstream;
903 use crate::executor::monitor::StreamingMetrics;
904 use crate::executor::prelude::StateTable;
905 use crate::executor::source::default_source_internal_table;
906 use crate::executor::test_utils::MockSource;
907 use crate::executor::{
908 ActorContext, Barrier, CdcBackfillExecutor, CdcScanOptions, ExternalStorageTable, Message,
909 };
910
911 #[tokio::test]
912 async fn test_transform_upstream_chunk() {
913 let schema = Schema::new(vec![
914 Field::unnamed(DataType::Jsonb), Field::unnamed(DataType::Varchar), Field::unnamed(DataType::Varchar), ]);
918 let pk_indices = vec![1];
919 let (mut tx, source) = MockSource::channel();
920 let source = source.into_executor(schema.clone(), pk_indices.clone());
921 let payload = r#"{ "payload": { "before": null, "after": { "O_ORDERKEY": 5, "O_CUSTKEY": 44485, "O_ORDERSTATUS": "F", "O_TOTALPRICE": "144659.20", "O_ORDERDATE": "1994-07-30" }, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002", "ts_ms": 1695277757000, "snapshot": "last", "db": "mydb", "sequence": null, "table": "orders_new", "server_id": 0, "gtid": null, "file": "binlog.000008", "pos": 3693, "row": 0, "thread": null, "query": null }, "op": "r", "ts_ms": 1695277757017, "transaction": null } }"#;
923
924 let datums: Vec<Datum> = vec![
925 Some(JsonbVal::from_str(payload).unwrap().into()),
926 Some("file: 1.binlog, pos: 100".to_owned().into()),
927 Some("mydb.orders".to_owned().into()),
928 ];
929
930 println!("datums: {:?}", datums[1]);
931
932 let mut builders = schema.create_array_builders(8);
933 for (builder, datum) in builders.iter_mut().zip_eq_fast(datums.iter()) {
934 builder.append(datum.clone());
935 }
936 let columns = builders
937 .into_iter()
938 .map(|builder| builder.finish().into())
939 .collect();
940
941 let chunk = StreamChunk::from_parts(vec![Op::Insert], DataChunk::new(columns, 1));
943
944 tx.push_chunk(chunk);
945 let upstream = Box::new(source).execute();
946
947 let columns = vec![
949 ColumnDesc::named("O_ORDERKEY", ColumnId::new(1), DataType::Int64),
950 ColumnDesc::named("O_CUSTKEY", ColumnId::new(2), DataType::Int64),
951 ColumnDesc::named("O_ORDERSTATUS", ColumnId::new(3), DataType::Varchar),
952 ColumnDesc::named("O_TOTALPRICE", ColumnId::new(4), DataType::Decimal),
953 ColumnDesc::named("O_ORDERDATE", ColumnId::new(5), DataType::Date),
954 ColumnDesc::named("commit_ts", ColumnId::new(6), DataType::Timestamptz),
955 ];
956
957 let parsed_stream = transform_upstream(upstream, columns);
958 pin_mut!(parsed_stream);
959 if let Some(message) = parsed_stream.next().await {
961 println!("chunk: {:#?}", message.unwrap());
962 }
963 }
964
965 #[tokio::test]
966 async fn test_build_reader_and_poll_upstream() {
967 let actor_context = ActorContext::for_test(1);
968 let external_storage_table = ExternalStorageTable::for_test_undefined();
969 let schema = Schema::new(vec![
970 Field::unnamed(DataType::Jsonb), Field::unnamed(DataType::Varchar), Field::unnamed(DataType::Varchar), ]);
974 let pk_indices = vec![1];
975 let (mut tx, source) = MockSource::channel();
976 let source = source.into_executor(schema.clone(), pk_indices.clone());
977 let output_indices = vec![1, 0, 4]; let output_columns = vec![
979 ColumnDesc::named("O_ORDERKEY", ColumnId::new(1), DataType::Int64),
980 ColumnDesc::named("O_CUSTKEY", ColumnId::new(2), DataType::Int64),
981 ColumnDesc::named("O_ORDERSTATUS", ColumnId::new(3), DataType::Varchar),
982 ColumnDesc::named("O_TOTALPRICE", ColumnId::new(4), DataType::Decimal),
983 ColumnDesc::named("O_DUMMY", ColumnId::new(5), DataType::Int64),
984 ColumnDesc::named("commit_ts", ColumnId::new(6), DataType::Timestamptz),
985 ];
986 let store = MemoryStateStore::new();
987 let state_table =
988 StateTable::from_table_catalog(&default_source_internal_table(0x2333), store, None)
989 .await;
990 let cdc = CdcBackfillExecutor::new(
991 actor_context,
992 external_storage_table,
993 source,
994 output_indices,
995 output_columns,
996 None,
997 StreamingMetrics::unused().into(),
998 state_table,
999 None,
1000 CdcScanOptions {
1001 disable_backfill: true,
1004 ..CdcScanOptions::default()
1005 },
1006 );
1007 let s = cdc.execute_inner();
1011 pin_mut!(s);
1012
1013 tx.send_barrier(Barrier::new_test_barrier(test_epoch(8)));
1015 {
1017 let payload = r#"{ "payload": { "before": null, "after": { "O_ORDERKEY": 5, "O_CUSTKEY": 44485, "O_ORDERSTATUS": "F", "O_TOTALPRICE": "144659.20", "O_DUMMY": 100 }, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002", "ts_ms": 1695277757000, "snapshot": "last", "db": "mydb", "sequence": null, "table": "orders_new", "server_id": 0, "gtid": null, "file": "binlog.000008", "pos": 3693, "row": 0, "thread": null, "query": null }, "op": "r", "ts_ms": 1695277757017, "transaction": null } }"#;
1018 let datums: Vec<Datum> = vec![
1019 Some(JsonbVal::from_str(payload).unwrap().into()),
1020 Some("file: 1.binlog, pos: 100".to_owned().into()),
1021 Some("mydb.orders".to_owned().into()),
1022 ];
1023 let mut builders = schema.create_array_builders(8);
1024 for (builder, datum) in builders.iter_mut().zip_eq_fast(datums.iter()) {
1025 builder.append(datum.clone());
1026 }
1027 let columns = builders
1028 .into_iter()
1029 .map(|builder| builder.finish().into())
1030 .collect();
1031 let chunk = StreamChunk::from_parts(vec![Op::Insert], DataChunk::new(columns, 1));
1033
1034 tx.push_chunk(chunk);
1035 }
1036 let _first_barrier = s.next().await.unwrap();
1037 let upstream_change_log = s.next().await.unwrap().unwrap();
1038 let Message::Chunk(chunk) = upstream_change_log else {
1039 panic!("expect chunk");
1040 };
1041 assert_eq!(chunk.columns().len(), 3);
1042 assert_eq!(chunk.rows().count(), 1);
1043 assert_eq!(
1044 chunk.columns()[0].as_int64().iter().collect::<Vec<_>>(),
1045 vec![Some(44485)]
1046 );
1047 assert_eq!(
1048 chunk.columns()[1].as_int64().iter().collect::<Vec<_>>(),
1049 vec![Some(5)]
1050 );
1051 assert_eq!(
1052 chunk.columns()[2].as_int64().iter().collect::<Vec<_>>(),
1053 vec![Some(100)]
1054 );
1055 }
1056}