diff --git a/.git-blame-ignore-revs b/.git-blame-ignore-revs index 6efd86227362..b8ca322d767a 100644 --- a/.git-blame-ignore-revs +++ b/.git-blame-ignore-revs @@ -39,3 +39,6 @@ d70dba827c303373f3220c9733f7c7443e5c2d37 # chore: cargo +nightly fmt (#13162) (format let-chains) c583e2c6c054764249acf484438c7bf7197765f4 + +# chore: replace all ProstXxx with PbXxx (#8621) +6fd8821f2e053957b183d648bea9c95b6703941f diff --git a/Makefile.toml b/Makefile.toml index bbf5198825b3..ba11c4910fc7 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -86,9 +86,9 @@ if ${is_not_ci} no_rust_log = not ${rust_log} if ${no_rust_log} - set_env RUST_LOG "pgwire_query_log=info" + set_env RUST_LOG "pgwire_query_log=info,hyper::client::connect::http=info" else - set_env RUST_LOG "pgwire_query_log=info,${rust_log}" + set_env RUST_LOG "pgwire_query_log=info,hyper::client::connect::http=info,${rust_log}" end end diff --git a/e2e_test/batch/catalog/pg_settings.slt.part b/e2e_test/batch/catalog/pg_settings.slt.part index 05fe8fcb443c..d9af757ba4c3 100644 --- a/e2e_test/batch/catalog/pg_settings.slt.part +++ b/e2e_test/batch/catalog/pg_settings.slt.part @@ -35,6 +35,7 @@ user rw_batch_enable_lookup_join user rw_batch_enable_sort_agg user rw_enable_join_ordering user rw_enable_share_plan +user rw_enable_shared_source user rw_enable_two_phase_agg user rw_force_split_distinct_agg user rw_force_two_phase_agg diff --git a/e2e_test/source/README.md b/e2e_test/source/README.md new file mode 100644 index 000000000000..b6e9dfa30816 --- /dev/null +++ b/e2e_test/source/README.md @@ -0,0 +1,7 @@ +Test in this directory needs some prior setup. + +See also `ci/scripts/e2e-source-test.sh`, and `scripts/source` + +## Kafka + +`scripts/source/test_data` contains the data. Filename's convention is `.`. diff --git a/e2e_test/source/basic/kafka_shared_source.slt b/e2e_test/source/basic/kafka_shared_source.slt new file mode 100644 index 000000000000..5245d6ea6863 --- /dev/null +++ b/e2e_test/source/basic/kafka_shared_source.slt @@ -0,0 +1,58 @@ +control substitution on + +statement ok +SET rw_enable_shared_source TO true; + +statement ok +create source s0 (v1 int, v2 varchar) with ( + connector = 'kafka', + topic = 'kafka_4_partition_topic', + properties.bootstrap.server = '${KAFKA_BOOTSTRAP_SERVER:message_queue:29092}', + scan.startup.mode = 'earliest' +) FORMAT PLAIN ENCODE JSON; + +statement ok +create materialized view mv_1 as select * from s0; + +statement ok +SET rw_enable_shared_source TO false; + +statement ok +create materialized view mv_2 as select * from s0; + +statement ok +flush; + +# Wait enough time to ensure SourceExecutor consumes all Kafka data. +sleep 1s + +query IT rowsort +select v1, v2 from s0; +---- +1 1 +2 22 +3 333 +4 4444 + +query IT rowsort +select v1, v2 from mv_1; +---- +1 1 +2 22 +3 333 +4 4444 + +query IT rowsort +select v1, v2 from mv_2; +---- +1 1 +2 22 +3 333 +4 4444 + +# TODO: add more data to the topic and re-check the data. Currently there's no good test infra to do this... +# To test the correctness of source backfill, we might need to keep producing data during an interval, to let it go +# through the backfill stage to the forward stage. + +statement ok +drop source s0 cascade; diff --git a/proto/catalog.proto b/proto/catalog.proto index e0720656cadd..c6d83f51612c 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -62,9 +62,22 @@ message StreamSourceInfo { SchemaRegistryNameStrategy name_strategy = 10; optional string key_message_name = 11; plan_common.ExternalTableDesc external_table = 12; - // Whether the stream source is a cdc source streaming job. - // We need this field to differentiate the cdc source job until we fully implement risingwavelabs/rfcs#72. + // **This field should now be called `is_shared`.** Not renamed for backwards compatibility. + // + // Whether the stream source is a shared source (it has a streaming job). + // This is related with [RFC: Reusable Source Executor](https://github.com/risingwavelabs/rfcs/pull/72). + // + // Currently, the following sources can be shared: + // + // - Direct CDC sources (mysql & postgresql) + // - MQ sources (Kafka) bool cdc_source_job = 13; + // Only used when `cdc_source_job` is `true`. + // If `false`, `requires_singleton` will be set in the stream plan. + // + // - Direct CDC sources: `false` + // - MQ sources (Kafka): `true` + bool is_distributed = 15; // Options specified by user in the FORMAT ENCODE clause. map format_encode_options = 14; } diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 78f4f3c818e4..80c46dd676a1 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -161,7 +161,7 @@ message DropViewResponse { // - GENERAL: Table streaming jobs w/ or w/o a connector // - SHARED_CDC_SOURCE: The table streaming job is created based on a shared CDC source job (risingwavelabs/rfcs#73). // -// And one may add other types to support Table jobs that based on other backfill-able sources (risingwavelabs/rfcs#72). +// And one may add other types to support Table jobs that based on other shared sources (risingwavelabs/rfcs#72). // // Currently, it's usages include: // - When creating the streaming actor graph, different table jobs may need different treatment. diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index a12b1c7171a8..b4393153b57a 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -206,7 +206,7 @@ message StreamFsFetchNode { } message SourceBackfillNode { - uint32 source_id = 1; + uint32 upstream_source_id = 1; optional uint32 row_id_index = 2; repeated plan_common.ColumnCatalog columns = 3; catalog.StreamSourceInfo info = 4; @@ -876,13 +876,14 @@ enum FragmentTypeFlag { FRAGMENT_TYPE_FLAG_MVIEW = 2; FRAGMENT_TYPE_FLAG_SINK = 4; FRAGMENT_TYPE_FLAG_NOW = 8; // TODO: Remove this and insert a `BarrierRecv` instead. + // Include StreamScan and StreamCdcScan FRAGMENT_TYPE_FLAG_STREAM_SCAN = 16; FRAGMENT_TYPE_FLAG_BARRIER_RECV = 32; FRAGMENT_TYPE_FLAG_VALUES = 64; FRAGMENT_TYPE_FLAG_DML = 128; FRAGMENT_TYPE_FLAG_CDC_FILTER = 256; FRAGMENT_TYPE_FLAG_SUBSCRIPTION = 512; - FRAGMENT_TYPE_FLAG_SOURCE_BACKFILL = 1024; + FRAGMENT_TYPE_FLAG_SOURCE_SCAN = 1024; } // The streaming context associated with a stream plan diff --git a/scripts/source/README.md b/scripts/source/README.md new file mode 100644 index 000000000000..4676aa7723b3 --- /dev/null +++ b/scripts/source/README.md @@ -0,0 +1,5 @@ +This folder contains scripts to prepare data for testing sources. + +## Kafka + +`scripts/source/test_data` contains the data. Filename's convention is `.`. diff --git a/src/common/metrics/src/monitor/connection.rs b/src/common/metrics/src/monitor/connection.rs index 13461dde90a2..44b0164a6f62 100644 --- a/src/common/metrics/src/monitor/connection.rs +++ b/src/common/metrics/src/monitor/connection.rs @@ -34,6 +34,7 @@ use prometheus::{ register_int_counter_vec_with_registry, register_int_gauge_vec_with_registry, IntCounter, IntCounterVec, IntGauge, IntGaugeVec, Registry, }; +use thiserror_ext::AsReport; use tokio::io::{AsyncRead, AsyncWrite, ReadBuf}; use tonic::transport::{Channel, Endpoint}; use tracing::{debug, info, warn}; @@ -548,7 +549,7 @@ impl tonic::transport::server::Router { config.tcp_nodelay, config.keepalive_duration, ) - .unwrap(); + .unwrap_or_else(|err| panic!("failed to connect to {listen_addr}: {}", err.as_report())); let incoming = MonitoredConnection::new( incoming, MonitorNewConnectionImpl { diff --git a/src/common/src/lib.rs b/src/common/src/lib.rs index d040245236e2..8bfd70f6248e 100644 --- a/src/common/src/lib.rs +++ b/src/common/src/lib.rs @@ -43,6 +43,8 @@ #![feature(negative_impls)] #![feature(bound_map)] #![feature(array_methods)] +#![feature(register_tool)] +#![register_tool(rw)] #[cfg_attr(not(test), allow(unused_extern_crates))] extern crate self as risingwave_common; diff --git a/src/common/src/session_config/mod.rs b/src/common/src/session_config/mod.rs index 887f70746559..bc8bb0b1506a 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -243,6 +243,13 @@ pub struct ConfigMap { #[parameter(default = false)] background_ddl: bool, + /// Enable shared source. Currently only for Kafka. + /// + /// When enabled, `CREATE SOURCE` will create a source streaming job, and `CREATE MATERIALIZED VIEWS` from the source + /// will forward the data from the same source streaming job, and also backfill prior data from the external source. + #[parameter(default = false)] + rw_enable_shared_source: bool, + /// Shows the server-side character set encoding. At present, this parameter can be shown but not set, because the encoding is determined at database creation time. #[parameter(default = SERVER_ENCODING)] server_encoding: String, diff --git a/src/common/src/system_param/mod.rs b/src/common/src/system_param/mod.rs index 19c36baf09c6..06d4cce2e4e6 100644 --- a/src/common/src/system_param/mod.rs +++ b/src/common/src/system_param/mod.rs @@ -317,7 +317,8 @@ macro_rules! impl_set_system_param { $( key_of!($field) => { let v = if let Some(v) = value { - v.as_ref().parse().map_err(|_| format!("cannot parse parameter value"))? + #[allow(rw::format_error)] + v.as_ref().parse().map_err(|e| format!("cannot parse parameter value: {e}"))? } else { $default.ok_or_else(|| format!("{} does not have a default value", key))? }; diff --git a/src/common/src/util/stream_graph_visitor.rs b/src/common/src/util/stream_graph_visitor.rs index c23cc6e2d65d..faab5ddab2ee 100644 --- a/src/common/src/util/stream_graph_visitor.rs +++ b/src/common/src/util/stream_graph_visitor.rs @@ -187,6 +187,9 @@ pub fn visit_stream_node_tables_inner( always!(source.state_table, "FsFetch"); } } + NodeBody::SourceBackfill(node) => { + always!(node.state_table, "SourceBackfill") + } // Sink NodeBody::Sink(node) => { diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 648061333b5e..1f2ba18373b9 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -25,6 +25,7 @@ arrow-array = { workspace = true } arrow-row = { workspace = true } arrow-schema = { workspace = true } arrow-select = { workspace = true } +assert_matches = "1" async-nats = "0.34" async-trait = "0.1" auto_enums = { version = "0.8", features = ["futures03"] } diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index 06cc06156669..59c47b06ee8b 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -185,6 +185,84 @@ pub fn build_additional_column_catalog( Ok(catalog) } +/// Utility function for adding partition and offset columns to the columns, if not specified by the user. +/// +/// ## Returns +/// - `columns_exist`: whether 1. `partition`/`file` and 2. `offset` columns are included in `columns`. +/// - `additional_columns`: The `ColumnCatalog` for `partition`/`file` and `offset` columns. +pub fn add_partition_offset_cols( + columns: &[ColumnCatalog], + connector_name: &str, +) -> ([bool; 2], [ColumnCatalog; 2]) { + let mut columns_exist = [false; 2]; + let mut last_column_id = columns + .iter() + .map(|c| c.column_desc.column_id) + .max() + .unwrap_or(ColumnId::placeholder()); + + let additional_columns: Vec<_> = { + let compat_col_types = COMPATIBLE_ADDITIONAL_COLUMNS + .get(connector_name) + .unwrap_or(&COMMON_COMPATIBLE_ADDITIONAL_COLUMNS); + ["partition", "file", "offset"] + .iter() + .filter_map(|col_type| { + last_column_id = last_column_id.next(); + if compat_col_types.contains(col_type) { + Some( + build_additional_column_catalog( + last_column_id, + connector_name, + col_type, + None, + None, + None, + false, + ) + .unwrap(), + ) + } else { + None + } + }) + .collect() + }; + assert_eq!(additional_columns.len(), 2); + use risingwave_pb::plan_common::additional_column::ColumnType; + assert_matches::assert_matches!( + additional_columns[0].column_desc.additional_column, + AdditionalColumn { + column_type: Some(ColumnType::Partition(_) | ColumnType::Filename(_)), + } + ); + assert_matches::assert_matches!( + additional_columns[1].column_desc.additional_column, + AdditionalColumn { + column_type: Some(ColumnType::Offset(_)), + } + ); + + // Check if partition/file/offset columns are included explicitly. + for col in columns { + match col.column_desc.additional_column { + AdditionalColumn { + column_type: Some(ColumnType::Partition(_) | ColumnType::Filename(_)), + } => { + columns_exist[0] = true; + } + AdditionalColumn { + column_type: Some(ColumnType::Offset(_)), + } => { + columns_exist[1] = true; + } + _ => (), + } + } + + (columns_exist, additional_columns.try_into().unwrap()) +} + fn build_header_catalog( column_id: ColumnId, col_name: &str, diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index ba2915ca47a8..62bb98241b59 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -72,8 +72,10 @@ pub trait SourceProperties: TryFromHashmap + Clone + WithOptions { type SplitEnumerator: SplitEnumerator; type SplitReader: SplitReader; + /// Load additional info from `PbSource`. Currently only used by CDC. fn init_from_pb_source(&mut self, _source: &PbSource) {} + /// Load additional info from `ExternalTableDesc`. Currently only used by CDC. fn init_from_pb_cdc_table_desc(&mut self, _table_desc: &ExternalTableDesc) {} } @@ -366,10 +368,12 @@ impl ConnectorProperties { matches!(self, ConnectorProperties::Kinesis(_)) } + /// Load additional info from `PbSource`. Currently only used by CDC. pub fn init_from_pb_source(&mut self, source: &PbSource) { dispatch_source_prop!(self, prop, prop.init_from_pb_source(source)) } + /// Load additional info from `ExternalTableDesc`. Currently only used by CDC. pub fn init_from_pb_cdc_table_desc(&mut self, cdc_table_desc: &ExternalTableDesc) { dispatch_source_prop!(self, prop, prop.init_from_pb_cdc_table_desc(cdc_table_desc)) } diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index 02e94dd337bd..d0d66ea221fd 100644 --- a/src/connector/src/source/cdc/mod.rs +++ b/src/connector/src/source/cdc/mod.rs @@ -148,7 +148,7 @@ where }; self.table_schema = table_schema; if let Some(info) = source.info.as_ref() { - self.is_cdc_source_job = info.cdc_source_job; + self.is_cdc_source_job = info.is_shared(); } } diff --git a/src/connector/src/source/kafka/private_link.rs b/src/connector/src/source/kafka/private_link.rs index 7569b5809fda..6187078ae24f 100644 --- a/src/connector/src/source/kafka/private_link.rs +++ b/src/connector/src/source/kafka/private_link.rs @@ -70,9 +70,9 @@ impl BrokerAddrRewriter { role: PrivateLinkContextRole, broker_rewrite_map: Option>, ) -> ConnectorResult { - tracing::info!("[{}] rewrite map {:?}", role, broker_rewrite_map); let rewrite_map: ConnectorResult> = broker_rewrite_map .map_or(Ok(BTreeMap::new()), |addr_map| { + tracing::info!("[{}] rewrite map {:?}", role, addr_map); addr_map .into_iter() .map(|(old_addr, new_addr)| { diff --git a/src/connector/src/source/reader/desc.rs b/src/connector/src/source/reader/desc.rs index 05fe87638c88..808ef1232c50 100644 --- a/src/connector/src/source/reader/desc.rs +++ b/src/connector/src/source/reader/desc.rs @@ -15,21 +15,18 @@ use std::collections::HashMap; use std::sync::Arc; +use itertools::Itertools; use risingwave_common::bail; -use risingwave_common::catalog::{ColumnDesc, ColumnId}; +use risingwave_common::catalog::{ColumnCatalog, ColumnDesc}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_pb::catalog::PbStreamSourceInfo; -use risingwave_pb::plan_common::additional_column::ColumnType; -use risingwave_pb::plan_common::{AdditionalColumn, PbColumnCatalog}; +use risingwave_pb::plan_common::PbColumnCatalog; #[expect(deprecated)] use super::fs_reader::FsSourceReader; use super::reader::SourceReader; use crate::error::ConnectorResult; -use crate::parser::additional_columns::{ - build_additional_column_catalog, COMMON_COMPATIBLE_ADDITIONAL_COLUMNS, - COMPATIBLE_ADDITIONAL_COLUMNS, -}; +use crate::parser::additional_columns::add_partition_offset_cols; use crate::parser::{EncodingProperties, ProtocolProperties, SpecificParserConfig}; use crate::source::monitor::SourceMetrics; use crate::source::{SourceColumnDesc, SourceColumnType, UPSTREAM_SOURCE_KEY}; @@ -94,65 +91,18 @@ impl SourceDescBuilder { /// This function builds `SourceColumnDesc` from `ColumnCatalog`, and handle the creation /// of hidden columns like partition/file, offset that are not specified by user. pub fn column_catalogs_to_source_column_descs(&self) -> Vec { - let mut columns_exist = [false; 2]; - let mut last_column_id = self - .columns - .iter() - .map(|c| c.column_desc.as_ref().unwrap().column_id.into()) - .max() - .unwrap_or(ColumnId::placeholder()); let connector_name = self .with_properties .get(UPSTREAM_SOURCE_KEY) .map(|s| s.to_lowercase()) .unwrap(); - - let additional_columns: Vec<_> = { - let compat_col_types = COMPATIBLE_ADDITIONAL_COLUMNS - .get(&*connector_name) - .unwrap_or(&COMMON_COMPATIBLE_ADDITIONAL_COLUMNS); - ["partition", "file", "offset"] - .iter() - .filter_map(|col_type| { - last_column_id = last_column_id.next(); - if compat_col_types.contains(col_type) { - Some( - build_additional_column_catalog( - last_column_id, - &connector_name, - col_type, - None, - None, - None, - false, - ) - .unwrap() - .to_protobuf(), - ) - } else { - None - } - }) - .collect() - }; - assert_eq!(additional_columns.len(), 2); - - // Check if partition/file/offset columns are included explicitly. - for col in &self.columns { - match col.column_desc.as_ref().unwrap().get_additional_column() { - Ok(AdditionalColumn { - column_type: Some(ColumnType::Partition(_) | ColumnType::Filename(_)), - }) => { - columns_exist[0] = true; - } - Ok(AdditionalColumn { - column_type: Some(ColumnType::Offset(_)), - }) => { - columns_exist[1] = true; - } - _ => (), - } - } + let columns = self + .columns + .iter() + .map(|c| ColumnCatalog::from(c.clone())) + .collect_vec(); + let (columns_exist, additional_columns) = + add_partition_offset_cols(&columns, &connector_name); let mut columns: Vec<_> = self .columns @@ -163,7 +113,7 @@ impl SourceDescBuilder { for (existed, c) in columns_exist.iter().zip_eq_fast(&additional_columns) { if !existed { columns.push(SourceColumnDesc::hidden_addition_col_from_column_desc( - &ColumnDesc::from(c.column_desc.as_ref().unwrap()), + &c.column_desc, )); } } diff --git a/src/connector/src/with_options.rs b/src/connector/src/with_options.rs index 64e6fc81d56c..5b7e75a47bcd 100644 --- a/src/connector/src/with_options.rs +++ b/src/connector/src/with_options.rs @@ -102,7 +102,8 @@ pub trait WithPropertiesExt: Get + Sized { connector.contains("-cdc") } - fn is_backfillable_cdc_connector(&self) -> bool { + /// It is shared when `CREATE SOURCE`, and not shared when `CREATE TABLE`. So called "shareable". + fn is_shareable_cdc_connector(&self) -> bool { self.is_cdc_connector() && CdcTableType::from_properties(self).can_backfill() } diff --git a/src/frontend/planner_test/tests/testdata/input/shared_source.yml b/src/frontend/planner_test/tests/testdata/input/shared_source.yml new file mode 100644 index 000000000000..0f68cc25f628 --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/input/shared_source.yml @@ -0,0 +1,68 @@ +- id: create_source + sql: | + CREATE SOURCE s(x int,y int) + WITH( + connector='kafka', + topic = 'mytopic', + properties.bootstrap.server = '6', + scan.startup.mode = 'earliest', + ) FORMAT PLAIN ENCODE JSON; + expected_outputs: [] +- with_config_map: + rw_enable_shared_source: true + sql: | + /* The shared source config doesn't affect table with connector. */ + EXPLAIN CREATE TABLE s(x int,y int) + WITH( + connector='kafka', + topic = 'mytopic', + properties.bootstrap.server = '6', + scan.startup.mode = 'earliest', + ) FORMAT PLAIN ENCODE JSON; + expected_outputs: + - explain_output +# Note: The execution order is first apply config, then execute "before", then execute "sql" +# We use with_config_map to control the config when CREATE SOURCE, and use another SET statement to change the config for CREATE MV +# +# batch: All 4 plans should be the same. +# stream: StreamSourceScan (with backfill) should be used only for the last 1. All other 3 use StreamSource. +- with_config_map: + rw_enable_shared_source: false + before: + - create_source + sql: | + SET rw_enable_shared_source = false; + select * from s; + expected_outputs: + - batch_plan + - stream_plan +- with_config_map: + rw_enable_shared_source: false + before: + - create_source + sql: | + SET rw_enable_shared_source = true; + select * from s; + expected_outputs: + - batch_plan + - stream_plan +- with_config_map: + rw_enable_shared_source: true + before: + - create_source + sql: | + SET rw_enable_shared_source = false; + select * from s; + expected_outputs: + - batch_plan + - stream_plan +- with_config_map: + rw_enable_shared_source: true + before: + - create_source + sql: | + SET rw_enable_shared_source = true; + select * from s; + expected_outputs: + - batch_plan + - stream_plan diff --git a/src/frontend/planner_test/tests/testdata/output/shared_source.yml b/src/frontend/planner_test/tests/testdata/output/shared_source.yml new file mode 100644 index 000000000000..54f50fc2c687 --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/output/shared_source.yml @@ -0,0 +1,94 @@ +# This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. +- id: create_source + sql: | + CREATE SOURCE s(x int,y int) + WITH( + connector='kafka', + topic = 'mytopic', + properties.bootstrap.server = '6', + scan.startup.mode = 'earliest', + ) FORMAT PLAIN ENCODE JSON; +- sql: | + /* The shared source config doesn't affect table with connector. */ + EXPLAIN CREATE TABLE s(x int,y int) + WITH( + connector='kafka', + topic = 'mytopic', + properties.bootstrap.server = '6', + scan.startup.mode = 'earliest', + ) FORMAT PLAIN ENCODE JSON; + explain_output: | + StreamMaterialize { columns: [x, y, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } + └─StreamRowIdGen { row_id_index: 2 } + └─StreamUnion { all: true } + ├─StreamExchange [no_shuffle] { dist: SomeShard } + │ └─StreamSource { source: s, columns: [x, y, _row_id] } + └─StreamExchange { dist: HashShard(_row_id) } + └─StreamDml { columns: [x, y, _row_id] } + └─StreamSource + with_config_map: + rw_enable_shared_source: 'true' +- before: + - create_source + sql: | + SET rw_enable_shared_source = false; + select * from s; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [x, y] } + └─BatchSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [x, y, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + └─StreamProject { exprs: [x, y, _row_id] } + └─StreamRowIdGen { row_id_index: 3 } + └─StreamSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id] } + with_config_map: + rw_enable_shared_source: 'false' +- before: + - create_source + sql: | + SET rw_enable_shared_source = true; + select * from s; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [x, y] } + └─BatchSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [x, y, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + └─StreamProject { exprs: [x, y, _row_id] } + └─StreamRowIdGen { row_id_index: 3 } + └─StreamSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id] } + with_config_map: + rw_enable_shared_source: 'false' +- before: + - create_source + sql: | + SET rw_enable_shared_source = false; + select * from s; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [x, y] } + └─BatchSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [x, y, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + └─StreamProject { exprs: [x, y, _row_id] } + └─StreamRowIdGen { row_id_index: 3 } + └─StreamSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id, _rw_kafka_partition, _rw_kafka_offset] } + with_config_map: + rw_enable_shared_source: 'true' +- before: + - create_source + sql: | + SET rw_enable_shared_source = true; + select * from s; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [x, y] } + └─BatchSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [x, y, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + └─StreamProject { exprs: [x, y, _row_id] } + └─StreamRowIdGen { row_id_index: 3 } + └─StreamSourceScan { columns: [x, y, _rw_kafka_timestamp, _row_id, _rw_kafka_partition, _rw_kafka_offset] } + with_config_map: + rw_enable_shared_source: 'true' diff --git a/src/frontend/src/binder/relation/table_or_source.rs b/src/frontend/src/binder/relation/table_or_source.rs index ef54c65cb5fe..782c533cd76d 100644 --- a/src/frontend/src/binder/relation/table_or_source.rs +++ b/src/frontend/src/binder/relation/table_or_source.rs @@ -55,8 +55,12 @@ pub struct BoundSource { } impl BoundSource { - pub fn is_backfillable_cdc_connector(&self) -> bool { - self.catalog.with_properties.is_backfillable_cdc_connector() + pub fn is_shareable_cdc_connector(&self) -> bool { + self.catalog.with_properties.is_shareable_cdc_connector() + } + + pub fn is_shared(&self) -> bool { + self.catalog.info.is_shared() } } diff --git a/src/frontend/src/catalog/source_catalog.rs b/src/frontend/src/catalog/source_catalog.rs index 59f77bba9fa4..17292b1324ed 100644 --- a/src/frontend/src/catalog/source_catalog.rs +++ b/src/frontend/src/catalog/source_catalog.rs @@ -16,6 +16,7 @@ use std::collections::BTreeMap; use risingwave_common::catalog::{ColumnCatalog, SourceVersionId}; use risingwave_common::util::epoch::Epoch; +use risingwave_connector::WithPropertiesExt; use risingwave_pb::catalog::source::OptionalAssociatedTableId; use risingwave_pb::catalog::{PbSource, StreamSourceInfo, WatermarkDesc}; @@ -83,6 +84,12 @@ impl SourceCatalog { pub fn version(&self) -> SourceVersionId { self.version } + + pub fn connector_name(&self) -> String { + self.with_properties + .get_connector() + .expect("connector name is missing") + } } impl From<&PbSource> for SourceCatalog { diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 0fdfed003053..6691b457da61 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -512,6 +512,7 @@ fn bind_columns_from_source_for_cdc( format_encode_options, use_schema_registry: json_schema_infer_use_schema_registry(&schema_config), cdc_source_job: true, + is_distributed: false, ..Default::default() }; if !format_encode_options_to_consume.is_empty() { @@ -1312,13 +1313,20 @@ pub async fn handle_create_source( ensure_table_constraints_supported(&stmt.constraints)?; let sql_pk_names = bind_sql_pk_names(&stmt.columns, &stmt.constraints)?; - let create_cdc_source_job = with_properties.is_backfillable_cdc_connector(); + let create_cdc_source_job = with_properties.is_shareable_cdc_connector(); + let is_shared = create_cdc_source_job + || (with_properties.is_kafka_connector() && session.config().rw_enable_shared_source()); - let (columns_from_resolve_source, source_info) = if create_cdc_source_job { + let (columns_from_resolve_source, mut source_info) = if create_cdc_source_job { bind_columns_from_source_for_cdc(&session, &source_schema, &with_properties)? } else { bind_columns_from_source(&session, &source_schema, &with_properties).await? }; + if is_shared { + // Note: this field should be called is_shared. Check field doc for more details. + source_info.cdc_source_job = true; + source_info.is_distributed = !create_cdc_source_job; + } let columns_from_sql = bind_sql_columns(&stmt.columns)?; let mut columns = bind_all_columns( @@ -1415,19 +1423,16 @@ pub async fn handle_create_source( let catalog_writer = session.catalog_writer()?; - if create_cdc_source_job { - // create a streaming job for the cdc source, which will mark as *singleton* in the Fragmenter + if is_shared { let graph = { let context = OptimizerContext::from_handler_args(handler_args); - // cdc source is an append-only source in plain json format let source_node = LogicalSource::with_catalog( Rc::new(SourceCatalog::from(&source)), - SourceNodeKind::CreateSourceWithStreamjob, + SourceNodeKind::CreateSharedSource, context.into(), None, )?; - // generate stream graph for cdc source job let stream_plan = source_node.to_stream(&mut ToStreamContext::new(false))?; let mut graph = build_graph(stream_plan)?; graph.parallelism = diff --git a/src/frontend/src/optimizer/plan_node/generic/source.rs b/src/frontend/src/optimizer/plan_node/generic/source.rs index 8d2a3dc8cec0..0c1caa620a4b 100644 --- a/src/frontend/src/optimizer/plan_node/generic/source.rs +++ b/src/frontend/src/optimizer/plan_node/generic/source.rs @@ -36,13 +36,13 @@ use crate::TableCatalog; pub enum SourceNodeKind { /// `CREATE TABLE` with a connector. CreateTable, - /// `CREATE SOURCE` with a streaming job (backfill-able source). - CreateSourceWithStreamjob, - /// `CREATE MATERIALIZED VIEW` which selects from a source. + /// `CREATE SOURCE` with a streaming job (shared source). + CreateSharedSource, + /// `CREATE MATERIALIZED VIEW` or batch scan from a source. /// /// Note: - /// - For non backfill-able source, `CREATE SOURCE` will not create a source node, and `CREATE MATERIALIZE VIEW` will create a `LogicalSource`. - /// - For backfill-able source, `CREATE MATERIALIZE VIEW` will create `LogicalSourceBackfill` instead of `LogicalSource`. + /// - For non-shared source, `CREATE SOURCE` will not create a source node, and `CREATE MATERIALIZE VIEW` will create a `StreamSource`. + /// - For shared source, `CREATE MATERIALIZE VIEW` will create `StreamSourceScan` instead of `StreamSource`. CreateMViewOrBatch, } diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 83fa891cb39a..fd4aaba5b7d2 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -35,7 +35,7 @@ use super::utils::{childless_record, Distill}; use super::{ generic, BatchProject, BatchSource, ColPrunable, ExprRewritable, Logical, LogicalFilter, LogicalProject, PlanBase, PlanRef, PredicatePushdown, StreamProject, StreamRowIdGen, - StreamSource, ToBatch, ToStream, + StreamSource, StreamSourceScan, ToBatch, ToStream, }; use crate::catalog::source_catalog::SourceCatalog; use crate::error::Result; @@ -76,6 +76,7 @@ impl LogicalSource { as_of: Option, ) -> Result { let kafka_timestamp_range = (Bound::Unbounded, Bound::Unbounded); + let core = generic::Source { catalog: source_catalog, column_catalog, @@ -517,9 +518,11 @@ impl ToBatch for LogicalSource { impl ToStream for LogicalSource { fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result { let mut plan: PlanRef; + match self.core.kind { - SourceNodeKind::CreateTable | SourceNodeKind::CreateSourceWithStreamjob => { - // Note: for create table, row_id and generated columns is created in plan_root.gen_table_plan + SourceNodeKind::CreateTable | SourceNodeKind::CreateSharedSource => { + // Note: for create table, row_id and generated columns is created in plan_root.gen_table_plan. + // for shared source, row_id and generated columns is created after SourceBackfill node. if self.core.is_new_fs_connector() { plan = Self::create_fs_list_plan(self.core.clone())?; plan = StreamFsFetch::new(plan, self.core.clone()).into(); @@ -529,11 +532,18 @@ impl ToStream for LogicalSource { } SourceNodeKind::CreateMViewOrBatch => { // Create MV on source. - if self.core.is_new_fs_connector() { - plan = Self::create_fs_list_plan(self.core.clone())?; - plan = StreamFsFetch::new(plan, self.core.clone()).into(); + let use_shared_source = self.source_catalog().is_some_and(|c| c.info.is_shared()) + && self.ctx().session_ctx().config().rw_enable_shared_source(); + if use_shared_source { + plan = StreamSourceScan::new(self.core.clone()).into(); } else { - plan = StreamSource::new(self.core.clone()).into() + // non-shared source + if self.core.is_new_fs_connector() { + plan = Self::create_fs_list_plan(self.core.clone())?; + plan = StreamFsFetch::new(plan, self.core.clone()).into(); + } else { + plan = StreamSource::new(self.core.clone()).into() + } } if let Some(exprs) = &self.output_exprs { diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 2ab2c8de8f74..780e1d2b39cd 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -697,8 +697,8 @@ impl dyn PlanNode { impl dyn PlanNode { /// Serialize the plan node and its children to a stream plan proto. /// - /// Note that [`StreamTableScan`] has its own implementation of `to_stream_prost`. We have a - /// hook inside to do some ad-hoc thing for [`StreamTableScan`]. + /// Note that some operators has their own implementation of `to_stream_prost`. We have a + /// hook inside to do some ad-hoc things. pub fn to_stream_prost( &self, state: &mut BuildFragmentGraphState, @@ -711,6 +711,9 @@ impl dyn PlanNode { if let Some(stream_cdc_table_scan) = self.as_stream_cdc_table_scan() { return stream_cdc_table_scan.adhoc_to_stream_prost(state); } + if let Some(stream_source_scan) = self.as_stream_source_scan() { + return stream_source_scan.adhoc_to_stream_prost(state); + } if let Some(stream_share) = self.as_stream_share() { return stream_share.adhoc_to_stream_prost(state); } @@ -874,6 +877,7 @@ mod stream_simple_agg; mod stream_sink; mod stream_sort; mod stream_source; +mod stream_source_scan; mod stream_stateless_simple_agg; mod stream_subscription; mod stream_table_scan; @@ -968,6 +972,7 @@ pub use stream_simple_agg::StreamSimpleAgg; pub use stream_sink::{IcebergPartitionInfo, PartitionComputeInfo, StreamSink}; pub use stream_sort::StreamEowcSort; pub use stream_source::StreamSource; +pub use stream_source_scan::StreamSourceScan; pub use stream_stateless_simple_agg::StreamStatelessSimpleAgg; pub use stream_subscription::StreamSubscription; pub use stream_table_scan::StreamTableScan; @@ -1064,6 +1069,7 @@ macro_rules! for_all_plan_nodes { , { Stream, Sink } , { Stream, Subscription } , { Stream, Source } + , { Stream, SourceScan } , { Stream, HashJoin } , { Stream, Exchange } , { Stream, HashAgg } @@ -1181,6 +1187,7 @@ macro_rules! for_stream_plan_nodes { , { Stream, Sink } , { Stream, Subscription } , { Stream, Source } + , { Stream, SourceScan } , { Stream, HashAgg } , { Stream, SimpleAgg } , { Stream, StatelessSimpleAgg } diff --git a/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs index 6aa63bbdb6ce..9df1a94879b2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs @@ -131,6 +131,7 @@ impl StreamNode for StreamCdcTableScan { } impl StreamCdcTableScan { + /// plan: merge -> filter -> exchange(simple) -> `stream_scan` pub fn adhoc_to_stream_prost( &self, state: &mut BuildFragmentGraphState, @@ -240,10 +241,10 @@ impl StreamCdcTableScan { .collect_vec(); tracing::debug!( - "output_column_ids: {:?}, upstream_column_ids: {:?}, output_indices: {:?}", - self.core.output_column_ids(), - upstream_column_ids, - output_indices + output_column_ids=?self.core.output_column_ids(), + ?upstream_column_ids, + ?output_indices, + "stream cdc table scan output indices" ); let stream_scan_body = PbNodeBody::StreamCdcScan(StreamCdcScanNode { diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index 537ade49687f..7b918a6312a6 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -17,6 +17,8 @@ use std::rc::Rc; use fixedbitset::FixedBitSet; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_connector::parser::additional_columns::add_partition_offset_cols; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::{PbStreamSource, SourceNode}; @@ -37,7 +39,23 @@ pub struct StreamSource { } impl StreamSource { - pub fn new(core: generic::Source) -> Self { + pub fn new(mut core: generic::Source) -> Self { + // For shared sources, we will include partition and offset cols in the *output*, to be used by the SourceBackfillExecutor. + // XXX: If we don't add here, these cols are also added in source reader, but pruned in the SourceExecutor's output. + // Should we simply add them here for all sources for consistency? + if let Some(source_catalog) = &core.catalog + && source_catalog.info.is_shared() + { + let (columns_exist, additional_columns) = + add_partition_offset_cols(&core.column_catalog, &source_catalog.connector_name()); + for (existed, mut c) in columns_exist.into_iter().zip_eq_fast(additional_columns) { + c.is_hidden = true; + if !existed { + core.column_catalog.push(c); + } + } + } + let base = PlanBase::new_stream_with_core( &core, Distribution::SomeShard, diff --git a/src/frontend/src/optimizer/plan_node/stream_source_scan.rs b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs new file mode 100644 index 000000000000..597517ce1e3f --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs @@ -0,0 +1,208 @@ +// 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 std::rc::Rc; + +use fixedbitset::FixedBitSet; +use itertools::Itertools; +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::catalog::Field; +use risingwave_common::types::DataType; +use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_common::util::sort_util::OrderType; +use risingwave_connector::parser::additional_columns::add_partition_offset_cols; +use risingwave_pb::stream_plan::stream_node::{NodeBody, PbNodeBody}; +use risingwave_pb::stream_plan::PbStreamNode; + +use super::stream::prelude::*; +use super::utils::TableCatalogBuilder; +use super::{PlanBase, PlanRef}; +use crate::catalog::source_catalog::SourceCatalog; +use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::plan_node::utils::{childless_record, Distill}; +use crate::optimizer::plan_node::{generic, ExprRewritable, StreamNode}; +use crate::optimizer::property::Distribution; +use crate::scheduler::SchedulerResult; +use crate::stream_fragmenter::BuildFragmentGraphState; +use crate::{Explain, TableCatalog}; + +/// `StreamSourceScan` scans from a *shared source*. It forwards data from the upstream [`StreamSource`], +/// and also backfills data from the external source. +/// +/// Unlike [`StreamSource`], which is a leaf node in the stream graph, `StreamSourceScan` is converted to `merge -> backfill` +/// +/// [`StreamSource`]:super::StreamSource +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct StreamSourceScan { + pub base: PlanBase, + core: generic::Source, +} + +impl_plan_tree_node_for_leaf! { StreamSourceScan } + +impl StreamSourceScan { + pub fn new(mut core: generic::Source) -> Self { + // XXX: do we need to include partition and offset cols here? It's needed by Backfill's input, but maybe not output? + // But the source's "schema" contains the hidden columns. + if let Some(source_catalog) = &core.catalog + && source_catalog.info.is_shared() + { + let (columns_exist, additional_columns) = + add_partition_offset_cols(&core.column_catalog, &source_catalog.connector_name()); + for (existed, mut c) in columns_exist.into_iter().zip_eq_fast(additional_columns) { + c.is_hidden = true; + if !existed { + core.column_catalog.push(c); + } + } + } + + let base = PlanBase::new_stream_with_core( + &core, + Distribution::SomeShard, + core.catalog.as_ref().map_or(true, |s| s.append_only), + false, + FixedBitSet::with_capacity(core.column_catalog.len()), + ); + + Self { base, core } + } + + fn get_columns(&self) -> Vec<&str> { + self.core + .column_catalog + .iter() + .map(|column| column.name()) + .collect() + } + + pub fn source_catalog(&self) -> Rc { + self.core + .catalog + .clone() + .expect("source scan should have source cataglog") + } + + pub fn infer_internal_table_catalog() -> TableCatalog { + // note that source's internal table is to store partition_id -> offset mapping and its + // schema is irrelevant to input schema + // On the premise of ensuring that the materialized_source data can be cleaned up, keep the + // state in source. + // Source state doesn't maintain retention_seconds, internal_table_subset function only + // returns retention_seconds so default is used here + let mut builder = TableCatalogBuilder::default(); + + let key = Field { + data_type: DataType::Varchar, + name: "partition_id".to_string(), + sub_fields: vec![], + type_name: "".to_string(), + }; + let value = Field { + data_type: DataType::Jsonb, + name: "backfill_progress".to_string(), + sub_fields: vec![], + type_name: "".to_string(), + }; + + let ordered_col_idx = builder.add_column(&key); + builder.add_column(&value); + builder.add_order_column(ordered_col_idx, OrderType::ascending()); + // read prefix hint is 0. We need to scan all data in the state table. + builder.build(vec![], 0) + } + + pub fn adhoc_to_stream_prost( + &self, + state: &mut BuildFragmentGraphState, + ) -> SchedulerResult { + use risingwave_pb::stream_plan::*; + + let stream_key = self + .stream_key() + .unwrap_or_else(|| { + panic!( + "should always have a stream key in the stream plan but not, sub plan: {}", + PlanRef::from(self.clone()).explain_to_string() + ) + }) + .iter() + .map(|x| *x as u32) + .collect_vec(); + + let source_catalog = self.source_catalog(); + let backfill = SourceBackfillNode { + upstream_source_id: source_catalog.id, + source_name: source_catalog.name.clone(), + state_table: Some( + Self::infer_internal_table_catalog() + .with_id(state.gen_table_id_wrapped()) + .to_internal_table_prost(), + ), + info: Some(source_catalog.info.clone()), + row_id_index: self.core.row_id_index.map(|index| index as _), + columns: self + .core + .column_catalog + .iter() + .map(|c| c.to_protobuf()) + .collect_vec(), + with_properties: source_catalog.with_properties.clone().into_iter().collect(), + rate_limit: self.base.ctx().overwrite_options().streaming_rate_limit, + }; + + let fields = self.schema().to_prost(); + // plan: merge -> backfill + Ok(PbStreamNode { + fields: fields.clone(), + input: vec![ + // The merge node body will be filled by the `ActorBuilder` on the meta service. + PbStreamNode { + node_body: Some(PbNodeBody::Merge(Default::default())), + identity: "Upstream".into(), + fields, + stream_key: vec![], // not used + ..Default::default() + }, + ], + node_body: Some(PbNodeBody::SourceBackfill(backfill)), + stream_key, + operator_id: self.base.id().0 as u64, + identity: self.distill_to_string(), + append_only: self.append_only(), + }) + } +} + +impl Distill for StreamSourceScan { + fn distill<'a>(&self) -> XmlNode<'a> { + let columns = self + .get_columns() + .iter() + .map(|ele| Pretty::from(ele.to_string())) + .collect(); + let col = Pretty::Array(columns); + childless_record("StreamSourceScan", vec![("columns", col)]) + } +} + +impl ExprRewritable for StreamSourceScan {} + +impl ExprVisitable for StreamSourceScan {} + +impl StreamNode for StreamSourceScan { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> NodeBody { + unreachable!("stream source scan cannot be converted into a prost body -- call `adhoc_to_stream_prost` instead.") + } +} diff --git a/src/frontend/src/planner/relation.rs b/src/frontend/src/planner/relation.rs index 74060b484e9c..62e32ad4dc3d 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -96,7 +96,7 @@ impl Planner { } pub(super) fn plan_source(&mut self, source: BoundSource) -> Result { - if source.is_backfillable_cdc_connector() { + if source.is_shareable_cdc_connector() { Err(ErrorCode::InternalError( "Should not create MATERIALIZED VIEW or SELECT directly on shared CDC source. HINT: create TABLE from the source instead.".to_string(), ) diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index bd22c8d7ea36..009449ec9228 100644 --- a/src/frontend/src/stream_fragmenter/mod.rs +++ b/src/frontend/src/stream_fragmenter/mod.rs @@ -264,9 +264,9 @@ fn build_fragment( if let Some(source) = node.source_inner.as_ref() && let Some(source_info) = source.info.as_ref() - && source_info.cdc_source_job + && source_info.is_shared() + && !source_info.is_distributed { - tracing::debug!("mark cdc source job as singleton"); current_fragment.requires_singleton = true; } } @@ -313,6 +313,13 @@ fn build_fragment( .upstream_table_ids .push(node.upstream_source_id); } + NodeBody::SourceBackfill(node) => { + current_fragment.fragment_type_mask |= FragmentTypeFlag::SourceScan as u32; + // memorize upstream source id for later use + let source_id = node.upstream_source_id; + state.dependent_table_ids.insert(source_id.into()); + current_fragment.upstream_table_ids.push(source_id); + } NodeBody::Now(_) => { // TODO: Remove this and insert a `BarrierRecv` instead. diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index c9554b4ecfaa..36245fcf7518 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -31,9 +31,9 @@ use risingwave_pb::stream_plan::barrier_mutation::Mutation; use risingwave_pb::stream_plan::throttle_mutation::RateLimit; use risingwave_pb::stream_plan::update_mutation::*; use risingwave_pb::stream_plan::{ - AddMutation, BarrierMutation, CombinedMutation, Dispatcher, Dispatchers, FragmentTypeFlag, - PauseMutation, ResumeMutation, SourceChangeSplitMutation, StopMutation, StreamActor, - ThrottleMutation, UpdateMutation, + AddMutation, BarrierMutation, CombinedMutation, Dispatcher, Dispatchers, PauseMutation, + ResumeMutation, SourceChangeSplitMutation, StopMutation, StreamActor, ThrottleMutation, + UpdateMutation, }; use risingwave_pb::stream_service::WaitEpochCommitRequest; use thiserror_ext::AsReport; @@ -158,7 +158,8 @@ pub enum Command { /// will be set to `Created`. CreateStreamingJob { table_fragments: TableFragments, - upstream_mview_actors: HashMap>, + /// Refer to the doc on [`MetadataManager::get_upstream_root_fragments`] for the meaning of "root". + upstream_root_actors: HashMap>, dispatchers: HashMap>, init_split_assignment: SplitAssignment, definition: String, @@ -690,29 +691,16 @@ impl CommandContext { } } - /// For `CreateStreamingJob`, returns the actors of the `StreamScan` nodes. For other commands, + /// For `CreateStreamingJob`, returns the actors of the `StreamScan`, and `StreamValue` nodes. For other commands, /// returns an empty set. pub fn actors_to_track(&self) -> HashSet { match &self.command { Command::CreateStreamingJob { - dispatchers, - table_fragments, - .. - } => { - // cdc backfill table job doesn't need to be tracked - if table_fragments.fragments().iter().any(|fragment| { - fragment.fragment_type_mask & FragmentTypeFlag::CdcFilter as u32 != 0 - }) { - Default::default() - } else { - dispatchers - .values() - .flatten() - .flat_map(|dispatcher| dispatcher.downstream_actor_id.iter().copied()) - .chain(table_fragments.values_actor_ids()) - .collect() - } - } + table_fragments, .. + } => table_fragments + .tracking_progress_actor_ids() + .into_iter() + .collect(), _ => Default::default(), } } @@ -793,7 +781,7 @@ impl CommandContext { .await?; self.barrier_manager_context .source_manager - .apply_source_change(None, Some(split_assignment.clone()), None) + .apply_source_change(None, None, Some(split_assignment.clone()), None) .await; } @@ -883,7 +871,7 @@ impl CommandContext { Command::CreateStreamingJob { table_fragments, dispatchers, - upstream_mview_actors, + upstream_root_actors, init_split_assignment, definition: _, replace_table, @@ -892,8 +880,8 @@ impl CommandContext { match &self.barrier_manager_context.metadata_manager { MetadataManager::V1(mgr) => { let mut dependent_table_actors = - Vec::with_capacity(upstream_mview_actors.len()); - for (table_id, actors) in upstream_mview_actors { + Vec::with_capacity(upstream_root_actors.len()); + for (table_id, actors) in upstream_root_actors { let downstream_actors = dispatchers .iter() .filter(|(upstream_actor_id, _)| actors.contains(upstream_actor_id)) @@ -966,11 +954,12 @@ impl CommandContext { // Extract the fragments that include source operators. let source_fragments = table_fragments.stream_source_fragments(); - + let backfill_fragments = table_fragments.source_backfill_fragments()?; self.barrier_manager_context .source_manager .apply_source_change( Some(source_fragments), + Some(backfill_fragments), Some(init_split_assignment.clone()), None, ) @@ -1034,10 +1023,13 @@ impl CommandContext { .drop_source_fragments(std::slice::from_ref(old_table_fragments)) .await; let source_fragments = new_table_fragments.stream_source_fragments(); + // XXX: is it possible to have backfill fragments here? + let backfill_fragments = new_table_fragments.source_backfill_fragments()?; self.barrier_manager_context .source_manager .apply_source_change( Some(source_fragments), + Some(backfill_fragments), Some(init_split_assignment.clone()), None, ) diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index 8483856168b5..a33380ea1cdd 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -238,6 +238,25 @@ impl TrackingJob { } } +impl std::fmt::Debug for TrackingJob { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + TrackingJob::New(command) => write!( + f, + "TrackingJob::New({:?})", + command.context.table_to_create() + ), + TrackingJob::Recovered(recovered) => { + write!( + f, + "TrackingJob::Recovered({:?})", + recovered.fragments.table_id() + ) + } + } + } +} + pub struct RecoveredTrackingJob { pub fragments: TableFragments, pub finished: Notifier, @@ -367,6 +386,7 @@ impl CreateMviewProgressTracker { /// /// Returns whether there are still remaining stashed jobs to finish. pub(super) async fn finish_jobs(&mut self, checkpoint: bool) -> MetaResult { + tracing::trace!(finished_jobs=?self.finished_jobs, progress_map=?self.progress_map, "finishing jobs"); for job in self .finished_jobs .extract_if(|job| checkpoint || !job.is_checkpoint_required()) @@ -414,7 +434,7 @@ impl CreateMviewProgressTracker { if let Command::CreateStreamingJob { table_fragments, dispatchers, - upstream_mview_actors, + upstream_root_actors, definition, ddl_type, .. @@ -422,7 +442,7 @@ impl CreateMviewProgressTracker { { // Keep track of how many times each upstream MV appears. let mut upstream_mv_count = HashMap::new(); - for (table_id, actors) in upstream_mview_actors { + for (table_id, actors) in upstream_root_actors { assert!(!actors.is_empty()); let dispatch_count: usize = dispatchers .iter() diff --git a/src/meta/src/barrier/schedule.rs b/src/meta/src/barrier/schedule.rs index af0badc0710d..50034811a339 100644 --- a/src/meta/src/barrier/schedule.rs +++ b/src/meta/src/barrier/schedule.rs @@ -279,15 +279,18 @@ impl BarrierScheduler { for (injected_rx, collect_rx, finish_rx) in contexts { // Wait for this command to be injected, and record the result. + tracing::trace!("waiting for injected_rx"); let info = injected_rx.await.ok().context("failed to inject barrier")?; infos.push(info); + tracing::trace!("waiting for collect_rx"); // Throw the error if it occurs when collecting this barrier. collect_rx .await .ok() .context("failed to collect barrier")??; + tracing::trace!("waiting for finish_rx"); // Wait for this command to be finished. finish_rx.await.ok().context("failed to finish command")??; } @@ -316,9 +319,13 @@ impl BarrierScheduler { /// /// Returns the barrier info of the actual command. pub async fn run_command(&self, command: Command) -> MetaResult { - self.run_multiple_commands(vec![command]) + tracing::trace!("run_command: {:?}", command); + let ret = self + .run_multiple_commands(vec![command]) .await - .map(|i| i[0]) + .map(|i| i[0]); + tracing::trace!("run_command finished"); + ret } /// Flush means waiting for the next barrier to collect. diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index d2d89dab586e..97fd0e081421 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -1978,7 +1978,7 @@ impl CatalogController { .map(|obj| obj.oid) .collect_vec(); - // cdc source streaming job. + // source streaming job. if object_type == ObjectType::Source { let source_info: Option = Source::find_by_id(object_id) .select_only() @@ -1988,7 +1988,7 @@ impl CatalogController { .await? .ok_or_else(|| MetaError::catalog_id_not_found("source", object_id))?; if let Some(source_info) = source_info - && source_info.into_inner().cdc_source_job + && source_info.into_inner().is_shared() { to_drop_streaming_jobs.push(object_id); } diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index 2e9f6a480299..92ee8295447a 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -51,8 +51,7 @@ use sea_orm::{ use crate::controller::catalog::{CatalogController, CatalogControllerInner}; use crate::controller::utils::{ - find_stream_source, get_actor_dispatchers, FragmentDesc, PartialActorLocation, - PartialFragmentStateTables, + get_actor_dispatchers, FragmentDesc, PartialActorLocation, PartialFragmentStateTables, }; use crate::manager::{ActorInfos, LocalNotification}; use crate::model::TableParallelism; @@ -1091,6 +1090,24 @@ impl CatalogController { Ok(actors) } + /// Get the actor ids, and each actor's upstream actor ids of the fragment with `fragment_id` with `Running` status. + pub async fn get_running_actors_and_upstream_of_fragment( + &self, + fragment_id: FragmentId, + ) -> MetaResult)>> { + let inner = self.inner.read().await; + let actors: Vec<(ActorId, Vec)> = Actor::find() + .select_only() + .column(actor::Column::ActorId) + .column(actor::Column::UpstreamActorIds) + .filter(actor::Column::FragmentId.eq(fragment_id)) + .filter(actor::Column::Status.eq(ActorStatus::Running)) + .into_tuple() + .all(&inner.db) + .await?; + Ok(actors) + } + pub async fn get_actors_by_job_ids(&self, job_ids: Vec) -> MetaResult> { let inner = self.inner.read().await; let actors: Vec = Actor::find() @@ -1205,9 +1222,9 @@ impl CatalogController { let mut source_fragment_ids = HashMap::new(); for (fragment_id, _, stream_node) in fragments { - if let Some(source) = find_stream_source(&stream_node.to_protobuf()) { + if let Some(source_id) = stream_node.to_protobuf().find_stream_source() { source_fragment_ids - .entry(source.source_id as SourceId) + .entry(source_id as SourceId) .or_insert_with(BTreeSet::new) .insert(fragment_id); } @@ -1215,31 +1232,33 @@ impl CatalogController { Ok(source_fragment_ids) } - pub async fn get_stream_source_fragment_ids( + pub async fn load_backfill_fragment_ids( &self, - job_id: ObjectId, - ) -> MetaResult>> { + ) -> MetaResult>> { let inner = self.inner.read().await; - let mut fragments: Vec<(FragmentId, i32, StreamNode)> = Fragment::find() + let mut fragments: Vec<(FragmentId, Vec, i32, StreamNode)> = Fragment::find() .select_only() .columns([ fragment::Column::FragmentId, + fragment::Column::UpstreamFragmentId, fragment::Column::FragmentTypeMask, fragment::Column::StreamNode, ]) - .filter(fragment::Column::JobId.eq(job_id)) .into_tuple() .all(&inner.db) .await?; - fragments.retain(|(_, mask, _)| *mask & PbFragmentTypeFlag::Source as i32 != 0); + fragments.retain(|(_, _, mask, _)| *mask & PbFragmentTypeFlag::SourceScan as i32 != 0); let mut source_fragment_ids = HashMap::new(); - for (fragment_id, _, stream_node) in fragments { - if let Some(source) = find_stream_source(&stream_node.to_protobuf()) { + for (fragment_id, upstream_fragment_id, _, stream_node) in fragments { + if let Some(source_id) = stream_node.to_protobuf().find_source_backfill() { + if upstream_fragment_id.len() != 1 { + bail!("SourceBackfill should have only one upstream fragment, found {} for fragment {}", upstream_fragment_id.len(), fragment_id); + } source_fragment_ids - .entry(source.source_id as SourceId) + .entry(source_id as SourceId) .or_insert_with(BTreeSet::new) - .insert(fragment_id); + .insert((fragment_id, upstream_fragment_id[0])); } } Ok(source_fragment_ids) diff --git a/src/meta/src/controller/streaming_job.rs b/src/meta/src/controller/streaming_job.rs index a4aeb1182d6c..9022921fe4e3 100644 --- a/src/meta/src/controller/streaming_job.rs +++ b/src/meta/src/controller/streaming_job.rs @@ -827,7 +827,7 @@ impl CatalogController { if let Some(table_id) = source.optional_associated_table_id { vec![table_id] } else if let Some(source_info) = &source.source_info - && source_info.inner_ref().cdc_source_job + && source_info.inner_ref().is_shared() { vec![source_id] } else { @@ -862,6 +862,7 @@ impl CatalogController { .map(|(id, mask, stream_node)| (id, mask, stream_node.to_protobuf())) .collect_vec(); + // TODO: limit source backfill? fragments.retain_mut(|(_, fragment_type_mask, stream_node)| { let mut found = false; if *fragment_type_mask & PbFragmentTypeFlag::Source as i32 != 0 { diff --git a/src/meta/src/manager/catalog/database.rs b/src/meta/src/manager/catalog/database.rs index 6759b0354113..54a0bea85dd1 100644 --- a/src/meta/src/manager/catalog/database.rs +++ b/src/meta/src/manager/catalog/database.rs @@ -227,7 +227,7 @@ impl DatabaseManager { && x.name.eq(&relation_key.2) }) { if t.stream_job_status == StreamJobStatus::Creating as i32 { - bail!("table is in creating procedure: {}", t.id); + bail!("table is in creating procedure, table id: {}", t.id); } else { Err(MetaError::catalog_duplicated("table", &relation_key.2)) } @@ -402,12 +402,9 @@ impl DatabaseManager { .chain(self.indexes.keys().copied()) .chain(self.sources.keys().copied()) .chain( - // filter cdc source jobs self.sources .iter() - .filter(|(_, source)| { - source.info.as_ref().is_some_and(|info| info.cdc_source_job) - }) + .filter(|(_, source)| source.info.as_ref().is_some_and(|info| info.is_shared())) .map(|(id, _)| id) .copied(), ) diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index cc606f8bf8fd..56873c2829a7 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -1027,6 +1027,30 @@ impl FragmentManager { bail!("fragment not found: {}", fragment_id) } + /// Get the actor ids, and each actor's upstream actor ids of the fragment with `fragment_id` with `Running` status. + pub async fn get_running_actors_and_upstream_of_fragment( + &self, + fragment_id: FragmentId, + ) -> MetaResult)>> { + let map = &self.core.read().await.table_fragments; + + for table_fragment in map.values() { + if let Some(fragment) = table_fragment.fragments.get(&fragment_id) { + let running_actors = fragment + .actors + .iter() + .filter(|a| { + table_fragment.actor_status[&a.actor_id].state == ActorState::Running as i32 + }) + .map(|a| (a.actor_id, a.upstream_actor_id.clone())) + .collect(); + return Ok(running_actors); + } + } + + bail!("fragment not found: {}", fragment_id) + } + /// Add the newly added Actor to the `FragmentManager` pub async fn pre_apply_reschedules( &self, diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 63c959d63a3b..7b8fa6bff760 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -1169,7 +1169,7 @@ impl CatalogManager { let mut all_subscription_ids: HashSet = HashSet::default(); let mut all_source_ids: HashSet = HashSet::default(); let mut all_view_ids: HashSet = HashSet::default(); - let mut all_cdc_source_ids: HashSet = HashSet::default(); + let mut all_streaming_job_source_ids: HashSet = HashSet::default(); let relations_depend_on = |relation_id: RelationId| -> Vec { let tables_depend_on = tables @@ -1453,11 +1453,10 @@ impl CatalogManager { continue; } - // cdc source streaming job if let Some(info) = source.info - && info.cdc_source_job + && info.is_shared() { - all_cdc_source_ids.insert(source.id); + all_streaming_job_source_ids.insert(source.id); let source_table_fragments = fragment_manager .select_table_fragments_by_table_id(&source.id.into()) .await?; @@ -1780,7 +1779,7 @@ impl CatalogManager { .map(|id| id.into()) .chain(all_sink_ids.into_iter().map(|id| id.into())) .chain(all_subscription_ids.into_iter().map(|id| id.into())) - .chain(all_cdc_source_ids.into_iter().map(|id| id.into())) + .chain(all_streaming_job_source_ids.into_iter().map(|id| id.into())) .collect_vec(); Ok((version, catalog_deleted_ids)) diff --git a/src/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs index bd2379b90c94..ebb03d461a2f 100644 --- a/src/meta/src/manager/metadata.rs +++ b/src/meta/src/manager/metadata.rs @@ -447,7 +447,7 @@ impl MetadataManager { /// In other words, it's the `MView` fragment if it exists, otherwise it's the `Source` fragment. /// /// ## What do we expect to get for different creating streaming job - /// - MV/Sink/Index should have MV upstream fragments for upstream MV/Tables, and Source upstream fragments for upstream backfill-able sources. + /// - MV/Sink/Index should have MV upstream fragments for upstream MV/Tables, and Source upstream fragments for upstream shared sources. /// - CDC Table has a Source upstream fragment. /// - Sources and other Tables shouldn't have an upstream fragment. pub async fn get_upstream_root_fragments( @@ -633,6 +633,34 @@ impl MetadataManager { } } + pub async fn get_running_actors_and_upstream_actors_of_fragment( + &self, + id: FragmentId, + ) -> MetaResult)>> { + match self { + MetadataManager::V1(mgr) => { + mgr.fragment_manager + .get_running_actors_and_upstream_of_fragment(id) + .await + } + MetadataManager::V2(mgr) => { + let actor_ids = mgr + .catalog_controller + .get_running_actors_and_upstream_of_fragment(id as _) + .await?; + Ok(actor_ids + .into_iter() + .map(|(id, actors)| { + ( + id as ActorId, + actors.into_iter().map(|id| id as ActorId).collect(), + ) + }) + .collect()) + } + } + } + pub async fn get_job_fragments_by_ids( &self, ids: &[TableId], diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index fb78c5f9ca11..058473580a96 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -30,7 +30,7 @@ use risingwave_pb::meta::{PbTableFragments, PbTableParallelism}; use risingwave_pb::plan_common::PbExprContext; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ - FragmentTypeFlag, PbFragmentTypeFlag, PbStreamContext, StreamActor, StreamNode, StreamSource, + FragmentTypeFlag, PbFragmentTypeFlag, PbStreamContext, StreamActor, StreamNode, }; use super::{ActorId, FragmentId}; @@ -337,7 +337,7 @@ impl TableFragments { } /// Returns the actor ids with the given fragment type. - fn filter_actor_ids(&self, check_type: impl Fn(u32) -> bool) -> Vec { + pub fn filter_actor_ids(&self, check_type: impl Fn(u32) -> bool) -> Vec { self.fragments .values() .filter(|fragment| check_type(fragment.get_fragment_type_mask())) @@ -367,11 +367,23 @@ impl TableFragments { }) } - /// Returns values actor ids. - pub fn values_actor_ids(&self) -> Vec { - Self::filter_actor_ids(self, |fragment_type_mask| { - (fragment_type_mask & FragmentTypeFlag::Values as u32) != 0 - }) + /// Returns actor ids that need to be tracked when creating MV. + pub fn tracking_progress_actor_ids(&self) -> Vec { + let mut actor_ids = vec![]; + for fragment in self.fragments.values() { + if fragment.fragment_type_mask & FragmentTypeFlag::CdcFilter as u32 != 0 { + // Note: CDC table job contains a StreamScan fragment (StreamCdcScan node) and a CdcFilter fragment. + // We don't track any fragments' progress. + return vec![]; + } + if (fragment.fragment_type_mask + & (FragmentTypeFlag::Values as u32 | FragmentTypeFlag::StreamScan as u32)) + != 0 + { + actor_ids.extend(fragment.actors.iter().map(|actor| actor.actor_id)); + } + } + actor_ids } /// Returns the fragment with the `Mview` type flag. @@ -411,23 +423,6 @@ impl TableFragments { .collect() } - /// Find the external stream source info inside the stream node, if any. - pub fn find_stream_source(stream_node: &StreamNode) -> Option<&StreamSource> { - if let Some(NodeBody::Source(source)) = stream_node.node_body.as_ref() { - if let Some(inner) = &source.source_inner { - return Some(inner); - } - } - - for child in &stream_node.input { - if let Some(source) = Self::find_stream_source(child) { - return Some(source); - } - } - - None - } - /// Extract the fragments that include source executors that contains an external stream source, /// grouping by source id. pub fn stream_source_fragments(&self) -> HashMap> { @@ -435,10 +430,7 @@ impl TableFragments { for fragment in self.fragments() { for actor in &fragment.actors { - if let Some(source_id) = - TableFragments::find_stream_source(actor.nodes.as_ref().unwrap()) - .map(|s| s.source_id) - { + if let Some(source_id) = actor.nodes.as_ref().unwrap().find_stream_source() { source_fragments .entry(source_id) .or_insert(BTreeSet::new()) @@ -451,6 +443,29 @@ impl TableFragments { source_fragments } + pub fn source_backfill_fragments( + &self, + ) -> MetadataModelResult>> { + let mut source_fragments = HashMap::new(); + + for fragment in self.fragments() { + for actor in &fragment.actors { + if let Some(source_id) = actor.nodes.as_ref().unwrap().find_source_backfill() { + if fragment.upstream_fragment_ids.len() != 1 { + return Err(anyhow::anyhow!("SourceBackfill should have only one upstream fragment, found {:?} for fragment {}", fragment.upstream_fragment_ids, fragment.fragment_id).into()); + } + source_fragments + .entry(source_id) + .or_insert(BTreeSet::new()) + .insert((fragment.fragment_id, fragment.upstream_fragment_ids[0])); + + break; + } + } + } + Ok(source_fragments) + } + /// Resolve dependent table fn resolve_dependent_table(stream_node: &StreamNode, table_ids: &mut HashMap) { let table_id = match stream_node.node_body.as_ref() { diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 681ad51fe994..f816ef3bb1c9 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -475,6 +475,7 @@ impl DdlController { .await; }; // 1. Drop source in catalog. + // If the source has a streaming job, it's also dropped here. let (version, streaming_job_ids) = mgr .catalog_manager .drop_relation( @@ -1360,7 +1361,7 @@ impl DdlController { .get_upstream_root_fragments(fragment_graph.dependent_table_ids()) .await?; - let upstream_actors: HashMap<_, _> = upstream_root_fragments + let upstream_root_actors: HashMap<_, _> = upstream_root_fragments .iter() .map(|(&table_id, fragment)| { ( @@ -1460,7 +1461,7 @@ impl DdlController { let ctx = CreateStreamingJobContext { dispatchers, - upstream_mview_actors: upstream_actors, + upstream_root_actors, internal_tables, building_locations, existing_locations, diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index 19a0d8942b5b..5f64aad214ef 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -732,7 +732,7 @@ impl ScaleController { if (fragment.get_fragment_type_mask() & FragmentTypeFlag::Source as u32) != 0 { let stream_node = fragment.actor_template.nodes.as_ref().unwrap(); - if TableFragments::find_stream_source(stream_node).is_some() { + if stream_node.find_stream_source().is_some() { stream_source_fragment_ids.insert(*fragment_id); } } @@ -1234,6 +1234,7 @@ impl ScaleController { fragment_stream_source_actor_splits.insert(*fragment_id, actor_splits); } } + // TODO: support migrate splits for SourceBackfill // Generate fragment reschedule plan let mut reschedule_fragment: HashMap = @@ -1738,6 +1739,7 @@ impl ScaleController { if !stream_source_actor_splits.is_empty() { self.source_manager .apply_source_change( + None, None, Some(stream_source_actor_splits), Some(stream_source_dropped_actors), diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index ec142f79a715..d950820889e1 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -31,6 +31,7 @@ use risingwave_connector::source::{ }; use risingwave_pb::catalog::Source; use risingwave_pb::source::{ConnectorSplit, ConnectorSplits}; +use risingwave_pb::stream_plan::Dispatcher; use thiserror_ext::AsReport; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; use tokio::sync::{oneshot, Mutex}; @@ -223,8 +224,8 @@ pub struct SourceManagerCore { managed_sources: HashMap, /// Fragments associated with each source source_fragments: HashMap>, - /// Revert index for `source_fragments` - fragment_sources: HashMap, + /// `source_id` -> `(fragment_id, upstream_fragment_id)` + backfill_fragments: HashMap>, /// Splits assigned per actor actor_splits: HashMap>, @@ -235,20 +236,14 @@ impl SourceManagerCore { metadata_manager: MetadataManager, managed_sources: HashMap, source_fragments: HashMap>, + backfill_fragments: HashMap>, actor_splits: HashMap>, ) -> Self { - let mut fragment_sources = HashMap::new(); - for (source_id, fragment_ids) in &source_fragments { - for fragment_id in fragment_ids { - fragment_sources.insert(*fragment_id, *source_id); - } - } - Self { metadata_manager, managed_sources, source_fragments, - fragment_sources, + backfill_fragments, actor_splits, } } @@ -262,12 +257,13 @@ impl SourceManagerCore { let mut split_assignment: SplitAssignment = HashMap::new(); for (source_id, handle) in &self.managed_sources { - let fragment_ids = match self.source_fragments.get(source_id) { + let source_fragment_ids = match self.source_fragments.get(source_id) { Some(fragment_ids) if !fragment_ids.is_empty() => fragment_ids, _ => { continue; } }; + let backfill_fragment_ids = self.backfill_fragments.get(source_id); let Some(discovered_splits) = handle.discovered_splits().await else { return Ok(split_assignment); @@ -276,20 +272,26 @@ impl SourceManagerCore { tracing::warn!("No splits discovered for source {}", source_id); } - for fragment_id in fragment_ids { - let actor_ids = match self + for &fragment_id in source_fragment_ids { + let actors = match self .metadata_manager - .get_running_actors_of_fragment(*fragment_id) + .get_running_actors_of_fragment(fragment_id) .await { - Ok(actor_ids) => actor_ids, + Ok(actors) => { + if actors.is_empty() { + tracing::warn!("No actors found for fragment {}", fragment_id); + continue; + } + actors + } Err(err) => { tracing::warn!(error = %err.as_report(), "Failed to get the actor of the fragment, maybe the fragment doesn't exist anymore"); continue; } }; - let prev_actor_splits: HashMap<_, _> = actor_ids + let prev_actor_splits: HashMap<_, _> = actors .into_iter() .map(|actor_id| { ( @@ -303,14 +305,51 @@ impl SourceManagerCore { .collect(); if let Some(new_assignment) = reassign_splits( - *fragment_id, + fragment_id, prev_actor_splits, &discovered_splits, SplitDiffOptions { enable_scale_in: handle.enable_scale_in, }, ) { - split_assignment.insert(*fragment_id, new_assignment); + split_assignment.insert(fragment_id, new_assignment); + } + } + + if let Some(backfill_fragment_ids) = backfill_fragment_ids { + // align splits for backfill fragments with its upstream source fragment + for (fragment_id, upstream_fragment_id) in backfill_fragment_ids { + let Some(upstream_assignment) = split_assignment.get(upstream_fragment_id) + else { + // upstream fragment unchanged, do not update backfill fragment too + continue; + }; + let actors = match self + .metadata_manager + .get_running_actors_and_upstream_actors_of_fragment(*fragment_id) + .await + { + Ok(actors) => { + if actors.is_empty() { + tracing::warn!("No actors found for fragment {}", fragment_id); + continue; + } + actors + } + Err(err) => { + tracing::warn!(error = %err.as_report(),"Failed to get the actor of the fragment, maybe the fragment doesn't exist anymore"); + continue; + } + }; + split_assignment.insert( + *fragment_id, + align_backfill_splits( + actors, + upstream_assignment, + *fragment_id, + *upstream_fragment_id, + )?, + ); } } } @@ -320,26 +359,32 @@ impl SourceManagerCore { fn apply_source_change( &mut self, - source_fragments: Option>>, + added_source_fragments: Option>>, + added_backfill_fragments: Option>>, split_assignment: Option, dropped_actors: Option>, ) { - if let Some(source_fragments) = source_fragments { + if let Some(source_fragments) = added_source_fragments { for (source_id, mut fragment_ids) in source_fragments { - for fragment_id in &fragment_ids { - self.fragment_sources.insert(*fragment_id, source_id); - } - self.source_fragments .entry(source_id) .or_default() .append(&mut fragment_ids); } } + if let Some(backfill_fragments) = added_backfill_fragments { + for (source_id, mut fragment_ids) in backfill_fragments { + self.backfill_fragments + .entry(source_id) + .or_default() + .append(&mut fragment_ids); + } + } if let Some(assignment) = split_assignment { for (_, actor_splits) in assignment { for (actor_id, splits) in actor_splits { + // override previous splits info self.actor_splits.insert(actor_id, splits); } } @@ -368,10 +413,6 @@ impl SourceManagerCore { entry.remove(); } } - - for fragment_id in &fragment_ids { - self.fragment_sources.remove(fragment_id); - } } for actor_id in removed_actors { @@ -427,6 +468,10 @@ impl Default for SplitDiffOptions { /// /// The existing splits will remain unmoved in their currently assigned actor. /// +/// If an actor has an upstream actor, it should be a backfill executor, +/// and its splits should be aligned with the upstream actor. `reassign_splits` should not be used in this case. +/// Use `align_backfill_splits` instead. +/// /// - `fragment_id`: just for logging /// /// ## Different connectors' behavior of split change @@ -539,6 +584,30 @@ where ) } +fn align_backfill_splits( + backfill_actors: impl IntoIterator)>, + upstream_assignment: &HashMap>, + fragment_id: FragmentId, + upstream_fragment_id: FragmentId, +) -> anyhow::Result>> { + backfill_actors + .into_iter() + .map(|(actor_id, upstream_actor_id)| { + let err = || anyhow::anyhow!("source backfill actor should have one upstream actor, fragment_id: {fragment_id}, upstream_fragment_id: {upstream_fragment_id}, actor_id: {actor_id}, upstream_assignment: {upstream_assignment:?}, upstream_actor_id: {upstream_actor_id:?}"); + if upstream_actor_id.len() != 1 { + return Err(err()); + } + let Some(splits) = upstream_assignment.get(&upstream_actor_id[0]) else { + return Err(err()); + }; + Ok(( + actor_id, + splits.clone(), + )) + }) + .collect() +} + impl SourceManager { const DEFAULT_SOURCE_TICK_INTERVAL: Duration = Duration::from_secs(10); const DEFAULT_SOURCE_TICK_TIMEOUT: Duration = Duration::from_secs(10); @@ -558,6 +627,7 @@ impl SourceManager { let mut actor_splits = HashMap::new(); let mut source_fragments = HashMap::new(); + let mut backfill_fragments = HashMap::new(); match &metadata_manager { MetadataManager::V1(mgr) => { @@ -569,6 +639,7 @@ impl SourceManager { .values() { source_fragments.extend(table_fragments.stream_source_fragments()); + backfill_fragments.extend(table_fragments.source_backfill_fragments()?); actor_splits.extend(table_fragments.actor_splits.clone()); } } @@ -585,6 +656,21 @@ impl SourceManager { ) }) .collect(); + backfill_fragments = mgr + .catalog_controller + .load_backfill_fragment_ids() + .await? + .into_iter() + .map(|(source_id, fragment_ids)| { + ( + source_id as SourceId, + fragment_ids + .into_iter() + .map(|(id, up_id)| (id as _, up_id as _)) + .collect(), + ) + }) + .collect(); actor_splits = mgr .catalog_controller .load_actor_splits() @@ -609,6 +695,7 @@ impl SourceManager { metadata_manager, managed_sources, source_fragments, + backfill_fragments, actor_splits, )); @@ -657,12 +744,18 @@ impl SourceManager { /// Updates states after split change (`post_collect` barrier) or scaling (`post_apply_reschedule`). pub async fn apply_source_change( &self, - source_fragments: Option>>, + added_source_fragments: Option>>, + added_backfill_fragments: Option>>, split_assignment: Option, dropped_actors: Option>, ) { let mut core = self.core.lock().await; - core.apply_source_change(source_fragments, split_assignment, dropped_actors); + core.apply_source_change( + added_source_fragments, + added_backfill_fragments, + split_assignment, + dropped_actors, + ); } /// Migrates splits from previous actors to the new actors for a rescheduled fragment. @@ -764,6 +857,67 @@ impl SourceManager { Ok(assigned) } + /// Allocates splits to actors for a newly created `SourceBackfill` executor. + /// + /// Unlike [`Self::allocate_splits`], which creates a new assignment, + /// this method aligns the splits for backfill fragments with its upstream source fragment ([`align_backfill_splits`]). + pub async fn allocate_splits_for_backfill( + &self, + table_id: &TableId, + dispatchers: &HashMap>, + ) -> MetaResult { + let core = self.core.lock().await; + let table_fragments = core + .metadata_manager + .get_job_fragments_by_id(table_id) + .await?; + + let upstream_assignment = &core.actor_splits; + let source_backfill_fragments = table_fragments.source_backfill_fragments()?; + + let mut assigned = HashMap::new(); + + for (_source_id, fragments) in source_backfill_fragments { + for (fragment_id, upstream_fragment_id) in fragments { + let upstream_actors = core + .metadata_manager + .get_running_actors_of_fragment(upstream_fragment_id) + .await?; + let mut backfill_actors = vec![]; + for upstream_actor in upstream_actors { + if let Some(dispatchers) = dispatchers.get(&upstream_actor) { + let err = || { + anyhow::anyhow!( + "source backfill fragment's upstream fragment should have one dispatcher, fragment_id: {fragment_id}, upstream_fragment_id: {upstream_fragment_id}, upstream_actor: {upstream_actor}, dispatchers: {dispatchers:?}", + fragment_id = fragment_id, + upstream_fragment_id = upstream_fragment_id, + upstream_actor = upstream_actor, + dispatchers = dispatchers + ) + }; + if dispatchers.len() != 1 || dispatchers[0].downstream_actor_id.len() != 1 { + return Err(err().into()); + } + + backfill_actors + .push((dispatchers[0].downstream_actor_id[0], vec![upstream_actor])); + } + } + assigned.insert( + fragment_id, + align_backfill_splits( + backfill_actors, + upstream_assignment, + fragment_id, + upstream_fragment_id, + )?, + ); + } + } + + Ok(assigned) + } + /// register connector worker for source. pub async fn register_source(&self, source: &Source) -> MetaResult<()> { let mut core = self.core.lock().await; diff --git a/src/meta/src/stream/stream_graph/actor.rs b/src/meta/src/stream/stream_graph/actor.rs index c42c2f5a5142..c25b48c28f04 100644 --- a/src/meta/src/stream/stream_graph/actor.rs +++ b/src/meta/src/stream/stream_graph/actor.rs @@ -122,7 +122,7 @@ impl ActorBuilder { /// During this process, the following things will be done: /// 1. Replace the logical `Exchange` in node's input with `Merge`, which can be executed on the /// compute nodes. - /// 2. Fill the upstream mview info of the `Merge` node under the `StreamScan` node. + /// 2. Fill the upstream mview info of the `Merge` node under the other "leaf" nodes. fn rewrite(&self) -> MetaResult { self.rewrite_inner(&self.nodes, 0) } @@ -210,20 +210,21 @@ impl ActorBuilder { }) } - // "Leaf" node `CdcFilter` used in multi-table cdc backfill plan: + // "Leaf" node `CdcFilter` and `SourceBackfill`. They both `Merge` an upstream `Source` // cdc_filter -> backfill -> mview - NodeBody::CdcFilter(node) => { + // source_backfill -> mview + NodeBody::CdcFilter(_) | NodeBody::SourceBackfill(_) => { let input = stream_node.get_input(); assert_eq!(input.len(), 1); let merge_node = &input[0]; assert_matches!(merge_node.node_body, Some(NodeBody::Merge(_))); - let upstream_source_id = node.upstream_source_id; - tracing::debug!( - "rewrite leaf cdc filter node: upstream source id {}", - upstream_source_id, - ); + let upstream_source_id = match stream_node.get_node_body()? { + NodeBody::CdcFilter(node) => node.upstream_source_id, + NodeBody::SourceBackfill(node) => node.upstream_source_id, + _ => unreachable!(), + }; // Index the upstreams by the an external edge ID. let upstreams = &self.upstreams[&EdgeId::UpstreamExternal { @@ -231,11 +232,13 @@ impl ActorBuilder { downstream_fragment_id: self.fragment_id, }]; - // Upstream Cdc Source should be singleton. let upstream_actor_id = upstreams.actors.as_global_ids(); + // Upstream Cdc Source should be singleton. + // SourceBackfill is NoShuffle 1-1 correspondence. + // So they both should have only one upstream actor. assert_eq!(upstream_actor_id.len(), 1); - // rewrite the input of `CdcFilter` + // rewrite the input let input = vec![ // Fill the merge node body with correct upstream info. StreamNode { diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index fbe250818e6e..0e332054d7ea 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -54,7 +54,8 @@ pub(super) struct BuildingFragment { /// The ID of the job if it contains the streaming job node. table_id: Option, - /// The required columns of each upstream table. + /// The required column IDs of each upstream table. + /// Will be converted to indices when building the edge connected to the upstream. /// /// For shared CDC table on source, its `vec![]`, since the upstream source's output schema is fixed. upstream_table_columns: HashMap>, @@ -182,6 +183,15 @@ impl BuildingFragment { stream_scan.upstream_column_ids.clone(), ), NodeBody::CdcFilter(cdc_filter) => (cdc_filter.upstream_source_id.into(), vec![]), + NodeBody::SourceBackfill(backfill) => ( + backfill.upstream_source_id.into(), + // FIXME: only pass required columns instead of all columns here + backfill + .columns + .iter() + .map(|c| c.column_desc.as_ref().unwrap().column_id) + .collect(), + ), _ => return, }; table_columns @@ -192,7 +202,7 @@ impl BuildingFragment { assert_eq!( table_columns.len(), fragment.upstream_table_ids.len(), - "fragment type: {}", + "fragment type: {:b}", fragment.fragment_type_mask ); @@ -663,50 +673,100 @@ impl CompleteStreamFragmentGraph { | DdlType::Sink | DdlType::Index | DdlType::Subscription => { - // handle MV on MV + // handle MV on MV/Source // Build the extra edges between the upstream `Materialize` and the downstream `StreamScan` // of the new materialized view. - let mview_fragment = upstream_root_fragments + let upstream_fragment = upstream_root_fragments .get(&upstream_table_id) .context("upstream materialized view fragment not found")?; - let mview_id = GlobalFragmentId::new(mview_fragment.fragment_id); - - // Resolve the required output columns from the upstream materialized view. - let (dist_key_indices, output_indices) = { - let nodes = mview_fragment.actors[0].get_nodes().unwrap(); - let mview_node = - nodes.get_node_body().unwrap().as_materialize().unwrap(); - let all_column_ids = mview_node.column_ids(); - let dist_key_indices = mview_node.dist_key_indices(); - let output_indices = output_columns - .iter() - .map(|c| { - all_column_ids - .iter() - .position(|&id| id == *c) - .map(|i| i as u32) - }) - .collect::>>() - .context( - "column not found in the upstream materialized view", - )?; - (dist_key_indices, output_indices) - }; - let dispatch_strategy = mv_on_mv_dispatch_strategy( - uses_arrangement_backfill, - dist_key_indices, - output_indices, - ); - let edge = StreamFragmentEdge { - id: EdgeId::UpstreamExternal { - upstream_table_id, - downstream_fragment_id: id, - }, - dispatch_strategy, - }; - - (mview_id, edge) + let upstream_root_fragment_id = + GlobalFragmentId::new(upstream_fragment.fragment_id); + + if upstream_fragment.fragment_type_mask & FragmentTypeFlag::Mview as u32 + != 0 + { + // Resolve the required output columns from the upstream materialized view. + let (dist_key_indices, output_indices) = { + let nodes = upstream_fragment.actors[0].get_nodes().unwrap(); + let mview_node = + nodes.get_node_body().unwrap().as_materialize().unwrap(); + let all_column_ids = mview_node.column_ids(); + let dist_key_indices = mview_node.dist_key_indices(); + let output_indices = output_columns + .iter() + .map(|c| { + all_column_ids + .iter() + .position(|&id| id == *c) + .map(|i| i as u32) + }) + .collect::>>() + .context( + "column not found in the upstream materialized view", + )?; + (dist_key_indices, output_indices) + }; + let dispatch_strategy = mv_on_mv_dispatch_strategy( + uses_arrangement_backfill, + dist_key_indices, + output_indices, + ); + let edge = StreamFragmentEdge { + id: EdgeId::UpstreamExternal { + upstream_table_id, + downstream_fragment_id: id, + }, + dispatch_strategy, + }; + + (upstream_root_fragment_id, edge) + } else if upstream_fragment.fragment_type_mask + & FragmentTypeFlag::Source as u32 + != 0 + { + let source_fragment = upstream_root_fragments + .get(&upstream_table_id) + .context("upstream source fragment not found")?; + let source_job_id = + GlobalFragmentId::new(source_fragment.fragment_id); + + let output_indices = { + let nodes = upstream_fragment.actors[0].get_nodes().unwrap(); + let source_node = + nodes.get_node_body().unwrap().as_source().unwrap(); + + let all_column_ids = source_node.column_ids().unwrap(); + output_columns + .iter() + .map(|c| { + all_column_ids + .iter() + .position(|&id| id == *c) + .map(|i| i as u32) + }) + .collect::>>() + .context("column not found in the upstream source node")? + }; + + let edge = StreamFragmentEdge { + id: EdgeId::UpstreamExternal { + upstream_table_id, + downstream_fragment_id: id, + }, + // We always use `NoShuffle` for the exchange between the upstream + // `Source` and the downstream `StreamScan` of the new MV. + dispatch_strategy: DispatchStrategy { + r#type: DispatcherType::NoShuffle as _, + dist_key_indices: vec![], // not used for `NoShuffle` + output_indices, + }, + }; + + (source_job_id, edge) + } else { + bail!("the upstream fragment should be a MView or Source, got fragment type: {:b}", upstream_fragment.fragment_type_mask) + } } DdlType::Source | DdlType::Table(_) => { bail!("the streaming job shouldn't have an upstream fragment, ddl_type: {:?}", ddl_type) diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index dc72fcd5119a..e851cc1042f4 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -49,8 +49,10 @@ pub struct CreateStreamingJobContext { /// New dispatchers to add from upstream actors to downstream actors. pub dispatchers: HashMap>, - /// Upstream mview actor ids grouped by table id. - pub upstream_mview_actors: HashMap>, + /// Upstream root fragments' actor ids grouped by table id. + /// + /// Refer to the doc on [`MetadataManager::get_upstream_root_fragments`] for the meaning of "root". + pub upstream_root_actors: HashMap>, /// Internal tables in the streaming job. pub internal_tables: HashMap, @@ -381,7 +383,7 @@ impl GlobalStreamManager { table_fragments: TableFragments, CreateStreamingJobContext { dispatchers, - upstream_mview_actors, + upstream_root_actors, building_locations, existing_locations, definition, @@ -396,6 +398,10 @@ impl GlobalStreamManager { self.build_actors(&table_fragments, &building_locations, &existing_locations) .await?; + tracing::debug!( + table_id = %table_fragments.table_id(), + "built actors finished" + ); if let Some((streaming_job, context, table_fragments)) = replace_table_job_info { self.build_actors( @@ -420,7 +426,6 @@ impl GlobalStreamManager { } let dummy_table_id = table_fragments.table_id(); - let init_split_assignment = self.source_manager.allocate_splits(&dummy_table_id).await?; @@ -437,18 +442,27 @@ impl GlobalStreamManager { let table_id = table_fragments.table_id(); - let init_split_assignment = self.source_manager.allocate_splits(&table_id).await?; + // Here we need to consider: + // - Shared source + // - Table with connector + // - MV on shared source + let mut init_split_assignment = self.source_manager.allocate_splits(&table_id).await?; + init_split_assignment.extend( + self.source_manager + .allocate_splits_for_backfill(&table_id, &dispatchers) + .await?, + ); let command = Command::CreateStreamingJob { table_fragments, - upstream_mview_actors, + upstream_root_actors, dispatchers, init_split_assignment, definition: definition.to_string(), ddl_type, replace_table: replace_table_command, }; - + tracing::debug!("sending Command::CreateStreamingJob"); if let Err(err) = self.barrier_scheduler.run_command(command).await { if create_type == CreateType::Foreground || err.is_cancelled() { let mut table_ids = HashSet::from_iter(std::iter::once(table_id)); @@ -483,7 +497,6 @@ impl GlobalStreamManager { .await?; let dummy_table_id = table_fragments.table_id(); - let init_split_assignment = self.source_manager.allocate_splits(&dummy_table_id).await?; if let Err(err) = self diff --git a/src/prost/src/lib.rs b/src/prost/src/lib.rs index 82f399d1a3de..825e721891fe 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -196,6 +196,68 @@ impl stream_plan::MaterializeNode { } } +impl stream_plan::SourceNode { + pub fn column_ids(&self) -> Option> { + Some( + self.source_inner + .as_ref()? + .columns + .iter() + .map(|c| c.get_column_desc().unwrap().column_id) + .collect(), + ) + } +} + +impl stream_plan::StreamNode { + /// Find the external stream source info inside the stream node, if any. + /// + /// Returns `source_id`. + pub fn find_stream_source(&self) -> Option { + if let Some(crate::stream_plan::stream_node::NodeBody::Source(source)) = + self.node_body.as_ref() + { + if let Some(inner) = &source.source_inner { + return Some(inner.source_id); + } + } + + for child in &self.input { + if let Some(source) = child.find_stream_source() { + return Some(source); + } + } + + None + } + + /// Find the external stream source info inside the stream node, if any. + /// + /// Returns `source_id`. + pub fn find_source_backfill(&self) -> Option { + if let Some(crate::stream_plan::stream_node::NodeBody::SourceBackfill(source)) = + self.node_body.as_ref() + { + return Some(source.upstream_source_id); + } + + for child in &self.input { + if let Some(source) = child.find_source_backfill() { + return Some(source); + } + } + + None + } +} + +impl catalog::StreamSourceInfo { + /// Refer to [`Self::cdc_source_job`] for details. + pub fn is_shared(&self) -> bool { + self.cdc_source_job + } +} + #[cfg(test)] mod tests { use crate::data::{data_type, DataType}; diff --git a/src/stream/src/executor/source/source_backfill_executor.rs b/src/stream/src/executor/source/source_backfill_executor.rs index b017021a01e3..6f8fca84bafd 100644 --- a/src/stream/src/executor/source/source_backfill_executor.rs +++ b/src/stream/src/executor/source/source_backfill_executor.rs @@ -512,11 +512,7 @@ impl SourceBackfillExecutorInner { // backfill Either::Right(msg) => { let chunk = msg?; - // TODO(optimize): actually each msg is from one split. We can - // include split from the message and avoid iterating over all rows. - let split_offset_mapping = - get_split_offset_mapping_from_chunk(&chunk, split_idx, offset_idx) - .unwrap(); + if last_barrier_time.elapsed().as_millis() > max_wait_barrier_time_ms { // Exceeds the max wait barrier time, the source will be paused. // Currently we can guarantee the @@ -538,8 +534,17 @@ impl SourceBackfillExecutorInner { self.system_params.load().barrier_interval_ms() as u128 * WAIT_BARRIER_MULTIPLE_TIMES; } - split_offset_mapping.iter().for_each(|(split_id, offset)| { + // TODO(optimize): actually each msg is from one split. We can + // include split from the message and avoid iterating over all rows. + let mut new_vis = BitmapBuilder::zeroed(chunk.visibility().len()); + + for (i, (_, row)) in chunk.rows().enumerate() { + let split_id: Arc = + row.datum_at(split_idx).unwrap().into_utf8().into(); + let offset: String = + row.datum_at(offset_idx).unwrap().into_utf8().into(); // update backfill progress + let mut vis = true; match backfill_stage.states.entry(split_id.clone()) { Entry::Occupied(mut entry) => { let state = entry.get_mut(); @@ -551,6 +556,7 @@ impl SourceBackfillExecutorInner { BackfillState::SourceCachingUp(_) | BackfillState::Finished => { // backfilling stopped. ignore + vis = false } } } @@ -558,11 +564,16 @@ impl SourceBackfillExecutorInner { entry.insert(BackfillState::Backfilling(Some(offset.clone()))); } } - }); - - source_backfill_row_count.inc_by(chunk.cardinality() as u64); + new_vis.set(i, vis); + } - yield Message::Chunk(chunk); + let new_vis = new_vis.finish(); + let card = new_vis.count_ones(); + if card != 0 { + let new_chunk = chunk.clone_with_vis(new_vis); + yield Message::Chunk(new_chunk); + source_backfill_row_count.inc_by(card as u64); + } } } } @@ -603,11 +614,11 @@ impl SourceBackfillExecutorInner { } _ => {} } - self.backfill_state_store - .state_store - .commit(barrier.epoch) - .await?; } + self.backfill_state_store + .state_store + .commit(barrier.epoch) + .await?; yield Message::Barrier(barrier); } Message::Chunk(chunk) => { @@ -717,11 +728,6 @@ impl SourceBackfillExecutorInner { } if split_changed { - tracing::info!( - state = ?target_state, - "apply split change" - ); - stage .unfinished_splits .retain(|split| target_state.get(split.id().as_ref()).is_some()); @@ -735,7 +741,7 @@ impl SourceBackfillExecutorInner { // trim dropped splits' state self.backfill_state_store.trim_state(dropped_splits).await?; } - + tracing::info!(old_state=?stage.states, new_state=?target_state, "finish split change"); stage.states = target_state; } diff --git a/src/stream/src/executor/source/source_backfill_state_table.rs b/src/stream/src/executor/source/source_backfill_state_table.rs index c359ff643426..678a76f39f92 100644 --- a/src/stream/src/executor/source/source_backfill_state_table.rs +++ b/src/stream/src/executor/source/source_backfill_state_table.rs @@ -67,7 +67,6 @@ impl BackfillStateTableHandler { let mut ret = vec![]; while let Some(item) = state_table_iter.next().await { let row = item?.into_owned_row(); - tracing::debug!("scanning backfill state table, row: {:?}", row); let state = match row.datum_at(1) { Some(ScalarRefImpl::Jsonb(jsonb_ref)) => { BackfillState::restore_from_json(jsonb_ref.to_owned_scalar())? diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 7fb514faac9c..d1d21473dce2 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -415,7 +415,7 @@ impl SourceExecutor { if barrier.is_pause_on_startup() { stream.pause_stream(); } - // TODO: for backfill-able source, pause until there's a MV. + // TODO: for shared source, pause until there's a MV. yield Message::Barrier(barrier); diff --git a/src/stream/src/from_proto/source_backfill.rs b/src/stream/src/from_proto/source_backfill.rs index 17304f170ff0..eadb94921707 100644 --- a/src/stream/src/from_proto/source_backfill.rs +++ b/src/stream/src/from_proto/source_backfill.rs @@ -34,7 +34,7 @@ impl ExecutorBuilder for SourceBackfillExecutorBuilder { node: &Self::Node, store: impl StateStore, ) -> StreamResult { - let source_id = TableId::new(node.source_id); + let source_id = TableId::new(node.upstream_source_id); let source_name = node.source_name.clone(); let source_info = node.get_info()?;