From 9c442012ad4b0d080fba6715c93385b2ee751e7c Mon Sep 17 00:00:00 2001 From: xxchan Date: Tue, 12 Mar 2024 16:03:53 +0800 Subject: [PATCH 01/29] fix: deny create MV on shared CDC source --- e2e_test/source/cdc/cdc.share_stream.slt | 3 +++ src/connector/src/source/cdc/external/mod.rs | 8 +++----- src/connector/src/with_options.rs | 7 ++++++- .../src/binder/relation/table_or_source.rs | 7 +++++++ src/frontend/src/handler/create_source.rs | 8 +------- src/frontend/src/planner/relation.rs | 19 +++++++++++++------ 6 files changed, 33 insertions(+), 19 deletions(-) diff --git a/e2e_test/source/cdc/cdc.share_stream.slt b/e2e_test/source/cdc/cdc.share_stream.slt index 07ffd6eb470a..478c748ceabf 100644 --- a/e2e_test/source/cdc/cdc.share_stream.slt +++ b/e2e_test/source/cdc/cdc.share_stream.slt @@ -23,6 +23,9 @@ create source mysql_mytest with ( server.id = '5601' ); +statement error Should not create MATERIALIZED VIEW directly on shared CDC source. +create materialized view mv as select * from mysql_mytest; + statement error The upstream table name must contain database name prefix* create table products_test ( id INT, name STRING, diff --git a/src/connector/src/source/cdc/external/mod.rs b/src/connector/src/source/cdc/external/mod.rs index a9808e3a9e1e..ee049a2248ca 100644 --- a/src/connector/src/source/cdc/external/mod.rs +++ b/src/connector/src/source/cdc/external/mod.rs @@ -36,6 +36,7 @@ use crate::error::{ConnectorError, ConnectorResult}; use crate::parser::mysql_row_to_owned_row; use crate::source::cdc::external::mock_external_table::MockExternalTableReader; use crate::source::cdc::external::postgres::{PostgresExternalTableReader, PostgresOffset}; +use crate::WithPropertiesExt; #[derive(Debug)] pub enum CdcTableType { @@ -46,11 +47,8 @@ pub enum CdcTableType { } impl CdcTableType { - pub fn from_properties(with_properties: &HashMap) -> Self { - let connector = with_properties - .get("connector") - .map(|c| c.to_ascii_lowercase()) - .unwrap_or_default(); + pub fn from_properties(with_properties: &impl WithPropertiesExt) -> Self { + let connector = with_properties.get_connector().unwrap_or_default(); match connector.as_str() { "mysql-cdc" => Self::MySql, "postgres-cdc" => Self::Postgres, diff --git a/src/connector/src/with_options.rs b/src/connector/src/with_options.rs index 941eaadd459e..a11cb6c12a6b 100644 --- a/src/connector/src/with_options.rs +++ b/src/connector/src/with_options.rs @@ -14,6 +14,7 @@ use std::collections::{BTreeMap, HashMap}; +use crate::source::cdc::external::CdcTableType; use crate::source::iceberg::ICEBERG_CONNECTOR; use crate::source::{ GCS_CONNECTOR, KAFKA_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR, UPSTREAM_SOURCE_KEY, @@ -79,7 +80,7 @@ impl Get for BTreeMap { } /// Utility methods for `WITH` properties (`HashMap` and `BTreeMap`). -pub trait WithPropertiesExt: Get { +pub trait WithPropertiesExt: Get + Sized { #[inline(always)] fn get_connector(&self) -> Option { self.get(UPSTREAM_SOURCE_KEY).map(|s| s.to_lowercase()) @@ -101,6 +102,10 @@ pub trait WithPropertiesExt: Get { connector.contains("-cdc") } + fn is_shared_cdc_source(&self) -> bool { + self.is_cdc_connector() && CdcTableType::from_properties(self).can_backfill() + } + #[inline(always)] fn is_iceberg_connector(&self) -> bool { let Some(connector) = self.get_connector() else { diff --git a/src/frontend/src/binder/relation/table_or_source.rs b/src/frontend/src/binder/relation/table_or_source.rs index a459efd39f01..2dd3f0fc8952 100644 --- a/src/frontend/src/binder/relation/table_or_source.rs +++ b/src/frontend/src/binder/relation/table_or_source.rs @@ -18,6 +18,7 @@ use itertools::Itertools; use risingwave_common::bail_not_implemented; use risingwave_common::catalog::{is_system_schema, Field}; use risingwave_common::session_config::USER_NAME_WILD_CARD; +use risingwave_connector::WithPropertiesExt; use risingwave_sqlparser::ast::{Statement, TableAlias}; use risingwave_sqlparser::parser::Parser; use thiserror_ext::AsReport; @@ -52,6 +53,12 @@ pub struct BoundSource { pub catalog: SourceCatalog, } +impl BoundSource { + pub fn is_shared_cdc_source(&self) -> bool { + self.catalog.with_properties.is_shared_cdc_source() + } +} + impl From<&SourceCatalog> for BoundSource { fn from(s: &SourceCatalog) -> Self { Self { catalog: s.clone() } diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 3585d58290c5..e5fd18badcaa 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -38,7 +38,6 @@ use risingwave_connector::schema::schema_registry::{ name_strategy_from_str, SchemaRegistryAuth, SCHEMA_REGISTRY_PASSWORD, SCHEMA_REGISTRY_USERNAME, }; use risingwave_connector::sink::iceberg::IcebergConfig; -use risingwave_connector::source::cdc::external::CdcTableType; use risingwave_connector::source::cdc::{ CDC_SHARING_MODE_KEY, CDC_SNAPSHOT_BACKFILL, CDC_SNAPSHOT_MODE_KEY, CDC_TRANSACTIONAL_KEY, CITUS_CDC_CONNECTOR, MONGODB_CDC_CONNECTOR, MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, @@ -1307,12 +1306,7 @@ pub async fn handle_create_source( ensure_table_constraints_supported(&stmt.constraints)?; let sql_pk_names = bind_sql_pk_names(&stmt.columns, &stmt.constraints)?; - // gated the feature with a session variable - let create_cdc_source_job = if with_properties.is_cdc_connector() { - CdcTableType::from_properties(&with_properties).can_backfill() - } else { - false - }; + let create_cdc_source_job = with_properties.is_shared_cdc_source(); let (columns_from_resolve_source, source_info) = if create_cdc_source_job { bind_columns_from_source_for_cdc(&session, &source_schema, &with_properties)? diff --git a/src/frontend/src/planner/relation.rs b/src/frontend/src/planner/relation.rs index 3f64a8fde440..f8f71d943121 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -86,12 +86,19 @@ impl Planner { } pub(super) fn plan_source(&mut self, source: BoundSource) -> Result { - Ok(LogicalSource::with_catalog( - Rc::new(source.catalog), - SourceNodeKind::CreateMViewOrBatch, - self.ctx(), - )? - .into()) + if source.is_shared_cdc_source() { + Err(ErrorCode::InternalError( + "Should not create MATERIALIZED VIEW directly on shared CDC source. HINT: create TABLE from the source instead.".to_string(), + ) + .into()) + } else { + Ok(LogicalSource::with_catalog( + Rc::new(source.catalog), + SourceNodeKind::CreateMViewOrBatch, + self.ctx(), + )? + .into()) + } } pub(super) fn plan_join(&mut self, join: BoundJoin) -> Result { From 55fde18071ff03853c2aef5d5f92ea778cbf8b4d Mon Sep 17 00:00:00 2001 From: xxchan Date: Tue, 2 Jan 2024 15:27:17 +0800 Subject: [PATCH 02/29] feat: add kafka backfill frontend --- .git-blame-ignore-revs | 3 + proto/catalog.proto | 13 +- src/common/src/monitor/connection.rs | 3 +- src/common/src/util/stream_graph_visitor.rs | 3 + .../src/parser/additional_columns.rs | 61 +++++ src/connector/src/source/base.rs | 4 + src/connector/src/source/cdc/mod.rs | 2 +- src/connector/src/source/reader/desc.rs | 74 +----- .../src/binder/relation/table_or_source.rs | 6 + src/frontend/src/catalog/source_catalog.rs | 8 + src/frontend/src/handler/create_source.rs | 14 +- .../src/optimizer/plan_node/logical_source.rs | 23 +- .../plan_node/logical_source_backfill.rs | 207 +++++++++++++++++ src/frontend/src/optimizer/plan_node/mod.rs | 15 +- .../plan_node/stream_cdc_table_scan.rs | 9 +- .../plan_node/stream_source_backfill.rs | 188 ++++++++++++++++ src/frontend/src/planner/relation.rs | 5 +- src/frontend/src/stream_fragmenter/mod.rs | 12 +- src/meta/src/barrier/command.rs | 37 ++- src/meta/src/barrier/mod.rs | 1 + src/meta/src/barrier/progress.rs | 8 + src/meta/src/barrier/schedule.rs | 2 + src/meta/src/controller/catalog.rs | 4 +- src/meta/src/controller/fragment.rs | 29 +-- src/meta/src/controller/streaming_job.rs | 3 +- src/meta/src/manager/catalog/database.rs | 6 +- src/meta/src/manager/catalog/fragment.rs | 23 ++ src/meta/src/manager/catalog/mod.rs | 9 +- src/meta/src/manager/metadata.rs | 16 ++ src/meta/src/model/stream.rs | 57 ++--- src/meta/src/rpc/ddl_controller.rs | 9 +- src/meta/src/stream/scale.rs | 3 +- src/meta/src/stream/source_manager.rs | 212 +++++++++++++++--- src/meta/src/stream/stream_graph/actor.rs | 41 +++- src/meta/src/stream/stream_graph/fragment.rs | 147 ++++++++---- src/meta/src/stream/stream_graph/schedule.rs | 1 + src/meta/src/stream/stream_manager.rs | 16 +- src/prost/src/lib.rs | 55 +++++ .../source/source_backfill_executor.rs | 15 +- .../source/source_backfill_state_table.rs | 1 - 40 files changed, 1096 insertions(+), 249 deletions(-) create mode 100644 src/frontend/src/optimizer/plan_node/logical_source_backfill.rs create mode 100644 src/frontend/src/optimizer/plan_node/stream_source_backfill.rs 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/proto/catalog.proto b/proto/catalog.proto index 67e71848e3d0..fae4d5ccc6f2 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -62,9 +62,16 @@ 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. - bool cdc_source_job = 13; + // Whether the stream source has a streaming job. + // This is related with [RFC: Reusable Source Executor](https://github.com/risingwavelabs/rfcs/pull/72). + // Currently, the following sources have streaming jobs: + // - Direct CDC sources (mysql & postgresql) + // - MQ sources (Kafka, Pulsar, Kinesis, etc.) + bool has_streaming_job = 13; + // Only used when `has_streaming_job` is `true`. + // If `false`, `requires_singleton` will be set in the stream plan. + bool is_distributed = 15; + reserved "cdc_source_job"; // deprecated // Options specified by user in the FORMAT ENCODE clause. map format_encode_options = 14; } diff --git a/src/common/src/monitor/connection.rs b/src/common/src/monitor/connection.rs index 2e28102bf507..5086cd36f81c 100644 --- a/src/common/src/monitor/connection.rs +++ b/src/common/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}; @@ -549,7 +550,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/util/stream_graph_visitor.rs b/src/common/src/util/stream_graph_visitor.rs index ce2820752f12..c9518a03c262 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/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index 06cc06156669..a4ace852a249 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -185,6 +185,67 @@ pub fn build_additional_column_catalog( Ok(catalog) } +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); + + // Check if partition/file/offset columns are included explicitly. + for col in columns { + use risingwave_pb::plan_common::additional_column::ColumnType; + 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 52724b170766..fc3c7c0f5f6e 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -73,8 +73,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) {} } @@ -371,10 +373,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..7d22ffd35a85 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.has_streaming_job; } } diff --git a/src/connector/src/source/reader/desc.rs b/src/connector/src/source/reader/desc.rs index 46107c2d73d0..16165944d048 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/frontend/src/binder/relation/table_or_source.rs b/src/frontend/src/binder/relation/table_or_source.rs index 2dd3f0fc8952..fa3a3ecefd7d 100644 --- a/src/frontend/src/binder/relation/table_or_source.rs +++ b/src/frontend/src/binder/relation/table_or_source.rs @@ -65,6 +65,12 @@ impl From<&SourceCatalog> for BoundSource { } } +impl BoundSource { + pub fn can_backfill(&self) -> bool { + self.catalog.info.has_streaming_job + } +} + impl Binder { /// Binds table or source, or logical view according to what we get from the catalog. pub fn bind_relation_by_name_inner( diff --git a/src/frontend/src/catalog/source_catalog.rs b/src/frontend/src/catalog/source_catalog.rs index 59f77bba9fa4..6e9ee89e8d2e 100644 --- a/src/frontend/src/catalog/source_catalog.rs +++ b/src/frontend/src/catalog/source_catalog.rs @@ -21,6 +21,7 @@ use risingwave_pb::catalog::{PbSource, StreamSourceInfo, WatermarkDesc}; use super::{ColumnId, ConnectionId, DatabaseId, OwnedByUserCatalog, SchemaId, SourceId}; use crate::catalog::TableId; +use crate::handler::create_source::UPSTREAM_SOURCE_KEY; use crate::user::UserId; /// This struct `SourceCatalog` is used in frontend. @@ -83,6 +84,13 @@ impl SourceCatalog { pub fn version(&self) -> SourceVersionId { self.version } + + pub fn connector_name(&self) -> String { + self.with_properties + .get(UPSTREAM_SOURCE_KEY) + .map(|s| s.to_lowercase()) + .unwrap() + } } impl From<&PbSource> for SourceCatalog { diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index e5fd18badcaa..f46353a17dd4 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -505,7 +505,7 @@ fn bind_columns_from_source_for_cdc( row_encode: row_encode_to_prost(&source_schema.row_encode) as i32, format_encode_options, use_schema_registry: json_schema_infer_use_schema_registry(&schema_config), - cdc_source_job: true, + has_streaming_job: true, ..Default::default() }; if !format_encode_options_to_consume.is_empty() { @@ -1307,12 +1307,17 @@ pub async fn handle_create_source( let sql_pk_names = bind_sql_pk_names(&stmt.columns, &stmt.constraints)?; let create_cdc_source_job = with_properties.is_shared_cdc_source(); + let has_streaming_job = create_cdc_source_job || with_properties.is_kafka_connector(); - 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 has_streaming_job { + source_info.has_streaming_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( @@ -1409,18 +1414,15 @@ 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 has_streaming_job { 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, context.into(), )?; - // 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/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index c155440ed32d..84533eec29b6 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -22,6 +22,8 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{ ColumnCatalog, ColumnDesc, Field, Schema, KAFKA_TIMESTAMP_COLUMN_NAME, }; +use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_connector::parser::additional_columns::add_partition_offset_cols; use risingwave_connector::source::iceberg::ICEBERG_CONNECTOR; use risingwave_connector::source::{DataType, UPSTREAM_SOURCE_KEY}; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; @@ -67,12 +69,27 @@ pub struct LogicalSource { impl LogicalSource { pub fn new( source_catalog: Option>, - column_catalog: Vec, + mut column_catalog: Vec, row_id_index: Option, kind: SourceNodeKind, ctx: OptimizerContextRef, ) -> Result { let kafka_timestamp_range = (Bound::Unbounded, Bound::Unbounded); + + // for sources with streaming job, we will include partition and offset cols in the output. + if let Some(source_catalog) = &source_catalog + && matches!(kind, SourceNodeKind::CreateSourceWithStreamjob) + { + let (columns_exist, additional_columns) = + add_partition_offset_cols(&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 { + column_catalog.push(c); + } + } + } + let core = generic::Source { catalog: source_catalog, column_catalog, @@ -503,9 +520,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 + // Note: for create table, row_id and generated columns is created in plan_root.gen_table_plan. + // for backfill-able 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(); diff --git a/src/frontend/src/optimizer/plan_node/logical_source_backfill.rs b/src/frontend/src/optimizer/plan_node/logical_source_backfill.rs new file mode 100644 index 000000000000..4a376794af57 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/logical_source_backfill.rs @@ -0,0 +1,207 @@ +// 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::ops::Bound; +use std::rc::Rc; + +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_connector::parser::additional_columns::add_partition_offset_cols; + +use super::generic::{GenericPlanRef, SourceNodeKind}; +use super::stream_watermark_filter::StreamWatermarkFilter; +use super::utils::{childless_record, Distill}; +use super::{ + generic, BatchProject, BatchSource, ColPrunable, ExprRewritable, Logical, LogicalFilter, + LogicalProject, LogicalSource, PlanBase, PlanRef, PredicatePushdown, StreamProject, + StreamRowIdGen, ToBatch, ToStream, +}; +use crate::catalog::source_catalog::SourceCatalog; +use crate::error::Result; +use crate::expr::{ExprImpl, ExprRewriter, ExprVisitor}; +use crate::optimizer::optimizer_context::OptimizerContextRef; +use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::plan_node::utils::column_names_pretty; +use crate::optimizer::plan_node::{ + ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, StreamSourceBackfill, + ToStreamContext, +}; +use crate::optimizer::property::Distribution::HashShard; +use crate::utils::{ColIndexMapping, Condition}; + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct LogicalSourceBackfill { + pub base: PlanBase, + pub core: generic::Source, + + /// Expressions to output. This field presents and will be turned to a `Project` when + /// converting to a physical plan, only if there are generated columns. + output_exprs: Option>, + /// When there are generated columns, the `StreamRowIdGen`'s row_id_index is different from + /// the one in `core`. So we store the one in `output_exprs` here. + output_row_id_index: Option, +} + +impl LogicalSourceBackfill { + pub fn new(source_catalog: Rc, ctx: OptimizerContextRef) -> Result { + let mut column_catalog = source_catalog.columns.clone(); + let row_id_index = source_catalog.row_id_index; + + let kafka_timestamp_range = (Bound::Unbounded, Bound::Unbounded); + + let (columns_exist, additional_columns) = + add_partition_offset_cols(&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 { + column_catalog.push(c); + } + } + let core = generic::Source { + catalog: Some(source_catalog), + column_catalog, + row_id_index, + // FIXME: this field is not useful for backfill. + kind: SourceNodeKind::CreateMViewOrBatch, + ctx, + kafka_timestamp_range, + }; + + let base = PlanBase::new_logical_with_core(&core); + + let output_exprs = + LogicalSource::derive_output_exprs_from_generated_columns(&core.column_catalog)?; + let (core, output_row_id_index) = core.exclude_generated_columns(); + + Ok(LogicalSourceBackfill { + base, + core, + output_exprs, + output_row_id_index, + }) + } + + pub fn source_catalog(&self) -> Rc { + self.core + .catalog + .clone() + .expect("source catalog should exist for LogicalSourceBackfill") + } +} + +impl_plan_tree_node_for_leaf! {LogicalSourceBackfill} +impl Distill for LogicalSourceBackfill { + fn distill<'a>(&self) -> XmlNode<'a> { + let src = Pretty::from(self.source_catalog().name.clone()); + let time = Pretty::debug(&self.core.kafka_timestamp_range); + let fields = vec![ + ("source", src), + ("columns", column_names_pretty(self.schema())), + ("time_range", time), + ]; + + childless_record("LogicalSourceBackfill", fields) + } +} + +impl ColPrunable for LogicalSourceBackfill { + fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef { + let mapping = ColIndexMapping::with_remaining_columns(required_cols, self.schema().len()); + LogicalProject::with_mapping(self.clone().into(), mapping).into() + } +} + +impl ExprRewritable for LogicalSourceBackfill { + fn has_rewritable_expr(&self) -> bool { + self.output_exprs.is_some() + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut output_exprs = self.output_exprs.clone(); + + for expr in output_exprs.iter_mut().flatten() { + *expr = r.rewrite_expr(expr.clone()); + } + + Self { + output_exprs, + ..self.clone() + } + .into() + } +} + +impl ExprVisitable for LogicalSourceBackfill { + fn visit_exprs(&self, v: &mut dyn ExprVisitor) { + self.output_exprs + .iter() + .flatten() + .for_each(|e| v.visit_expr(e)); + } +} + +impl PredicatePushdown for LogicalSourceBackfill { + fn predicate_pushdown( + &self, + predicate: Condition, + _ctx: &mut PredicatePushdownContext, + ) -> PlanRef { + LogicalFilter::create(self.clone().into(), predicate) + } +} + +impl ToBatch for LogicalSourceBackfill { + fn to_batch(&self) -> Result { + let mut plan: PlanRef = BatchSource::new(self.core.clone()).into(); + + if let Some(exprs) = &self.output_exprs { + let logical_project = generic::Project::new(exprs.to_vec(), plan); + plan = BatchProject::new(logical_project).into(); + } + + Ok(plan) + } +} + +impl ToStream for LogicalSourceBackfill { + fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result { + let mut plan = StreamSourceBackfill::new(self.core.clone()).into(); + + if let Some(exprs) = &self.output_exprs { + let logical_project = generic::Project::new(exprs.to_vec(), plan); + plan = StreamProject::new(logical_project).into(); + } + + let catalog = self.source_catalog(); + if !catalog.watermark_descs.is_empty() { + plan = StreamWatermarkFilter::new(plan, catalog.watermark_descs.clone()).into(); + } + + if let Some(row_id_index) = self.output_row_id_index { + plan = StreamRowIdGen::new_with_dist(plan, row_id_index, HashShard(vec![row_id_index])) + .into(); + } + Ok(plan) + } + + fn logical_rewrite_for_stream( + &self, + _ctx: &mut RewriteStreamContext, + ) -> Result<(PlanRef, ColIndexMapping)> { + Ok(( + self.clone().into(), + ColIndexMapping::identity(self.schema().len()), + )) + } +} diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index e4dfb0e8f2fe..0b6e007b06b1 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -676,8 +676,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, @@ -690,6 +690,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_backfill) = self.as_stream_source_backfill() { + return stream_source_backfill.adhoc_to_stream_prost(state); + } if let Some(stream_share) = self.as_stream_share() { return stream_share.adhoc_to_stream_prost(state); } @@ -824,6 +827,7 @@ mod logical_project_set; mod logical_scan; mod logical_share; mod logical_source; +mod logical_source_backfill; mod logical_sys_scan; mod logical_table_function; mod logical_topn; @@ -853,6 +857,7 @@ mod stream_simple_agg; mod stream_sink; mod stream_sort; mod stream_source; +mod stream_source_backfill; mod stream_stateless_simple_agg; mod stream_table_scan; mod stream_topn; @@ -915,6 +920,7 @@ pub use logical_project_set::LogicalProjectSet; pub use logical_scan::LogicalScan; pub use logical_share::LogicalShare; pub use logical_source::LogicalSource; +pub use logical_source_backfill::LogicalSourceBackfill; pub use logical_sys_scan::LogicalSysScan; pub use logical_table_function::LogicalTableFunction; pub use logical_topn::LogicalTopN; @@ -946,6 +952,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_backfill::StreamSourceBackfill; pub use stream_stateless_simple_agg::StreamStatelessSimpleAgg; pub use stream_table_scan::StreamTableScan; pub use stream_temporal_join::StreamTemporalJoin; @@ -987,6 +994,7 @@ macro_rules! for_all_plan_nodes { , { Logical, CdcScan } , { Logical, SysScan } , { Logical, Source } + , { Logical, SourceBackfill } , { Logical, Insert } , { Logical, Delete } , { Logical, Update } @@ -1040,6 +1048,7 @@ macro_rules! for_all_plan_nodes { , { Stream, CdcTableScan } , { Stream, Sink } , { Stream, Source } + , { Stream, SourceBackfill } , { Stream, HashJoin } , { Stream, Exchange } , { Stream, HashAgg } @@ -1083,6 +1092,7 @@ macro_rules! for_logical_plan_nodes { , { Logical, CdcScan } , { Logical, SysScan } , { Logical, Source } + , { Logical, SourceBackfill } , { Logical, Insert } , { Logical, Delete } , { Logical, Update } @@ -1156,6 +1166,7 @@ macro_rules! for_stream_plan_nodes { , { Stream, CdcTableScan } , { Stream, Sink } , { Stream, Source } + , { Stream, SourceBackfill } , { 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_backfill.rs b/src/frontend/src/optimizer/plan_node/stream_source_backfill.rs new file mode 100644 index 000000000000..fd27d693e86d --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/stream_source_backfill.rs @@ -0,0 +1,188 @@ +// 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::sort_util::OrderType; +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}; + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct StreamSourceBackfill { + pub base: PlanBase, + core: generic::Source, +} + +impl_plan_tree_node_for_leaf! { StreamSourceBackfill } + +impl StreamSourceBackfill { + pub fn new(source: generic::Source) -> Self { + let base = PlanBase::new_stream_with_core( + &source, + Distribution::SomeShard, + source.catalog.as_ref().map_or(true, |s| s.append_only), + false, + FixedBitSet::with_capacity(source.column_catalog.len()), + ); + + Self { base, core: source } + } + + 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 backfill 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 source_inner = SourceBackfillNode { + 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()), + // XXX: what's the usage of this? + 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 stream_scan_body = PbNodeBody::SourceBackfill(source_inner); + + 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(stream_scan_body), + stream_key, + operator_id: self.base.id().0 as u64, + identity: self.distill_to_string(), + append_only: self.append_only(), + }) + } +} + +impl Distill for StreamSourceBackfill { + 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("StreamSourceBackfill", vec![("columns", col)]) + } +} + +impl ExprRewritable for StreamSourceBackfill {} + +impl ExprVisitable for StreamSourceBackfill {} + +impl StreamNode for StreamSourceBackfill { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> NodeBody { + unreachable!("stream source backfill 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 f8f71d943121..9a18752030e2 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -28,7 +28,8 @@ use crate::expr::{Expr, ExprImpl, ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::generic::SourceNodeKind; use crate::optimizer::plan_node::{ LogicalApply, LogicalHopWindow, LogicalJoin, LogicalProject, LogicalScan, LogicalShare, - LogicalSource, LogicalSysScan, LogicalTableFunction, LogicalValues, PlanRef, + LogicalSource, LogicalSourceBackfill, LogicalSysScan, LogicalTableFunction, LogicalValues, + PlanRef, }; use crate::optimizer::property::Cardinality; use crate::planner::Planner; @@ -91,6 +92,8 @@ impl Planner { "Should not create MATERIALIZED VIEW directly on shared CDC source. HINT: create TABLE from the source instead.".to_string(), ) .into()) + } else if source.can_backfill() { + Ok(LogicalSourceBackfill::new(Rc::new(source.catalog), self.ctx())?.into()) } else { Ok(LogicalSource::with_catalog( Rc::new(source.catalog), diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index a3d18a2c6dc1..87cb9a6703f1 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.has_streaming_job + && !source_info.is_distributed { - tracing::debug!("mark cdc source job as singleton"); current_fragment.requires_singleton = true; } } @@ -294,6 +294,7 @@ fn build_fragment( } NodeBody::StreamCdcScan(_) => { + // XXX: Should we use a different flag for CDC scan? current_fragment.fragment_type_mask |= FragmentTypeFlag::StreamScan as u32; // the backfill algorithm is not parallel safe current_fragment.requires_singleton = true; @@ -309,6 +310,13 @@ fn build_fragment( .upstream_table_ids .push(node.upstream_source_id); } + NodeBody::SourceBackfill(node) => { + current_fragment.fragment_type_mask |= FragmentTypeFlag::SourceBackfill as u32; + // memorize upstream source id for later use + let source_id = node.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 22311a2b4391..9c74d2656d19 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; @@ -691,24 +691,11 @@ impl CommandContext { 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(), } } @@ -789,7 +776,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; } @@ -952,11 +939,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, ) @@ -1020,10 +1008,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/mod.rs b/src/meta/src/barrier/mod.rs index 652a4b51d926..213f001906ba 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -925,6 +925,7 @@ impl GlobalBarrierManagerContext { } commands }; + tracing::trace!("finished_commands: {}", finished_commands.len()); for command in finished_commands { tracker.stash_command_to_finish(command); diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index 1fcdace3f28b..83ad0c4bf794 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -205,6 +205,11 @@ impl TrackingJob { pub(crate) fn notify_finished(self) { match self { TrackingJob::New(command) => { + tracing::trace!( + "notify finished, command: {:?}, curr_epoch: {:?}", + command.context.command, + command.context.curr_epoch + ); command .notifiers .into_iter() @@ -405,6 +410,7 @@ impl CreateMviewProgressTracker { version_stats: &HummockVersionStats, ) -> Option { let actors = command.context.actors_to_track(); + tracing::trace!("add actors to track: {:?}", actors); if actors.is_empty() { // The command can be finished immediately. return Some(TrackingJob::New(command)); @@ -463,6 +469,7 @@ impl CreateMviewProgressTracker { upstream_total_key_count, definition, ); + tracing::trace!("add progress: {:?}", progress); if *ddl_type == DdlType::Sink { // We return the original tracking job immediately. // This is because sink can be decoupled with backfill progress. @@ -487,6 +494,7 @@ impl CreateMviewProgressTracker { progress: &CreateMviewProgress, version_stats: &HummockVersionStats, ) -> Option { + tracing::trace!("update progress: {:?}", progress); let actor = progress.backfill_actor_id; let Some(table_id) = self.actor_map.get(&actor).copied() else { // On restart, backfill will ALWAYS notify CreateMviewProgressTracker, diff --git a/src/meta/src/barrier/schedule.rs b/src/meta/src/barrier/schedule.rs index f282f99061b4..6467e878ae13 100644 --- a/src/meta/src/barrier/schedule.rs +++ b/src/meta/src/barrier/schedule.rs @@ -276,9 +276,11 @@ impl BarrierScheduler { let mut infos = Vec::with_capacity(contexts.len()); for (injected_rx, collect_rx, finish_rx) in contexts { + tracing::trace!("waiting for command to be injected"); // Wait for this command to be injected, and record the result. let info = injected_rx.await.ok().context("failed to inject barrier")?; infos.push(info); + tracing::trace!("injected_rx finished"); // Throw the error if it occurs when collecting this barrier. collect_rx diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index 9c85afc24140..a0700fa2d70c 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -1970,7 +1970,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() @@ -1980,7 +1980,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().has_streaming_job { to_drop_streaming_jobs.push(object_id); } diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index c9beada284d5..67f3ce5363e9 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; @@ -1204,9 +1203,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); } @@ -1214,31 +1213,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::SourceBackfill 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 d929f76bb8f8..cdd489779a2d 100644 --- a/src/meta/src/controller/streaming_job.rs +++ b/src/meta/src/controller/streaming_job.rs @@ -816,7 +816,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().has_streaming_job { vec![source_id] } else { @@ -851,6 +851,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..9fc6b04bd01c 100644 --- a/src/meta/src/manager/catalog/database.rs +++ b/src/meta/src/manager/catalog/database.rs @@ -402,11 +402,13 @@ 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) + source + .info + .as_ref() + .is_some_and(|info| info.has_streaming_job) }) .map(|(id, _)| id) .copied(), diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index 4cc95a2352a7..38edf3a27a66 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -1027,6 +1027,29 @@ impl FragmentManager { bail!("fragment not found: {}", fragment_id) } + pub async fn get_running_actors_and_upstream_fragment_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 3db6828df3b0..50860030029a 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -1168,7 +1168,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 @@ -1452,11 +1452,10 @@ impl CatalogManager { continue; } - // cdc source streaming job if let Some(info) = source.info - && info.cdc_source_job + && info.has_streaming_job { - 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?; @@ -1779,7 +1778,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 66d3fcc9a41e..ca8fc3cf2ad9 100644 --- a/src/meta/src/manager/metadata.rs +++ b/src/meta/src/manager/metadata.rs @@ -613,6 +613,22 @@ 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_fragment_of_fragment(id) + .await + } + MetadataManager::V2(_mgr) => { + todo!() + } + } + } + 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 ef55f78493f8..da1a310ee8d8 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,10 +367,12 @@ impl TableFragments { }) } - /// Returns values actor ids. - pub fn values_actor_ids(&self) -> Vec { + /// Returns actor ids that need to be tracked when creating MV. + pub fn tracking_progress_actor_ids(&self) -> Vec { Self::filter_actor_ids(self, |fragment_type_mask| { - (fragment_type_mask & FragmentTypeFlag::Values as u32) != 0 + (fragment_type_mask + & (FragmentTypeFlag::Values as u32 | FragmentTypeFlag::StreamScan as u32)) + != 0 }) } @@ -411,23 +413,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 +420,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 +433,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 65f94e550da0..61a9ec7a64ce 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,8 +1361,12 @@ impl DdlController { .get_upstream_root_fragments(fragment_graph.dependent_table_ids()) .await?; - let upstream_actors: HashMap<_, _> = upstream_root_fragments + // XXX: do we need to filter here? + let upstream_mview_actors: HashMap<_, _> = upstream_root_fragments .iter() + // .filter(|(_, fragment)| { + // fragment.fragment_type_mask & FragmentTypeFlag::Mview as u32 != 0 + // }) .map(|(&table_id, fragment)| { ( table_id, @@ -1460,7 +1465,7 @@ impl DdlController { let ctx = CreateStreamingJobContext { dispatchers, - upstream_mview_actors: upstream_actors, + upstream_mview_actors, internal_tables, building_locations, existing_locations, diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index 99ae32d26bb9..88f4be1cf81d 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); } } @@ -1738,6 +1738,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 e48d1ae8bcb1..930f6a9d2ada 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 risingwave_rpc_client::ConnectorClient; use thiserror_ext::AsReport; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; @@ -230,8 +231,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>, @@ -242,20 +243,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, } } @@ -269,12 +264,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); @@ -283,20 +279,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| { ( @@ -310,14 +312,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!("Failed to get the actor of the fragment {}, maybe the fragment doesn't exist anymore", err.to_string()); + continue; + } + }; + split_assignment.insert( + *fragment_id, + align_backfill_splits( + actors, + upstream_assignment, + *fragment_id, + *upstream_fragment_id, + )?, + ); } } } @@ -327,26 +366,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); } } @@ -375,10 +420,6 @@ impl SourceManagerCore { entry.remove(); } } - - for fragment_id in &fragment_ids { - self.fragment_sources.remove(fragment_id); - } } for actor_id in removed_actors { @@ -434,6 +475,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 @@ -546,6 +591,32 @@ 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 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); @@ -571,6 +642,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) => { @@ -582,6 +654,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()); } } @@ -598,6 +671,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() @@ -622,6 +710,7 @@ impl SourceManager { metadata_manager, managed_sources, source_fragments, + backfill_fragments, actor_splits, )); @@ -671,12 +760,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. @@ -778,6 +873,63 @@ impl SourceManager { Ok(assigned) } + 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..52a14af0629c 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) } @@ -254,6 +254,44 @@ impl ActorBuilder { }) } + // "Leaf" node `SourceBackfill`. + NodeBody::SourceBackfill(source_backfill) => { + 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 = source_backfill.source_id; + + // Index the upstreams by the an external edge ID. + let upstreams = &self.upstreams[&EdgeId::UpstreamExternal { + upstream_table_id: upstream_source_id.into(), + downstream_fragment_id: self.fragment_id, + }]; + + let upstream_actor_id = upstreams.actors.as_global_ids(); + + // rewrite the input of `SourceBackfill` + let input = vec![ + // Fill the merge node body with correct upstream info. + StreamNode { + node_body: Some(NodeBody::Merge(MergeNode { + upstream_actor_id, + upstream_fragment_id: upstreams.fragment_id.as_global_id(), + upstream_dispatcher_type: DispatcherType::NoShuffle as _, + fields: merge_node.fields.clone(), + })), + ..merge_node.clone() + }, + ]; + + Ok(StreamNode { + input, + ..stream_node.clone() + }) + } + // For other nodes, visit the children recursively. _ => { let mut new_stream_node = stream_node.clone(); @@ -622,6 +660,7 @@ impl ActorGraphBuildState { /// The result of a built actor graph. Will be further embedded into the `Context` for building /// actors on the compute nodes. +#[derive(Debug)] pub struct ActorGraphBuildResult { /// The graph of sealed fragments, including all actors. pub graph: BTreeMap, diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index fbe250818e6e..22b6c7a4a2a9 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.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 ); @@ -291,7 +301,7 @@ impl StreamFragmentEdge { /// This only includes nodes and edges of the current job itself. It will be converted to [`CompleteStreamFragmentGraph`] later, /// that contains the additional information of pre-existing /// fragments, which are connected to the graph's top-most or bottom-most fragments. -#[derive(Default)] +#[derive(Default, Debug)] pub struct StreamFragmentGraph { /// stores all the fragments in the graph. fragments: HashMap, @@ -518,7 +528,7 @@ pub(super) enum EitherFragment { /// An internal fragment that is being built for the current streaming job. Building(BuildingFragment), - /// An existing fragment that is external but connected to the fragments being built. + /// An existing fragment that is external but connected to the fragments being built.!!!!!!!!!!!!! Existing(Fragment), } @@ -530,6 +540,7 @@ pub(super) enum EitherFragment { /// `Materialize` node will be included in this structure. /// - if we're going to replace the plan of a table with downstream mviews, the downstream fragments /// containing the `StreamScan` nodes will be included in this structure. +#[derive(Debug)] pub struct CompleteStreamFragmentGraph { /// The fragment graph of the streaming job being built. building_graph: StreamFragmentGraph, @@ -663,50 +674,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_graph/schedule.rs b/src/meta/src/stream/stream_graph/schedule.rs index ed2dac5be0e0..1ae24ec1b7d5 100644 --- a/src/meta/src/stream/stream_graph/schedule.rs +++ b/src/meta/src/stream/stream_graph/schedule.rs @@ -326,6 +326,7 @@ impl Scheduler { /// [`Locations`] represents the parallel unit and worker locations of the actors. #[cfg_attr(test, derive(Default))] +#[derive(Debug)] pub struct Locations { /// actor location map. pub actor_locations: BTreeMap, diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index fa16b039236b..e451a6dab893 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -456,7 +456,7 @@ impl GlobalStreamManager { } let dummy_table_id = table_fragments.table_id(); - + // TODO: does this need change? for replace_table let init_split_assignment = self.source_manager.allocate_splits(&dummy_table_id).await?; @@ -473,7 +473,16 @@ 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: + // - Source with streaming job (backfill-able source) + // - Table with connector + // - MV on backfill-able 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, @@ -484,7 +493,6 @@ impl GlobalStreamManager { ddl_type, replace_table: replace_table_command, }; - 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)); @@ -519,7 +527,7 @@ impl GlobalStreamManager { .await?; let dummy_table_id = table_fragments.table_id(); - + // TODO: does this need change? for replace_table 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..775394668977 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -196,6 +196,61 @@ 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.source_id); + } + + for child in &self.input { + if let Some(source) = child.find_source_backfill() { + return Some(source); + } + } + + None + } +} + #[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 7109538c38fe..17849ecd7768 100644 --- a/src/stream/src/executor/source/source_backfill_executor.rs +++ b/src/stream/src/executor/source/source_backfill_executor.rs @@ -611,11 +611,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) => { @@ -725,11 +725,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()); @@ -743,7 +738,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())? From 2285c1ffeea14942041baeafbf42c825ed903705 Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 28 Feb 2024 14:22:38 +0800 Subject: [PATCH 03/29] add config for easy of testing --- Makefile.toml | 4 ++-- src/common/src/lib.rs | 2 ++ src/common/src/session_config/mod.rs | 4 ++++ src/common/src/system_param/mod.rs | 3 ++- src/connector/src/source/kafka/private_link.rs | 2 +- src/frontend/src/handler/create_source.rs | 2 +- src/frontend/src/planner/relation.rs | 4 +++- 7 files changed, 15 insertions(+), 6 deletions(-) diff --git a/Makefile.toml b/Makefile.toml index ed39ea0fc942..494db8903aec 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/src/common/src/lib.rs b/src/common/src/lib.rs index 4df0999828c8..fd6a30f56bbd 100644 --- a/src/common/src/lib.rs +++ b/src/common/src/lib.rs @@ -44,6 +44,8 @@ #![feature(bound_map)] #![feature(array_methods)] #![feature(btree_cursors)] +#![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 83e8d8542960..83a3e83e492a 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -243,6 +243,10 @@ pub struct ConfigMap { #[parameter(default = false)] background_ddl: bool, + /// Run DDL statements in background + #[parameter(default = false)] + enable_reusable_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/connector/src/source/kafka/private_link.rs b/src/connector/src/source/kafka/private_link.rs index d2e3d6877d16..7d49feac097b 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/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index f46353a17dd4..91d106c83839 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -1414,7 +1414,7 @@ pub async fn handle_create_source( let catalog_writer = session.catalog_writer()?; - if has_streaming_job { + if has_streaming_job && session.config().enable_reusable_source() { let graph = { let context = OptimizerContext::from_handler_args(handler_args); let source_node = LogicalSource::with_catalog( diff --git a/src/frontend/src/planner/relation.rs b/src/frontend/src/planner/relation.rs index 9a18752030e2..dd4be752ffb6 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -92,7 +92,9 @@ impl Planner { "Should not create MATERIALIZED VIEW directly on shared CDC source. HINT: create TABLE from the source instead.".to_string(), ) .into()) - } else if source.can_backfill() { + } else if source.can_backfill() + && self.ctx().session_ctx().config().enable_reusable_source() + { Ok(LogicalSourceBackfill::new(Rc::new(source.catalog), self.ctx())?.into()) } else { Ok(LogicalSource::with_catalog( From 6aa8502a534899929b01fc952d4fff9dff28b7f3 Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 13 Mar 2024 10:26:48 +0800 Subject: [PATCH 04/29] cleanup silly issues --- e2e_test/batch/catalog/pg_settings.slt.part | 1 + src/common/src/session_config/mod.rs | 4 ++-- src/frontend/src/binder/relation/table_or_source.rs | 10 ++++------ src/frontend/src/catalog/source_catalog.rs | 7 +++---- src/frontend/src/handler/create_source.rs | 2 +- .../src/optimizer/plan_node/stream_source_backfill.rs | 6 ++---- src/frontend/src/planner/relation.rs | 6 +++++- src/meta/src/barrier/command.rs | 2 +- src/meta/src/barrier/mod.rs | 1 - src/meta/src/barrier/progress.rs | 8 -------- src/meta/src/barrier/schedule.rs | 2 -- src/meta/src/stream/source_manager.rs | 2 +- src/meta/src/stream/stream_graph/actor.rs | 1 - src/meta/src/stream/stream_graph/fragment.rs | 5 ++--- src/meta/src/stream/stream_graph/schedule.rs | 1 - 15 files changed, 22 insertions(+), 36 deletions(-) diff --git a/e2e_test/batch/catalog/pg_settings.slt.part b/e2e_test/batch/catalog/pg_settings.slt.part index 9ff41b8dbea4..c1b88b547361 100644 --- a/e2e_test/batch/catalog/pg_settings.slt.part +++ b/e2e_test/batch/catalog/pg_settings.slt.part @@ -36,6 +36,7 @@ user rw_batch_enable_sort_agg user rw_enable_join_ordering user rw_enable_share_plan user rw_enable_two_phase_agg +user rw_enable_reusable_source user rw_force_split_distinct_agg user rw_force_two_phase_agg user rw_implicit_flush diff --git a/src/common/src/session_config/mod.rs b/src/common/src/session_config/mod.rs index 83a3e83e492a..2e2629ddf7ca 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -243,9 +243,9 @@ pub struct ConfigMap { #[parameter(default = false)] background_ddl: bool, - /// Run DDL statements in background + /// Enable reusable source. Currently only for Kafka. #[parameter(default = false)] - enable_reusable_source: bool, + rw_enable_reusable_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)] diff --git a/src/frontend/src/binder/relation/table_or_source.rs b/src/frontend/src/binder/relation/table_or_source.rs index fa3a3ecefd7d..8f3bb1a84ed4 100644 --- a/src/frontend/src/binder/relation/table_or_source.rs +++ b/src/frontend/src/binder/relation/table_or_source.rs @@ -57,6 +57,10 @@ impl BoundSource { pub fn is_shared_cdc_source(&self) -> bool { self.catalog.with_properties.is_shared_cdc_source() } + + pub fn can_backfill(&self) -> bool { + self.catalog.info.has_streaming_job + } } impl From<&SourceCatalog> for BoundSource { @@ -65,12 +69,6 @@ impl From<&SourceCatalog> for BoundSource { } } -impl BoundSource { - pub fn can_backfill(&self) -> bool { - self.catalog.info.has_streaming_job - } -} - impl Binder { /// Binds table or source, or logical view according to what we get from the catalog. pub fn bind_relation_by_name_inner( diff --git a/src/frontend/src/catalog/source_catalog.rs b/src/frontend/src/catalog/source_catalog.rs index 6e9ee89e8d2e..17292b1324ed 100644 --- a/src/frontend/src/catalog/source_catalog.rs +++ b/src/frontend/src/catalog/source_catalog.rs @@ -16,12 +16,12 @@ 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}; use super::{ColumnId, ConnectionId, DatabaseId, OwnedByUserCatalog, SchemaId, SourceId}; use crate::catalog::TableId; -use crate::handler::create_source::UPSTREAM_SOURCE_KEY; use crate::user::UserId; /// This struct `SourceCatalog` is used in frontend. @@ -87,9 +87,8 @@ impl SourceCatalog { pub fn connector_name(&self) -> String { self.with_properties - .get(UPSTREAM_SOURCE_KEY) - .map(|s| s.to_lowercase()) - .unwrap() + .get_connector() + .expect("connector name is missing") } } diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 91d106c83839..53d9cd5cb21f 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -1414,7 +1414,7 @@ pub async fn handle_create_source( let catalog_writer = session.catalog_writer()?; - if has_streaming_job && session.config().enable_reusable_source() { + if has_streaming_job && session.config().rw_enable_reusable_source() { let graph = { let context = OptimizerContext::from_handler_args(handler_args); let source_node = LogicalSource::with_catalog( diff --git a/src/frontend/src/optimizer/plan_node/stream_source_backfill.rs b/src/frontend/src/optimizer/plan_node/stream_source_backfill.rs index fd27d693e86d..ed7702e12250 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source_backfill.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source_backfill.rs @@ -119,7 +119,7 @@ impl StreamSourceBackfill { .collect_vec(); let source_catalog = self.source_catalog(); - let source_inner = SourceBackfillNode { + let backfill = SourceBackfillNode { source_id: source_catalog.id, source_name: source_catalog.name.clone(), state_table: Some( @@ -140,8 +140,6 @@ impl StreamSourceBackfill { rate_limit: self.base.ctx().overwrite_options().streaming_rate_limit, }; - let stream_scan_body = PbNodeBody::SourceBackfill(source_inner); - let fields = self.schema().to_prost(); // plan: merge -> backfill Ok(PbStreamNode { @@ -156,7 +154,7 @@ impl StreamSourceBackfill { ..Default::default() }, ], - node_body: Some(stream_scan_body), + node_body: Some(PbNodeBody::SourceBackfill(backfill)), stream_key, operator_id: self.base.id().0 as u64, identity: self.distill_to_string(), diff --git a/src/frontend/src/planner/relation.rs b/src/frontend/src/planner/relation.rs index dd4be752ffb6..9d30ec698b76 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -93,7 +93,11 @@ impl Planner { ) .into()) } else if source.can_backfill() - && self.ctx().session_ctx().config().enable_reusable_source() + && self + .ctx() + .session_ctx() + .config() + .rw_enable_reusable_source() { Ok(LogicalSourceBackfill::new(Rc::new(source.catalog), self.ctx())?.into()) } else { diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 9c74d2656d19..18cc69217c91 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -686,7 +686,7 @@ 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 { diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 213f001906ba..652a4b51d926 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -925,7 +925,6 @@ impl GlobalBarrierManagerContext { } commands }; - tracing::trace!("finished_commands: {}", finished_commands.len()); for command in finished_commands { tracker.stash_command_to_finish(command); diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index 83ad0c4bf794..1fcdace3f28b 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -205,11 +205,6 @@ impl TrackingJob { pub(crate) fn notify_finished(self) { match self { TrackingJob::New(command) => { - tracing::trace!( - "notify finished, command: {:?}, curr_epoch: {:?}", - command.context.command, - command.context.curr_epoch - ); command .notifiers .into_iter() @@ -410,7 +405,6 @@ impl CreateMviewProgressTracker { version_stats: &HummockVersionStats, ) -> Option { let actors = command.context.actors_to_track(); - tracing::trace!("add actors to track: {:?}", actors); if actors.is_empty() { // The command can be finished immediately. return Some(TrackingJob::New(command)); @@ -469,7 +463,6 @@ impl CreateMviewProgressTracker { upstream_total_key_count, definition, ); - tracing::trace!("add progress: {:?}", progress); if *ddl_type == DdlType::Sink { // We return the original tracking job immediately. // This is because sink can be decoupled with backfill progress. @@ -494,7 +487,6 @@ impl CreateMviewProgressTracker { progress: &CreateMviewProgress, version_stats: &HummockVersionStats, ) -> Option { - tracing::trace!("update progress: {:?}", progress); let actor = progress.backfill_actor_id; let Some(table_id) = self.actor_map.get(&actor).copied() else { // On restart, backfill will ALWAYS notify CreateMviewProgressTracker, diff --git a/src/meta/src/barrier/schedule.rs b/src/meta/src/barrier/schedule.rs index 6467e878ae13..f282f99061b4 100644 --- a/src/meta/src/barrier/schedule.rs +++ b/src/meta/src/barrier/schedule.rs @@ -276,11 +276,9 @@ impl BarrierScheduler { let mut infos = Vec::with_capacity(contexts.len()); for (injected_rx, collect_rx, finish_rx) in contexts { - tracing::trace!("waiting for command to be injected"); // Wait for this command to be injected, and record the result. let info = injected_rx.await.ok().context("failed to inject barrier")?; infos.push(info); - tracing::trace!("injected_rx finished"); // Throw the error if it occurs when collecting this barrier. collect_rx diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index 930f6a9d2ada..a42ea7dba5a8 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -344,7 +344,7 @@ impl SourceManagerCore { actors } Err(err) => { - tracing::warn!("Failed to get the actor of the fragment {}, maybe the fragment doesn't exist anymore", err.to_string()); + tracing::warn!(error = %err.as_report(),"Failed to get the actor of the fragment, maybe the fragment doesn't exist anymore"); continue; } }; diff --git a/src/meta/src/stream/stream_graph/actor.rs b/src/meta/src/stream/stream_graph/actor.rs index 52a14af0629c..e4aac7ea1991 100644 --- a/src/meta/src/stream/stream_graph/actor.rs +++ b/src/meta/src/stream/stream_graph/actor.rs @@ -660,7 +660,6 @@ impl ActorGraphBuildState { /// The result of a built actor graph. Will be further embedded into the `Context` for building /// actors on the compute nodes. -#[derive(Debug)] pub struct ActorGraphBuildResult { /// The graph of sealed fragments, including all actors. pub graph: BTreeMap, diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index 22b6c7a4a2a9..548203bbf1e7 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -301,7 +301,7 @@ impl StreamFragmentEdge { /// This only includes nodes and edges of the current job itself. It will be converted to [`CompleteStreamFragmentGraph`] later, /// that contains the additional information of pre-existing /// fragments, which are connected to the graph's top-most or bottom-most fragments. -#[derive(Default, Debug)] +#[derive(Default)] pub struct StreamFragmentGraph { /// stores all the fragments in the graph. fragments: HashMap, @@ -528,7 +528,7 @@ pub(super) enum EitherFragment { /// An internal fragment that is being built for the current streaming job. Building(BuildingFragment), - /// An existing fragment that is external but connected to the fragments being built.!!!!!!!!!!!!! + /// An existing fragment that is external but connected to the fragments being built. Existing(Fragment), } @@ -540,7 +540,6 @@ pub(super) enum EitherFragment { /// `Materialize` node will be included in this structure. /// - if we're going to replace the plan of a table with downstream mviews, the downstream fragments /// containing the `StreamScan` nodes will be included in this structure. -#[derive(Debug)] pub struct CompleteStreamFragmentGraph { /// The fragment graph of the streaming job being built. building_graph: StreamFragmentGraph, diff --git a/src/meta/src/stream/stream_graph/schedule.rs b/src/meta/src/stream/stream_graph/schedule.rs index 1ae24ec1b7d5..ed2dac5be0e0 100644 --- a/src/meta/src/stream/stream_graph/schedule.rs +++ b/src/meta/src/stream/stream_graph/schedule.rs @@ -326,7 +326,6 @@ impl Scheduler { /// [`Locations`] represents the parallel unit and worker locations of the actors. #[cfg_attr(test, derive(Default))] -#[derive(Debug)] pub struct Locations { /// actor location map. pub actor_locations: BTreeMap, From 4c7ae697026bae8f203cfbe7e6d1b78ed51da5ac Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 13 Mar 2024 12:08:30 +0800 Subject: [PATCH 05/29] fix shared cdc source --- src/frontend/src/handler/create_source.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 53d9cd5cb21f..e899d9319f8c 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -1307,7 +1307,8 @@ pub async fn handle_create_source( let sql_pk_names = bind_sql_pk_names(&stmt.columns, &stmt.constraints)?; let create_cdc_source_job = with_properties.is_shared_cdc_source(); - let has_streaming_job = create_cdc_source_job || with_properties.is_kafka_connector(); + let has_streaming_job = create_cdc_source_job + || (with_properties.is_kafka_connector() && session.config().rw_enable_reusable_source()); let (columns_from_resolve_source, mut source_info) = if create_cdc_source_job { bind_columns_from_source_for_cdc(&session, &source_schema, &with_properties)? @@ -1414,7 +1415,7 @@ pub async fn handle_create_source( let catalog_writer = session.catalog_writer()?; - if has_streaming_job && session.config().rw_enable_reusable_source() { + if has_streaming_job { let graph = { let context = OptimizerContext::from_handler_args(handler_args); let source_node = LogicalSource::with_catalog( From 2daa0a8392c8c08cff571d542a503421d23a196e Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 13 Mar 2024 13:40:10 +0800 Subject: [PATCH 06/29] fix test --- e2e_test/batch/catalog/pg_settings.slt.part | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/e2e_test/batch/catalog/pg_settings.slt.part b/e2e_test/batch/catalog/pg_settings.slt.part index c1b88b547361..8b39a3dcb02c 100644 --- a/e2e_test/batch/catalog/pg_settings.slt.part +++ b/e2e_test/batch/catalog/pg_settings.slt.part @@ -34,9 +34,9 @@ user row_security user rw_batch_enable_lookup_join user rw_batch_enable_sort_agg user rw_enable_join_ordering +user rw_enable_reusable_source user rw_enable_share_plan user rw_enable_two_phase_agg -user rw_enable_reusable_source user rw_force_split_distinct_agg user rw_force_two_phase_agg user rw_implicit_flush From 8d996f452a7c057e0977836c3a7647d90f749ee1 Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 15 Mar 2024 15:22:59 +0800 Subject: [PATCH 07/29] rename has_streaming_job -> is_shared --- proto/catalog.proto | 4 ++-- src/connector/src/source/cdc/mod.rs | 2 +- src/frontend/src/binder/relation/table_or_source.rs | 2 +- src/frontend/src/handler/create_source.rs | 10 +++++----- src/frontend/src/stream_fragmenter/mod.rs | 2 +- src/meta/src/controller/catalog.rs | 2 +- src/meta/src/controller/streaming_job.rs | 2 +- src/meta/src/manager/catalog/database.rs | 7 +------ src/meta/src/manager/catalog/mod.rs | 2 +- 9 files changed, 14 insertions(+), 19 deletions(-) diff --git a/proto/catalog.proto b/proto/catalog.proto index fae4d5ccc6f2..c6bc6fa2eb50 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -67,8 +67,8 @@ message StreamSourceInfo { // Currently, the following sources have streaming jobs: // - Direct CDC sources (mysql & postgresql) // - MQ sources (Kafka, Pulsar, Kinesis, etc.) - bool has_streaming_job = 13; - // Only used when `has_streaming_job` is `true`. + bool is_shared = 13; + // Only used when `is_shared` is `true`. // If `false`, `requires_singleton` will be set in the stream plan. bool is_distributed = 15; reserved "cdc_source_job"; // deprecated diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index 7d22ffd35a85..c8512118bd5a 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.has_streaming_job; + self.is_cdc_source_job = info.is_shared; } } diff --git a/src/frontend/src/binder/relation/table_or_source.rs b/src/frontend/src/binder/relation/table_or_source.rs index 8f3bb1a84ed4..c33284039bef 100644 --- a/src/frontend/src/binder/relation/table_or_source.rs +++ b/src/frontend/src/binder/relation/table_or_source.rs @@ -59,7 +59,7 @@ impl BoundSource { } pub fn can_backfill(&self) -> bool { - self.catalog.info.has_streaming_job + self.catalog.info.is_shared } } diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index e899d9319f8c..c08cad6338f6 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -505,7 +505,7 @@ fn bind_columns_from_source_for_cdc( row_encode: row_encode_to_prost(&source_schema.row_encode) as i32, format_encode_options, use_schema_registry: json_schema_infer_use_schema_registry(&schema_config), - has_streaming_job: true, + is_shared: true, ..Default::default() }; if !format_encode_options_to_consume.is_empty() { @@ -1307,7 +1307,7 @@ pub async fn handle_create_source( let sql_pk_names = bind_sql_pk_names(&stmt.columns, &stmt.constraints)?; let create_cdc_source_job = with_properties.is_shared_cdc_source(); - let has_streaming_job = create_cdc_source_job + let is_shared = create_cdc_source_job || (with_properties.is_kafka_connector() && session.config().rw_enable_reusable_source()); let (columns_from_resolve_source, mut source_info) = if create_cdc_source_job { @@ -1315,8 +1315,8 @@ pub async fn handle_create_source( } else { bind_columns_from_source(&session, &source_schema, &with_properties).await? }; - if has_streaming_job { - source_info.has_streaming_job = true; + if is_shared { + source_info.is_shared = true; source_info.is_distributed = !create_cdc_source_job; } let columns_from_sql = bind_sql_columns(&stmt.columns)?; @@ -1415,7 +1415,7 @@ pub async fn handle_create_source( let catalog_writer = session.catalog_writer()?; - if has_streaming_job { + if is_shared { let graph = { let context = OptimizerContext::from_handler_args(handler_args); let source_node = LogicalSource::with_catalog( diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index 87cb9a6703f1..07954ae9b7b6 100644 --- a/src/frontend/src/stream_fragmenter/mod.rs +++ b/src/frontend/src/stream_fragmenter/mod.rs @@ -264,7 +264,7 @@ fn build_fragment( if let Some(source) = node.source_inner.as_ref() && let Some(source_info) = source.info.as_ref() - && source_info.has_streaming_job + && source_info.is_shared && !source_info.is_distributed { current_fragment.requires_singleton = true; diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index a0700fa2d70c..3c5d082fe4e2 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -1980,7 +1980,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().has_streaming_job + && source_info.into_inner().is_shared { to_drop_streaming_jobs.push(object_id); } diff --git a/src/meta/src/controller/streaming_job.rs b/src/meta/src/controller/streaming_job.rs index cdd489779a2d..8d51b958d36c 100644 --- a/src/meta/src/controller/streaming_job.rs +++ b/src/meta/src/controller/streaming_job.rs @@ -816,7 +816,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().has_streaming_job + && source_info.inner_ref().is_shared { vec![source_id] } else { diff --git a/src/meta/src/manager/catalog/database.rs b/src/meta/src/manager/catalog/database.rs index 9fc6b04bd01c..fee5eacc3847 100644 --- a/src/meta/src/manager/catalog/database.rs +++ b/src/meta/src/manager/catalog/database.rs @@ -404,12 +404,7 @@ impl DatabaseManager { .chain( self.sources .iter() - .filter(|(_, source)| { - source - .info - .as_ref() - .is_some_and(|info| info.has_streaming_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/mod.rs b/src/meta/src/manager/catalog/mod.rs index 50860030029a..ed3bafaa1948 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -1453,7 +1453,7 @@ impl CatalogManager { } if let Some(info) = source.info - && info.has_streaming_job + && info.is_shared { all_streaming_job_source_ids.insert(source.id); let source_table_fragments = fragment_manager From 82afb3072216cd73f0755561cbf8d35c98e7a558 Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 15 Mar 2024 16:34:03 +0800 Subject: [PATCH 08/29] refine add_partition_offset_cols --- src/connector/Cargo.toml | 3 ++- .../src/parser/additional_columns.rs | 19 ++++++++++++++++++- 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index e18b5d2bb2c8..f7c1f913f887 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.33" async-trait = "0.1" auto_enums = { version = "0.8", features = ["futures03"] } @@ -119,7 +120,7 @@ risingwave_pb = { workspace = true } risingwave_rpc_client = { workspace = true } rumqttc = { version = "0.22.0", features = ["url"] } rust_decimal = "1" -rustls-native-certs = "0.6" +rustls-native-certs = "0.6" rustls-pemfile = "1" rw_futures_util = { workspace = true } serde = { version = "1", features = ["derive", "rc"] } diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index a4ace852a249..59c47b06ee8b 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -185,6 +185,11 @@ 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, @@ -224,10 +229,22 @@ pub fn add_partition_offset_cols( .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 { - use risingwave_pb::plan_common::additional_column::ColumnType; match col.column_desc.additional_column { AdditionalColumn { column_type: Some(ColumnType::Partition(_) | ColumnType::Filename(_)), From 5abbb1150a51693359e2b746e144488163c6d635 Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 15 Mar 2024 17:07:04 +0800 Subject: [PATCH 09/29] rename the plan SourceBackfill -> SourceScan --- proto/stream_plan.proto | 2 +- .../src/optimizer/plan_node/generic/source.rs | 2 +- ...rce_backfill.rs => logical_source_scan.rs} | 31 ++++++++++--------- src/frontend/src/optimizer/plan_node/mod.rs | 20 ++++++------ ...urce_backfill.rs => stream_source_scan.rs} | 22 ++++++++----- src/frontend/src/planner/relation.rs | 5 ++- src/frontend/src/stream_fragmenter/mod.rs | 2 +- src/meta/src/controller/fragment.rs | 2 +- 8 files changed, 46 insertions(+), 40 deletions(-) rename src/frontend/src/optimizer/plan_node/{logical_source_backfill.rs => logical_source_scan.rs} (90%) rename src/frontend/src/optimizer/plan_node/{stream_source_backfill.rs => stream_source_scan.rs} (90%) diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index a12b1c7171a8..51fb4f5d2a08 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -882,7 +882,7 @@ enum FragmentTypeFlag { 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/src/frontend/src/optimizer/plan_node/generic/source.rs b/src/frontend/src/optimizer/plan_node/generic/source.rs index 406a3654def2..c61ceda59e86 100644 --- a/src/frontend/src/optimizer/plan_node/generic/source.rs +++ b/src/frontend/src/optimizer/plan_node/generic/source.rs @@ -41,7 +41,7 @@ pub enum SourceNodeKind { /// /// 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 backfill-able source, `CREATE MATERIALIZE VIEW` will create `LogicalSourceScan` instead of `LogicalSource`. CreateMViewOrBatch, } diff --git a/src/frontend/src/optimizer/plan_node/logical_source_backfill.rs b/src/frontend/src/optimizer/plan_node/logical_source_scan.rs similarity index 90% rename from src/frontend/src/optimizer/plan_node/logical_source_backfill.rs rename to src/frontend/src/optimizer/plan_node/logical_source_scan.rs index 4a376794af57..f64b872a38b4 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source_backfill.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source_scan.rs @@ -34,14 +34,15 @@ use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::utils::column_names_pretty; use crate::optimizer::plan_node::{ - ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, StreamSourceBackfill, + ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, StreamSourceScan, ToStreamContext, }; use crate::optimizer::property::Distribution::HashShard; use crate::utils::{ColIndexMapping, Condition}; +/// `LogicalSourceScan` is used for batch scan from source, and MV on a *shared source*. #[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct LogicalSourceBackfill { +pub struct LogicalSourceScan { pub base: PlanBase, pub core: generic::Source, @@ -53,7 +54,7 @@ pub struct LogicalSourceBackfill { output_row_id_index: Option, } -impl LogicalSourceBackfill { +impl LogicalSourceScan { pub fn new(source_catalog: Rc, ctx: OptimizerContextRef) -> Result { let mut column_catalog = source_catalog.columns.clone(); let row_id_index = source_catalog.row_id_index; @@ -84,7 +85,7 @@ impl LogicalSourceBackfill { LogicalSource::derive_output_exprs_from_generated_columns(&core.column_catalog)?; let (core, output_row_id_index) = core.exclude_generated_columns(); - Ok(LogicalSourceBackfill { + Ok(LogicalSourceScan { base, core, output_exprs, @@ -96,12 +97,12 @@ impl LogicalSourceBackfill { self.core .catalog .clone() - .expect("source catalog should exist for LogicalSourceBackfill") + .expect("source catalog should exist for LogicalSourceScan") } } -impl_plan_tree_node_for_leaf! {LogicalSourceBackfill} -impl Distill for LogicalSourceBackfill { +impl_plan_tree_node_for_leaf! {LogicalSourceScan} +impl Distill for LogicalSourceScan { fn distill<'a>(&self) -> XmlNode<'a> { let src = Pretty::from(self.source_catalog().name.clone()); let time = Pretty::debug(&self.core.kafka_timestamp_range); @@ -111,18 +112,18 @@ impl Distill for LogicalSourceBackfill { ("time_range", time), ]; - childless_record("LogicalSourceBackfill", fields) + childless_record("LogicalSourceScan", fields) } } -impl ColPrunable for LogicalSourceBackfill { +impl ColPrunable for LogicalSourceScan { fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef { let mapping = ColIndexMapping::with_remaining_columns(required_cols, self.schema().len()); LogicalProject::with_mapping(self.clone().into(), mapping).into() } } -impl ExprRewritable for LogicalSourceBackfill { +impl ExprRewritable for LogicalSourceScan { fn has_rewritable_expr(&self) -> bool { self.output_exprs.is_some() } @@ -142,7 +143,7 @@ impl ExprRewritable for LogicalSourceBackfill { } } -impl ExprVisitable for LogicalSourceBackfill { +impl ExprVisitable for LogicalSourceScan { fn visit_exprs(&self, v: &mut dyn ExprVisitor) { self.output_exprs .iter() @@ -151,7 +152,7 @@ impl ExprVisitable for LogicalSourceBackfill { } } -impl PredicatePushdown for LogicalSourceBackfill { +impl PredicatePushdown for LogicalSourceScan { fn predicate_pushdown( &self, predicate: Condition, @@ -161,7 +162,7 @@ impl PredicatePushdown for LogicalSourceBackfill { } } -impl ToBatch for LogicalSourceBackfill { +impl ToBatch for LogicalSourceScan { fn to_batch(&self) -> Result { let mut plan: PlanRef = BatchSource::new(self.core.clone()).into(); @@ -174,9 +175,9 @@ impl ToBatch for LogicalSourceBackfill { } } -impl ToStream for LogicalSourceBackfill { +impl ToStream for LogicalSourceScan { fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result { - let mut plan = StreamSourceBackfill::new(self.core.clone()).into(); + let mut plan = StreamSourceScan::new(self.core.clone()).into(); if let Some(exprs) = &self.output_exprs { let logical_project = generic::Project::new(exprs.to_vec(), plan); diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 0b6e007b06b1..ff3a27852b6a 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -690,8 +690,8 @@ 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_backfill) = self.as_stream_source_backfill() { - return stream_source_backfill.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); @@ -827,7 +827,7 @@ mod logical_project_set; mod logical_scan; mod logical_share; mod logical_source; -mod logical_source_backfill; +mod logical_source_scan; mod logical_sys_scan; mod logical_table_function; mod logical_topn; @@ -857,7 +857,7 @@ mod stream_simple_agg; mod stream_sink; mod stream_sort; mod stream_source; -mod stream_source_backfill; +mod stream_source_scan; mod stream_stateless_simple_agg; mod stream_table_scan; mod stream_topn; @@ -920,7 +920,7 @@ pub use logical_project_set::LogicalProjectSet; pub use logical_scan::LogicalScan; pub use logical_share::LogicalShare; pub use logical_source::LogicalSource; -pub use logical_source_backfill::LogicalSourceBackfill; +pub use logical_source_scan::LogicalSourceScan; pub use logical_sys_scan::LogicalSysScan; pub use logical_table_function::LogicalTableFunction; pub use logical_topn::LogicalTopN; @@ -952,7 +952,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_backfill::StreamSourceBackfill; +pub use stream_source_scan::StreamSourceScan; pub use stream_stateless_simple_agg::StreamStatelessSimpleAgg; pub use stream_table_scan::StreamTableScan; pub use stream_temporal_join::StreamTemporalJoin; @@ -994,7 +994,7 @@ macro_rules! for_all_plan_nodes { , { Logical, CdcScan } , { Logical, SysScan } , { Logical, Source } - , { Logical, SourceBackfill } + , { Logical, SourceScan } , { Logical, Insert } , { Logical, Delete } , { Logical, Update } @@ -1048,7 +1048,7 @@ macro_rules! for_all_plan_nodes { , { Stream, CdcTableScan } , { Stream, Sink } , { Stream, Source } - , { Stream, SourceBackfill } + , { Stream, SourceScan } , { Stream, HashJoin } , { Stream, Exchange } , { Stream, HashAgg } @@ -1092,7 +1092,7 @@ macro_rules! for_logical_plan_nodes { , { Logical, CdcScan } , { Logical, SysScan } , { Logical, Source } - , { Logical, SourceBackfill } + , { Logical, SourceScan } , { Logical, Insert } , { Logical, Delete } , { Logical, Update } @@ -1166,7 +1166,7 @@ macro_rules! for_stream_plan_nodes { , { Stream, CdcTableScan } , { Stream, Sink } , { Stream, Source } - , { Stream, SourceBackfill } + , { Stream, SourceScan } , { Stream, HashAgg } , { Stream, SimpleAgg } , { Stream, StatelessSimpleAgg } diff --git a/src/frontend/src/optimizer/plan_node/stream_source_backfill.rs b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs similarity index 90% rename from src/frontend/src/optimizer/plan_node/stream_source_backfill.rs rename to src/frontend/src/optimizer/plan_node/stream_source_scan.rs index ed7702e12250..ee2d24905fab 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source_backfill.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs @@ -35,15 +35,21 @@ 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 StreamSourceBackfill { +pub struct StreamSourceScan { pub base: PlanBase, core: generic::Source, } -impl_plan_tree_node_for_leaf! { StreamSourceBackfill } +impl_plan_tree_node_for_leaf! { StreamSourceScan } -impl StreamSourceBackfill { +impl StreamSourceScan { pub fn new(source: generic::Source) -> Self { let base = PlanBase::new_stream_with_core( &source, @@ -163,7 +169,7 @@ impl StreamSourceBackfill { } } -impl Distill for StreamSourceBackfill { +impl Distill for StreamSourceScan { fn distill<'a>(&self) -> XmlNode<'a> { let columns = self .get_columns() @@ -171,15 +177,15 @@ impl Distill for StreamSourceBackfill { .map(|ele| Pretty::from(ele.to_string())) .collect(); let col = Pretty::Array(columns); - childless_record("StreamSourceBackfill", vec![("columns", col)]) + childless_record("StreamSourceScan", vec![("columns", col)]) } } -impl ExprRewritable for StreamSourceBackfill {} +impl ExprRewritable for StreamSourceScan {} -impl ExprVisitable for StreamSourceBackfill {} +impl ExprVisitable for StreamSourceScan {} -impl StreamNode for StreamSourceBackfill { +impl StreamNode for StreamSourceScan { fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> NodeBody { unreachable!("stream source backfill 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 9d30ec698b76..c48af33fe1a4 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -28,8 +28,7 @@ use crate::expr::{Expr, ExprImpl, ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::generic::SourceNodeKind; use crate::optimizer::plan_node::{ LogicalApply, LogicalHopWindow, LogicalJoin, LogicalProject, LogicalScan, LogicalShare, - LogicalSource, LogicalSourceBackfill, LogicalSysScan, LogicalTableFunction, LogicalValues, - PlanRef, + LogicalSource, LogicalSourceScan, LogicalSysScan, LogicalTableFunction, LogicalValues, PlanRef, }; use crate::optimizer::property::Cardinality; use crate::planner::Planner; @@ -99,7 +98,7 @@ impl Planner { .config() .rw_enable_reusable_source() { - Ok(LogicalSourceBackfill::new(Rc::new(source.catalog), self.ctx())?.into()) + Ok(LogicalSourceScan::new(Rc::new(source.catalog), self.ctx())?.into()) } else { Ok(LogicalSource::with_catalog( Rc::new(source.catalog), diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index 07954ae9b7b6..27d980e55258 100644 --- a/src/frontend/src/stream_fragmenter/mod.rs +++ b/src/frontend/src/stream_fragmenter/mod.rs @@ -311,7 +311,7 @@ fn build_fragment( .push(node.upstream_source_id); } NodeBody::SourceBackfill(node) => { - current_fragment.fragment_type_mask |= FragmentTypeFlag::SourceBackfill as u32; + current_fragment.fragment_type_mask |= FragmentTypeFlag::SourceScan as u32; // memorize upstream source id for later use let source_id = node.source_id; state.dependent_table_ids.insert(source_id.into()); diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index 67f3ce5363e9..1ede0941ba98 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -1228,7 +1228,7 @@ impl CatalogController { .into_tuple() .all(&inner.db) .await?; - fragments.retain(|(_, _, mask, _)| *mask & PbFragmentTypeFlag::SourceBackfill as i32 != 0); + fragments.retain(|(_, _, mask, _)| *mask & PbFragmentTypeFlag::SourceScan as i32 != 0); let mut source_fragment_ids = HashMap::new(); for (fragment_id, upstream_fragment_id, _, stream_node) in fragments { From fef7611096f5a0b73d47db43efdd3b9bd63735d7 Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 15 Mar 2024 17:15:21 +0800 Subject: [PATCH 10/29] rename upstream_mview_actors->upstream_root_actors --- src/meta/src/barrier/command.rs | 9 +++++---- src/meta/src/barrier/progress.rs | 4 ++-- src/meta/src/rpc/ddl_controller.rs | 8 ++------ src/meta/src/stream/stream_manager.rs | 10 ++++++---- 4 files changed, 15 insertions(+), 16 deletions(-) diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 18cc69217c91..ce1980a7f904 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -154,7 +154,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, @@ -856,7 +857,7 @@ impl CommandContext { Command::CreateStreamingJob { table_fragments, dispatchers, - upstream_mview_actors, + upstream_root_actors, init_split_assignment, definition: _, replace_table, @@ -865,8 +866,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)) diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index 1fcdace3f28b..893a621abb7e 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -414,7 +414,7 @@ impl CreateMviewProgressTracker { if let Command::CreateStreamingJob { table_fragments, dispatchers, - upstream_mview_actors, + upstream_root_actors, definition, ddl_type, .. @@ -422,7 +422,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/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 61a9ec7a64ce..a64db9df9b7f 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -1361,12 +1361,8 @@ impl DdlController { .get_upstream_root_fragments(fragment_graph.dependent_table_ids()) .await?; - // XXX: do we need to filter here? - let upstream_mview_actors: HashMap<_, _> = upstream_root_fragments + let upstream_root_actors: HashMap<_, _> = upstream_root_fragments .iter() - // .filter(|(_, fragment)| { - // fragment.fragment_type_mask & FragmentTypeFlag::Mview as u32 != 0 - // }) .map(|(&table_id, fragment)| { ( table_id, @@ -1465,7 +1461,7 @@ impl DdlController { let ctx = CreateStreamingJobContext { dispatchers, - upstream_mview_actors, + upstream_root_actors, internal_tables, building_locations, existing_locations, diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index e451a6dab893..4ff4130f32cd 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -50,8 +50,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, @@ -391,7 +393,7 @@ impl GlobalStreamManager { table_fragments: TableFragments, CreateStreamingJobContext { dispatchers, - upstream_mview_actors, + upstream_root_actors, building_locations, existing_locations, definition, @@ -486,7 +488,7 @@ impl GlobalStreamManager { let command = Command::CreateStreamingJob { table_fragments, - upstream_mview_actors, + upstream_root_actors, dispatchers, init_split_assignment, definition: definition.to_string(), From 8805238276c3b1a96d46b0dd0f261cd8a27cc971 Mon Sep 17 00:00:00 2001 From: xxchan Date: Mon, 18 Mar 2024 10:15:32 +0800 Subject: [PATCH 11/29] refine source_manager --- src/frontend/src/stream_fragmenter/mod.rs | 1 - src/meta/src/stream/scale.rs | 1 + src/meta/src/stream/source_manager.rs | 36 ++++++++++++----------- src/meta/src/stream/stream_manager.rs | 4 +-- 4 files changed, 22 insertions(+), 20 deletions(-) diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index 27d980e55258..6fbb7b6633b2 100644 --- a/src/frontend/src/stream_fragmenter/mod.rs +++ b/src/frontend/src/stream_fragmenter/mod.rs @@ -294,7 +294,6 @@ fn build_fragment( } NodeBody::StreamCdcScan(_) => { - // XXX: Should we use a different flag for CDC scan? current_fragment.fragment_type_mask |= FragmentTypeFlag::StreamScan as u32; // the backfill algorithm is not parallel safe current_fragment.requires_singleton = true; diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index 88f4be1cf81d..e2b63f54e5b2 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -1232,6 +1232,7 @@ impl ScaleController { .await?; fragment_stream_source_actor_splits.insert(*fragment_id, actor_splits); + todo!("migrate_splits_backfill"); } } diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index a42ea7dba5a8..5bd931b18050 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -598,23 +598,21 @@ fn align_backfill_splits( 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 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() + .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 { @@ -873,6 +871,10 @@ 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, diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 4ff4130f32cd..f9cf11512cfd 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -476,9 +476,9 @@ impl GlobalStreamManager { let table_id = table_fragments.table_id(); // Here we need to consider: - // - Source with streaming job (backfill-able source) + // - Shared source // - Table with connector - // - MV on backfill-able source + // - MV on shared source let mut init_split_assignment = self.source_manager.allocate_splits(&table_id).await?; init_split_assignment.extend( self.source_manager From d0284692f01637681f8907ead3e13eeeac1801ab Mon Sep 17 00:00:00 2001 From: xxchan Date: Mon, 18 Mar 2024 10:36:34 +0800 Subject: [PATCH 12/29] refine rewrite_inner, unif CdcFilter and SourceBackfill --- proto/stream_plan.proto | 2 +- .../optimizer/plan_node/stream_source_scan.rs | 2 +- src/frontend/src/stream_fragmenter/mod.rs | 2 +- src/meta/src/stream/stream_graph/actor.rs | 59 ++++--------------- src/meta/src/stream/stream_graph/fragment.rs | 2 +- src/prost/src/lib.rs | 2 +- src/stream/src/from_proto/source_backfill.rs | 2 +- 7 files changed, 18 insertions(+), 53 deletions(-) diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 51fb4f5d2a08..0d3f8aa16613 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; diff --git a/src/frontend/src/optimizer/plan_node/stream_source_scan.rs b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs index ee2d24905fab..0d6d2dd7f009 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs @@ -126,7 +126,7 @@ impl StreamSourceScan { let source_catalog = self.source_catalog(); let backfill = SourceBackfillNode { - source_id: source_catalog.id, + upstream_source_id: source_catalog.id, source_name: source_catalog.name.clone(), state_table: Some( Self::infer_internal_table_catalog() diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index 6fbb7b6633b2..10dae3692565 100644 --- a/src/frontend/src/stream_fragmenter/mod.rs +++ b/src/frontend/src/stream_fragmenter/mod.rs @@ -312,7 +312,7 @@ fn build_fragment( NodeBody::SourceBackfill(node) => { current_fragment.fragment_type_mask |= FragmentTypeFlag::SourceScan as u32; // memorize upstream source id for later use - let source_id = node.source_id; + let source_id = node.upstream_source_id; state.dependent_table_ids.insert(source_id.into()); current_fragment.upstream_table_ids.push(source_id); } diff --git a/src/meta/src/stream/stream_graph/actor.rs b/src/meta/src/stream/stream_graph/actor.rs index e4aac7ea1991..c25b48c28f04 100644 --- a/src/meta/src/stream/stream_graph/actor.rs +++ b/src/meta/src/stream/stream_graph/actor.rs @@ -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,48 +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` - let input = vec![ - // Fill the merge node body with correct upstream info. - StreamNode { - node_body: Some(NodeBody::Merge(MergeNode { - upstream_actor_id, - upstream_fragment_id: upstreams.fragment_id.as_global_id(), - upstream_dispatcher_type: DispatcherType::NoShuffle as _, - fields: merge_node.fields.clone(), - })), - ..merge_node.clone() - }, - ]; - Ok(StreamNode { - input, - ..stream_node.clone() - }) - } - - // "Leaf" node `SourceBackfill`. - NodeBody::SourceBackfill(source_backfill) => { - 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 = source_backfill.source_id; - - // Index the upstreams by the an external edge ID. - let upstreams = &self.upstreams[&EdgeId::UpstreamExternal { - upstream_table_id: upstream_source_id.into(), - downstream_fragment_id: self.fragment_id, - }]; - - let upstream_actor_id = upstreams.actors.as_global_ids(); - - // rewrite the input of `SourceBackfill` + // rewrite the input let input = vec![ // Fill the merge node body with correct upstream info. StreamNode { @@ -285,7 +251,6 @@ impl ActorBuilder { ..merge_node.clone() }, ]; - Ok(StreamNode { input, ..stream_node.clone() diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index 548203bbf1e7..0e332054d7ea 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -184,7 +184,7 @@ impl BuildingFragment { ), NodeBody::CdcFilter(cdc_filter) => (cdc_filter.upstream_source_id.into(), vec![]), NodeBody::SourceBackfill(backfill) => ( - backfill.source_id.into(), + backfill.upstream_source_id.into(), // FIXME: only pass required columns instead of all columns here backfill .columns diff --git a/src/prost/src/lib.rs b/src/prost/src/lib.rs index 775394668977..ac7fc0b1c4a4 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -238,7 +238,7 @@ impl stream_plan::StreamNode { if let Some(crate::stream_plan::stream_node::NodeBody::SourceBackfill(source)) = self.node_body.as_ref() { - return Some(source.source_id); + return Some(source.upstream_source_id); } for child in &self.input { diff --git a/src/stream/src/from_proto/source_backfill.rs b/src/stream/src/from_proto/source_backfill.rs index 7a089464cf22..40d0f339be69 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()?; From 31b34f00182f63b388531651b470a7de8946740f Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 20 Mar 2024 01:32:20 +0800 Subject: [PATCH 13/29] rename reuseable source -> shared source --- e2e_test/batch/catalog/pg_settings.slt.part | 2 +- src/common/src/session_config/mod.rs | 2 +- src/frontend/src/handler/create_source.rs | 2 +- src/frontend/src/planner/relation.rs | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/e2e_test/batch/catalog/pg_settings.slt.part b/e2e_test/batch/catalog/pg_settings.slt.part index 6db4dd19a511..907a5da6bb7c 100644 --- a/e2e_test/batch/catalog/pg_settings.slt.part +++ b/e2e_test/batch/catalog/pg_settings.slt.part @@ -34,7 +34,7 @@ user row_security user rw_batch_enable_lookup_join user rw_batch_enable_sort_agg user rw_enable_join_ordering -user rw_enable_reusable_source +user rw_enable_shared_source user rw_enable_share_plan user rw_enable_two_phase_agg user rw_force_split_distinct_agg diff --git a/src/common/src/session_config/mod.rs b/src/common/src/session_config/mod.rs index 514e30f84dbb..feba2a4160f9 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -245,7 +245,7 @@ pub struct ConfigMap { /// Enable reusable source. Currently only for Kafka. #[parameter(default = false)] - rw_enable_reusable_source: bool, + 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)] diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index f2116e1a9c0f..da449c884bc2 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -1314,7 +1314,7 @@ pub async fn handle_create_source( let create_cdc_source_job = with_properties.is_backfillable_cdc_connector(); let is_shared = create_cdc_source_job - || (with_properties.is_kafka_connector() && session.config().rw_enable_reusable_source()); + || (with_properties.is_kafka_connector() && session.config().rw_enable_shared_source()); let (columns_from_resolve_source, mut source_info) = if create_cdc_source_job { bind_columns_from_source_for_cdc(&session, &source_schema, &with_properties)? diff --git a/src/frontend/src/planner/relation.rs b/src/frontend/src/planner/relation.rs index 2789072d016e..824795d1f199 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -96,7 +96,7 @@ impl Planner { .ctx() .session_ctx() .config() - .rw_enable_reusable_source() + .rw_enable_shared_source() { Ok(LogicalSourceScan::new(Rc::new(source.catalog), self.ctx())?.into()) } else { From 27db3f65905488aff79ad16968e081f6b93e958d Mon Sep 17 00:00:00 2001 From: xxchan Date: Mon, 25 Mar 2024 13:03:09 +0800 Subject: [PATCH 14/29] fix compile --- .../plan_node/logical_source_scan.rs | 8 +++- src/frontend/src/planner/relation.rs | 44 +++++++++---------- 2 files changed, 28 insertions(+), 24 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_source_scan.rs b/src/frontend/src/optimizer/plan_node/logical_source_scan.rs index f64b872a38b4..f5d50987fb89 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source_scan.rs @@ -18,6 +18,7 @@ use std::rc::Rc; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_connector::parser::additional_columns::add_partition_offset_cols; +use risingwave_sqlparser::ast::AsOf; use super::generic::{GenericPlanRef, SourceNodeKind}; use super::stream_watermark_filter::StreamWatermarkFilter; @@ -55,7 +56,11 @@ pub struct LogicalSourceScan { } impl LogicalSourceScan { - pub fn new(source_catalog: Rc, ctx: OptimizerContextRef) -> Result { + pub fn new( + source_catalog: Rc, + ctx: OptimizerContextRef, + as_of: Option, + ) -> Result { let mut column_catalog = source_catalog.columns.clone(); let row_id_index = source_catalog.row_id_index; @@ -77,6 +82,7 @@ impl LogicalSourceScan { kind: SourceNodeKind::CreateMViewOrBatch, ctx, kafka_timestamp_range, + as_of, }; let base = PlanBase::new_logical_with_core(&core); diff --git a/src/frontend/src/planner/relation.rs b/src/frontend/src/planner/relation.rs index 5041bc204fdf..458801107115 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -97,32 +97,30 @@ impl Planner { pub(super) fn plan_source(&mut self, source: BoundSource) -> Result { if source.is_backfillable_cdc_connector() { - Err(ErrorCode::InternalError( + return Err(ErrorCode::InternalError( "Should not create MATERIALIZED VIEW or SELECT directly on shared CDC source. HINT: create TABLE from the source instead.".to_string(), ) - .into()) - } else if source.is_shared() - && self - .ctx() - .session_ctx() - .config() - .rw_enable_shared_source() - { - Ok(LogicalSourceScan::new(Rc::new(source.catalog), self.ctx())?.into()) - } else { - let as_of = source.as_of.clone(); - match as_of { - None => {} - Some(AsOf::ProcessTime) => { - bail_not_implemented!("As Of ProcessTime() is not supported yet.") - } - Some(AsOf::TimestampString(_)) | Some(AsOf::TimestampNum(_)) => { - bail_not_implemented!("As Of Timestamp is not supported yet.") - } - Some(AsOf::VersionNum(_)) | Some(AsOf::VersionString(_)) => { - bail_not_implemented!("As Of Version is not supported yet.") - } + .into()); + } + + let as_of = source.as_of.clone(); + match as_of { + None => {} + Some(AsOf::ProcessTime) => { + bail_not_implemented!("As Of ProcessTime() is not supported yet.") } + Some(AsOf::TimestampString(_)) | Some(AsOf::TimestampNum(_)) => { + bail_not_implemented!("As Of Timestamp is not supported yet.") + } + Some(AsOf::VersionNum(_)) | Some(AsOf::VersionString(_)) => { + bail_not_implemented!("As Of Version is not supported yet.") + } + } + // XXX: can we unify them into one operator? + // Note that batch scan on source goes through the same code path. It makes little sense to use different operators in this case. + if source.is_shared() && self.ctx().session_ctx().config().rw_enable_shared_source() { + Ok(LogicalSourceScan::new(Rc::new(source.catalog), self.ctx(), as_of)?.into()) + } else { Ok(LogicalSource::with_catalog( Rc::new(source.catalog), SourceNodeKind::CreateMViewOrBatch, From 9f8f533fc3b02bcaa3000bca447da7a6583d7496 Mon Sep 17 00:00:00 2001 From: xxchan Date: Mon, 25 Mar 2024 13:54:19 +0800 Subject: [PATCH 15/29] fix cdc table stuck. due to track id --- proto/stream_plan.proto | 1 + src/meta/src/barrier/progress.rs | 20 ++++++++++++++++++++ src/meta/src/barrier/schedule.rs | 11 +++++++++-- src/meta/src/manager/catalog/database.rs | 2 +- src/meta/src/model/stream.rs | 7 +++++-- src/meta/src/stream/stream_manager.rs | 5 +++++ 6 files changed, 41 insertions(+), 5 deletions(-) diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 0d3f8aa16613..b4393153b57a 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -876,6 +876,7 @@ 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; diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index 1374e2e0a736..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()) diff --git a/src/meta/src/barrier/schedule.rs b/src/meta/src/barrier/schedule.rs index 662a1006baae..596a8bfd15d6 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/manager/catalog/database.rs b/src/meta/src/manager/catalog/database.rs index fee5eacc3847..e6c2da61018b 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)) } diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index da1a310ee8d8..670ecb99e217 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -370,9 +370,12 @@ impl TableFragments { /// Returns actor ids that need to be tracked when creating MV. pub fn tracking_progress_actor_ids(&self) -> Vec { Self::filter_actor_ids(self, |fragment_type_mask| { - (fragment_type_mask + let is_value_or_scan = (fragment_type_mask & (FragmentTypeFlag::Values as u32 | FragmentTypeFlag::StreamScan as u32)) - != 0 + != 0; + // Note: CDC table fragment is both StreamScan and CdcFilter fragment. We don't want to track CDC progress. + let is_cdc = (fragment_type_mask & FragmentTypeFlag::CdcFilter as u32) == 0; + is_value_or_scan && !is_cdc }) } diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index f9cf11512cfd..7cb85d91af7d 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -434,6 +434,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( @@ -495,6 +499,7 @@ impl GlobalStreamManager { 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)); From 78a783f80d87298336d21d16514b21a59c3c4cae Mon Sep 17 00:00:00 2001 From: xxchan Date: Mon, 25 Mar 2024 17:07:46 +0800 Subject: [PATCH 16/29] fix --- src/meta/src/model/stream.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index 670ecb99e217..701613574350 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -374,7 +374,7 @@ impl TableFragments { & (FragmentTypeFlag::Values as u32 | FragmentTypeFlag::StreamScan as u32)) != 0; // Note: CDC table fragment is both StreamScan and CdcFilter fragment. We don't want to track CDC progress. - let is_cdc = (fragment_type_mask & FragmentTypeFlag::CdcFilter as u32) == 0; + let is_cdc = (fragment_type_mask & FragmentTypeFlag::CdcFilter as u32) != 0; is_value_or_scan && !is_cdc }) } From 06027cacdb49ba947b3df38bec52a2fee7e88823 Mon Sep 17 00:00:00 2001 From: xxchan Date: Mon, 25 Mar 2024 17:34:06 +0800 Subject: [PATCH 17/29] fix pg_settings order --- e2e_test/batch/catalog/pg_settings.slt.part | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/e2e_test/batch/catalog/pg_settings.slt.part b/e2e_test/batch/catalog/pg_settings.slt.part index 907a5da6bb7c..d9af757ba4c3 100644 --- a/e2e_test/batch/catalog/pg_settings.slt.part +++ b/e2e_test/batch/catalog/pg_settings.slt.part @@ -34,8 +34,8 @@ user row_security user rw_batch_enable_lookup_join user rw_batch_enable_sort_agg user rw_enable_join_ordering -user rw_enable_shared_source 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 From fa521f05f715092809af52129dddbf8b0e066e5e Mon Sep 17 00:00:00 2001 From: xxchan Date: Mon, 25 Mar 2024 18:07:31 +0800 Subject: [PATCH 18/29] really fix tracking_progress_actor_ids --- src/meta/src/model/stream.rs | 21 ++++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index 701613574350..7635e7eb8ed8 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -369,14 +369,21 @@ impl TableFragments { /// Returns actor ids that need to be tracked when creating MV. pub fn tracking_progress_actor_ids(&self) -> Vec { - Self::filter_actor_ids(self, |fragment_type_mask| { - let is_value_or_scan = (fragment_type_mask + 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; - // Note: CDC table fragment is both StreamScan and CdcFilter fragment. We don't want to track CDC progress. - let is_cdc = (fragment_type_mask & FragmentTypeFlag::CdcFilter as u32) != 0; - is_value_or_scan && !is_cdc - }) + != 0 + { + actor_ids.extend(fragment.actors.iter().map(|actor| actor.actor_id)); + } + } + actor_ids } /// Returns the fragment with the `Mview` type flag. From 58034cc30067ed6999179587e535d62cb686d451 Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 27 Mar 2024 00:28:52 +0800 Subject: [PATCH 19/29] - no need for a separated LogicalSourceScan node. - add planner tests --- e2e_test/source/README.md | 7 + scripts/source/README.md | 5 + src/common/src/session_config/mod.rs | 5 +- .../tests/testdata/input/shared_source.yml | 55 +++++ .../tests/testdata/output/shared_source.yml | 74 ++++++ src/frontend/src/handler/create_source.rs | 2 +- .../src/optimizer/plan_node/generic/source.rs | 10 +- .../src/optimizer/plan_node/logical_source.rs | 37 ++- .../plan_node/logical_source_scan.rs | 214 ------------------ src/frontend/src/optimizer/plan_node/mod.rs | 4 - .../src/optimizer/plan_node/stream_source.rs | 20 +- .../optimizer/plan_node/stream_source_scan.rs | 27 ++- src/frontend/src/planner/relation.rs | 22 +- 13 files changed, 214 insertions(+), 268 deletions(-) create mode 100644 e2e_test/source/README.md create mode 100644 scripts/source/README.md create mode 100644 src/frontend/planner_test/tests/testdata/input/shared_source.yml create mode 100644 src/frontend/planner_test/tests/testdata/output/shared_source.yml delete mode 100644 src/frontend/src/optimizer/plan_node/logical_source_scan.rs diff --git a/e2e_test/source/README.md b/e2e_test/source/README.md new file mode 100644 index 000000000000..97c08fc0f583 --- /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/scripts/source/README.md b/scripts/source/README.md new file mode 100644 index 000000000000..3716865ab3e3 --- /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/src/session_config/mod.rs b/src/common/src/session_config/mod.rs index feba2a4160f9..4822ff5bafea 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -243,7 +243,10 @@ pub struct ConfigMap { #[parameter(default = false)] background_ddl: bool, - /// Enable reusable source. Currently only for Kafka. + /// 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, 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..24f9a25b8042 --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/input/shared_source.yml @@ -0,0 +1,55 @@ +- 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: [] +# 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..fbc13c1c1fe3 --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/output/shared_source.yml @@ -0,0 +1,74 @@ +# 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; +- 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/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index cf08d6e064ad..d7a8a3cd2c4e 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -1426,7 +1426,7 @@ pub async fn handle_create_source( let context = OptimizerContext::from_handler_args(handler_args); let source_node = LogicalSource::with_catalog( Rc::new(SourceCatalog::from(&source)), - SourceNodeKind::CreateSourceWithStreamjob, + SourceNodeKind::CreateSharedSource, context.into(), None, )?; diff --git a/src/frontend/src/optimizer/plan_node/generic/source.rs b/src/frontend/src/optimizer/plan_node/generic/source.rs index 442d5a50dc5b..8ec45dd34d97 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 `LogicalSourceScan` 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 dfd0ca3af679..4689e8024045 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -22,8 +22,6 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{ ColumnCatalog, ColumnDesc, Field, Schema, KAFKA_TIMESTAMP_COLUMN_NAME, }; -use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_connector::parser::additional_columns::add_partition_offset_cols; use risingwave_connector::source::iceberg::ICEBERG_CONNECTOR; use risingwave_connector::source::{DataType, UPSTREAM_SOURCE_KEY}; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; @@ -36,7 +34,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; @@ -70,7 +68,7 @@ pub struct LogicalSource { impl LogicalSource { pub fn new( source_catalog: Option>, - mut column_catalog: Vec, + column_catalog: Vec, row_id_index: Option, kind: SourceNodeKind, ctx: OptimizerContextRef, @@ -78,20 +76,6 @@ impl LogicalSource { ) -> Result { let kafka_timestamp_range = (Bound::Unbounded, Bound::Unbounded); - // for sources with streaming job, we will include partition and offset cols in the output. - if let Some(source_catalog) = &source_catalog - && matches!(kind, SourceNodeKind::CreateSourceWithStreamjob) - { - let (columns_exist, additional_columns) = - add_partition_offset_cols(&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 { - column_catalog.push(c); - } - } - } - let core = generic::Source { catalog: source_catalog, column_catalog, @@ -527,7 +511,7 @@ impl ToStream for LogicalSource { let mut plan: PlanRef; match self.core.kind { - SourceNodeKind::CreateTable | SourceNodeKind::CreateSourceWithStreamjob => { + SourceNodeKind::CreateTable | SourceNodeKind::CreateSharedSource => { // Note: for create table, row_id and generated columns is created in plan_root.gen_table_plan. // for backfill-able source, row_id and generated columns is created after SourceBackfill node. if self.core.is_new_fs_connector() { @@ -539,11 +523,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/logical_source_scan.rs b/src/frontend/src/optimizer/plan_node/logical_source_scan.rs deleted file mode 100644 index f5d50987fb89..000000000000 --- a/src/frontend/src/optimizer/plan_node/logical_source_scan.rs +++ /dev/null @@ -1,214 +0,0 @@ -// 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::ops::Bound; -use std::rc::Rc; - -use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_connector::parser::additional_columns::add_partition_offset_cols; -use risingwave_sqlparser::ast::AsOf; - -use super::generic::{GenericPlanRef, SourceNodeKind}; -use super::stream_watermark_filter::StreamWatermarkFilter; -use super::utils::{childless_record, Distill}; -use super::{ - generic, BatchProject, BatchSource, ColPrunable, ExprRewritable, Logical, LogicalFilter, - LogicalProject, LogicalSource, PlanBase, PlanRef, PredicatePushdown, StreamProject, - StreamRowIdGen, ToBatch, ToStream, -}; -use crate::catalog::source_catalog::SourceCatalog; -use crate::error::Result; -use crate::expr::{ExprImpl, ExprRewriter, ExprVisitor}; -use crate::optimizer::optimizer_context::OptimizerContextRef; -use crate::optimizer::plan_node::expr_visitable::ExprVisitable; -use crate::optimizer::plan_node::utils::column_names_pretty; -use crate::optimizer::plan_node::{ - ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, StreamSourceScan, - ToStreamContext, -}; -use crate::optimizer::property::Distribution::HashShard; -use crate::utils::{ColIndexMapping, Condition}; - -/// `LogicalSourceScan` is used for batch scan from source, and MV on a *shared source*. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct LogicalSourceScan { - pub base: PlanBase, - pub core: generic::Source, - - /// Expressions to output. This field presents and will be turned to a `Project` when - /// converting to a physical plan, only if there are generated columns. - output_exprs: Option>, - /// When there are generated columns, the `StreamRowIdGen`'s row_id_index is different from - /// the one in `core`. So we store the one in `output_exprs` here. - output_row_id_index: Option, -} - -impl LogicalSourceScan { - pub fn new( - source_catalog: Rc, - ctx: OptimizerContextRef, - as_of: Option, - ) -> Result { - let mut column_catalog = source_catalog.columns.clone(); - let row_id_index = source_catalog.row_id_index; - - let kafka_timestamp_range = (Bound::Unbounded, Bound::Unbounded); - - let (columns_exist, additional_columns) = - add_partition_offset_cols(&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 { - column_catalog.push(c); - } - } - let core = generic::Source { - catalog: Some(source_catalog), - column_catalog, - row_id_index, - // FIXME: this field is not useful for backfill. - kind: SourceNodeKind::CreateMViewOrBatch, - ctx, - kafka_timestamp_range, - as_of, - }; - - let base = PlanBase::new_logical_with_core(&core); - - let output_exprs = - LogicalSource::derive_output_exprs_from_generated_columns(&core.column_catalog)?; - let (core, output_row_id_index) = core.exclude_generated_columns(); - - Ok(LogicalSourceScan { - base, - core, - output_exprs, - output_row_id_index, - }) - } - - pub fn source_catalog(&self) -> Rc { - self.core - .catalog - .clone() - .expect("source catalog should exist for LogicalSourceScan") - } -} - -impl_plan_tree_node_for_leaf! {LogicalSourceScan} -impl Distill for LogicalSourceScan { - fn distill<'a>(&self) -> XmlNode<'a> { - let src = Pretty::from(self.source_catalog().name.clone()); - let time = Pretty::debug(&self.core.kafka_timestamp_range); - let fields = vec![ - ("source", src), - ("columns", column_names_pretty(self.schema())), - ("time_range", time), - ]; - - childless_record("LogicalSourceScan", fields) - } -} - -impl ColPrunable for LogicalSourceScan { - fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef { - let mapping = ColIndexMapping::with_remaining_columns(required_cols, self.schema().len()); - LogicalProject::with_mapping(self.clone().into(), mapping).into() - } -} - -impl ExprRewritable for LogicalSourceScan { - fn has_rewritable_expr(&self) -> bool { - self.output_exprs.is_some() - } - - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - let mut output_exprs = self.output_exprs.clone(); - - for expr in output_exprs.iter_mut().flatten() { - *expr = r.rewrite_expr(expr.clone()); - } - - Self { - output_exprs, - ..self.clone() - } - .into() - } -} - -impl ExprVisitable for LogicalSourceScan { - fn visit_exprs(&self, v: &mut dyn ExprVisitor) { - self.output_exprs - .iter() - .flatten() - .for_each(|e| v.visit_expr(e)); - } -} - -impl PredicatePushdown for LogicalSourceScan { - fn predicate_pushdown( - &self, - predicate: Condition, - _ctx: &mut PredicatePushdownContext, - ) -> PlanRef { - LogicalFilter::create(self.clone().into(), predicate) - } -} - -impl ToBatch for LogicalSourceScan { - fn to_batch(&self) -> Result { - let mut plan: PlanRef = BatchSource::new(self.core.clone()).into(); - - if let Some(exprs) = &self.output_exprs { - let logical_project = generic::Project::new(exprs.to_vec(), plan); - plan = BatchProject::new(logical_project).into(); - } - - Ok(plan) - } -} - -impl ToStream for LogicalSourceScan { - fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result { - let mut plan = StreamSourceScan::new(self.core.clone()).into(); - - if let Some(exprs) = &self.output_exprs { - let logical_project = generic::Project::new(exprs.to_vec(), plan); - plan = StreamProject::new(logical_project).into(); - } - - let catalog = self.source_catalog(); - if !catalog.watermark_descs.is_empty() { - plan = StreamWatermarkFilter::new(plan, catalog.watermark_descs.clone()).into(); - } - - if let Some(row_id_index) = self.output_row_id_index { - plan = StreamRowIdGen::new_with_dist(plan, row_id_index, HashShard(vec![row_id_index])) - .into(); - } - Ok(plan) - } - - fn logical_rewrite_for_stream( - &self, - _ctx: &mut RewriteStreamContext, - ) -> Result<(PlanRef, ColIndexMapping)> { - Ok(( - self.clone().into(), - ColIndexMapping::identity(self.schema().len()), - )) - } -} diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 45a27bf56014..1f4b60e73999 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -848,7 +848,6 @@ mod logical_project_set; mod logical_scan; mod logical_share; mod logical_source; -mod logical_source_scan; mod logical_sys_scan; mod logical_table_function; mod logical_topn; @@ -942,7 +941,6 @@ pub use logical_project_set::LogicalProjectSet; pub use logical_scan::LogicalScan; pub use logical_share::LogicalShare; pub use logical_source::LogicalSource; -pub use logical_source_scan::LogicalSourceScan; pub use logical_sys_scan::LogicalSysScan; pub use logical_table_function::LogicalTableFunction; pub use logical_topn::LogicalTopN; @@ -1017,7 +1015,6 @@ macro_rules! for_all_plan_nodes { , { Logical, CdcScan } , { Logical, SysScan } , { Logical, Source } - , { Logical, SourceScan } , { Logical, Insert } , { Logical, Delete } , { Logical, Update } @@ -1116,7 +1113,6 @@ macro_rules! for_logical_plan_nodes { , { Logical, CdcScan } , { Logical, SysScan } , { Logical, Source } - , { Logical, SourceScan } , { Logical, Insert } , { Logical, Delete } , { Logical, Update } diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index 537ade49687f..a4df97c5bc4c 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 index 0d6d2dd7f009..7016ca4ccb56 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs @@ -19,7 +19,9 @@ 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; @@ -50,16 +52,31 @@ pub struct StreamSourceScan { impl_plan_tree_node_for_leaf! { StreamSourceScan } impl StreamSourceScan { - pub fn new(source: generic::Source) -> Self { + 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( - &source, + &core, Distribution::SomeShard, - source.catalog.as_ref().map_or(true, |s| s.append_only), + core.catalog.as_ref().map_or(true, |s| s.append_only), false, - FixedBitSet::with_capacity(source.column_catalog.len()), + FixedBitSet::with_capacity(core.column_catalog.len()), ); - Self { base, core: source } + Self { base, core } } fn get_columns(&self) -> Vec<&str> { diff --git a/src/frontend/src/planner/relation.rs b/src/frontend/src/planner/relation.rs index 458801107115..f8b3138b6b99 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -29,7 +29,7 @@ use crate::expr::{Expr, ExprImpl, ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::generic::SourceNodeKind; use crate::optimizer::plan_node::{ LogicalApply, LogicalHopWindow, LogicalJoin, LogicalProject, LogicalScan, LogicalShare, - LogicalSource, LogicalSourceScan, LogicalSysScan, LogicalTableFunction, LogicalValues, PlanRef, + LogicalSource, LogicalSysScan, LogicalTableFunction, LogicalValues, PlanRef, }; use crate::optimizer::property::Cardinality; use crate::planner::Planner; @@ -116,19 +116,13 @@ impl Planner { bail_not_implemented!("As Of Version is not supported yet.") } } - // XXX: can we unify them into one operator? - // Note that batch scan on source goes through the same code path. It makes little sense to use different operators in this case. - if source.is_shared() && self.ctx().session_ctx().config().rw_enable_shared_source() { - Ok(LogicalSourceScan::new(Rc::new(source.catalog), self.ctx(), as_of)?.into()) - } else { - Ok(LogicalSource::with_catalog( - Rc::new(source.catalog), - SourceNodeKind::CreateMViewOrBatch, - self.ctx(), - as_of, - )? - .into()) - } + Ok(LogicalSource::with_catalog( + Rc::new(source.catalog), + SourceNodeKind::CreateMViewOrBatch, + self.ctx(), + as_of, + )? + .into()) } pub(super) fn plan_join(&mut self, join: BoundJoin) -> Result { From 18ea279bf7a94473d7b7ac441b0d9184b7aec343 Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 27 Mar 2024 03:02:14 +0800 Subject: [PATCH 20/29] add (very simple) e2e tests --- e2e_test/source/README.md | 2 +- e2e_test/source/basic/kafka_shared_source.slt | 58 +++++++++++++++++++ scripts/source/README.md | 2 +- .../tests/testdata/input/shared_source.yml | 13 +++++ .../tests/testdata/output/shared_source.yml | 20 +++++++ 5 files changed, 93 insertions(+), 2 deletions(-) create mode 100644 e2e_test/source/basic/kafka_shared_source.slt diff --git a/e2e_test/source/README.md b/e2e_test/source/README.md index 97c08fc0f583..b6e9dfa30816 100644 --- a/e2e_test/source/README.md +++ b/e2e_test/source/README.md @@ -4,4 +4,4 @@ See also `ci/scripts/e2e-source-test.sh`, and `scripts/source` ## Kafka -`scripts/source/test_data` contains the data. Filename's convention is `.`. +`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/scripts/source/README.md b/scripts/source/README.md index 3716865ab3e3..4676aa7723b3 100644 --- a/scripts/source/README.md +++ b/scripts/source/README.md @@ -2,4 +2,4 @@ This folder contains scripts to prepare data for testing sources. ## Kafka -`scripts/source/test_data` contains the data. Filename's convention is `.`. +`scripts/source/test_data` contains the data. Filename's convention is `.`. diff --git a/src/frontend/planner_test/tests/testdata/input/shared_source.yml b/src/frontend/planner_test/tests/testdata/input/shared_source.yml index 24f9a25b8042..0f68cc25f628 100644 --- a/src/frontend/planner_test/tests/testdata/input/shared_source.yml +++ b/src/frontend/planner_test/tests/testdata/input/shared_source.yml @@ -8,6 +8,19 @@ 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 # diff --git a/src/frontend/planner_test/tests/testdata/output/shared_source.yml b/src/frontend/planner_test/tests/testdata/output/shared_source.yml index fbc13c1c1fe3..54f50fc2c687 100644 --- a/src/frontend/planner_test/tests/testdata/output/shared_source.yml +++ b/src/frontend/planner_test/tests/testdata/output/shared_source.yml @@ -8,6 +8,26 @@ 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: | From 658cb52e7299aca06196776cccd599e44e9205e1 Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 27 Mar 2024 03:22:38 +0800 Subject: [PATCH 21/29] fix executor bug: should ignore backfilled rows when the partition is done (note this is manually tested. no tests...) --- .../source/source_backfill_executor.rs | 31 +++++++++++++------ 1 file changed, 21 insertions(+), 10 deletions(-) diff --git a/src/stream/src/executor/source/source_backfill_executor.rs b/src/stream/src/executor/source/source_backfill_executor.rs index 83ec797d4476..51c7b2ed5968 100644 --- a/src/stream/src/executor/source/source_backfill_executor.rs +++ b/src/stream/src/executor/source/source_backfill_executor.rs @@ -520,11 +520,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 @@ -546,8 +542,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(); @@ -559,6 +564,7 @@ impl SourceBackfillExecutorInner { BackfillState::SourceCachingUp(_) | BackfillState::Finished => { // backfilling stopped. ignore + vis = false } } } @@ -566,11 +572,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); + } } } } From 231b93c212976d299232dbfe79f500675dcf0e36 Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 27 Mar 2024 04:47:17 +0800 Subject: [PATCH 22/29] support scaling source backfill - We must support this? Otherwise scale source will break... We can't migrate actors?? - NOT TESTED --- src/meta/src/barrier/command.rs | 3 +- src/meta/src/stream/scale.rs | 52 +++++++++++++++++++++++---- src/meta/src/stream/source_manager.rs | 48 ++++++++++++++++++++----- 3 files changed, 86 insertions(+), 17 deletions(-) diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 36476aabd4da..ad0331a98e13 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -70,7 +70,8 @@ pub struct Reschedule { /// The downstream fragments of this fragment. pub downstream_fragment_ids: Vec, - /// Reassigned splits for source actors + /// Reassigned splits for source actors. + /// It becomes the `actor_splits` in `UpdateMutation`. pub actor_splits: HashMap>, /// Whether this fragment is injectable. The injectable means whether the fragment contains diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index f5d4a2ec5da0..9566ff465954 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -194,8 +194,10 @@ pub struct RescheduleContext { worker_nodes: HashMap, /// Index of all `Actor` upstreams, specific to `Dispatcher` upstream_dispatchers: HashMap>, - /// Fragments with stream source + /// Fragments with `StreamSource` stream_source_fragment_ids: HashSet, + /// Fragments with `StreamSourceBackfill` + stream_source_backfill_fragment_ids: HashSet, /// Target fragments in `NoShuffle` relation no_shuffle_target_fragment_ids: HashSet, /// Source fragments in `NoShuffle` relation @@ -663,6 +665,7 @@ impl ScaleController { } let mut stream_source_fragment_ids = HashSet::new(); + let mut stream_source_backfill_fragment_ids = HashSet::new(); let mut no_shuffle_reschedule = HashMap::new(); for ( fragment_id, @@ -736,6 +739,12 @@ impl ScaleController { stream_source_fragment_ids.insert(*fragment_id); } } + if (fragment.get_fragment_type_mask() & FragmentTypeFlag::SourceScan as u32) != 0 { + let stream_node = fragment.actor_template.nodes.as_ref().unwrap(); + if stream_node.find_source_backfill().is_some() { + stream_source_backfill_fragment_ids.insert(*fragment_id); + } + } // Check if the reschedule plan is valid. let current_parallel_units = fragment @@ -811,6 +820,7 @@ impl ScaleController { worker_nodes, upstream_dispatchers, stream_source_fragment_ids, + stream_source_backfill_fragment_ids, no_shuffle_target_fragment_ids, no_shuffle_source_fragment_ids, fragment_dispatcher_map, @@ -1208,9 +1218,9 @@ impl ScaleController { .await?; } - // For stream source fragments, we need to reallocate the splits. + // For stream source & source backfill fragments, we need to reallocate the splits. // Because we are in the Pause state, so it's no problem to reallocate - let mut fragment_stream_source_actor_splits = HashMap::new(); + let mut fragment_actor_splits = HashMap::new(); for fragment_id in reschedules.keys() { let actors_after_reschedule = fragment_actors_after_reschedule.get(fragment_id).unwrap(); @@ -1228,11 +1238,39 @@ impl ScaleController { let actor_splits = self .source_manager - .migrate_splits(*fragment_id, &prev_actor_ids, &curr_actor_ids) + .migrate_splits_for_source_actors( + *fragment_id, + &prev_actor_ids, + &curr_actor_ids, + ) .await?; - fragment_stream_source_actor_splits.insert(*fragment_id, actor_splits); - todo!("migrate_splits_backfill"); + fragment_actor_splits.insert(*fragment_id, actor_splits); + } + } + // Loop another round to make sure source actors are migrated first, and then align backfill actors + if !ctx.stream_source_backfill_fragment_ids.is_empty() { + for fragment_id in reschedules.keys() { + let actors_after_reschedule = + fragment_actors_after_reschedule.get(fragment_id).unwrap(); + + if ctx + .stream_source_backfill_fragment_ids + .contains(fragment_id) + { + let fragment = ctx.fragment_map.get(fragment_id).unwrap(); + + let curr_actor_ids = actors_after_reschedule.keys().cloned().collect_vec(); + + let actor_splits = self.source_manager.migrate_splits_for_backfill_actors( + *fragment_id, + &fragment.upstream_fragment_ids, + &curr_actor_ids, + &fragment_actor_splits, + &no_shuffle_upstream_actor_map, + )?; + fragment_actor_splits.insert(*fragment_id, actor_splits); + } } } @@ -1386,7 +1424,7 @@ impl ScaleController { let upstream_fragment_dispatcher_ids = upstream_fragment_dispatcher_set.into_iter().collect_vec(); - let actor_splits = fragment_stream_source_actor_splits + let actor_splits = fragment_actor_splits .get(&fragment_id) .cloned() .unwrap_or_default(); diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index 5bd931b18050..c40d8679b4ab 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -471,13 +471,13 @@ impl Default for SplitDiffOptions { } /// Reassigns splits if there are new splits or dropped splits, -/// i.e., `actor_splits` and `discovered_splits` differ. +/// i.e., `actor_splits` and `discovered_splits` differ, or actors are rescheduled. /// /// 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. +/// 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 /// @@ -774,11 +774,10 @@ impl SourceManager { /// Migrates splits from previous actors to the new actors for a rescheduled fragment. /// - /// Very occasionally split removal may happen - /// during scaling, in which case we need to use the old splits for reallocation instead of the - /// latest splits (which may be missing), so that we can resolve the split removal in the next - /// command. - pub async fn migrate_splits( + /// Very occasionally split removal may happen during scaling, in which case we need to + /// use the old splits for reallocation instead of the latest splits (which may be missing), + /// so that we can resolve the split removal in the next command. + pub async fn migrate_splits_for_source_actors( &self, fragment_id: FragmentId, prev_actor_ids: &[ActorId], @@ -801,7 +800,7 @@ impl SourceManager { fragment_id, empty_actor_splits, &prev_splits, - // pre-allocate splits is the first time getting splits and it does not have scale in scene + // pre-allocate splits is the first time getting splits and it does not have scale-in scene SplitDiffOptions::default(), ) .unwrap_or_default(); @@ -809,6 +808,37 @@ impl SourceManager { Ok(diff) } + /// Migrates splits from previous actors to the new actors for a rescheduled fragment. + pub fn migrate_splits_for_backfill_actors( + &self, + fragment_id: FragmentId, + upstream_fragment_ids: &Vec, + curr_actor_ids: &[ActorId], + fragment_actor_splits: &HashMap>>, + no_shuffle_upstream_actor_map: &HashMap>, + ) -> MetaResult>> { + // align splits for backfill fragments with its upstream source fragment + debug_assert!(upstream_fragment_ids.len() == 1); + let upstream_fragment_id = upstream_fragment_ids[0]; + let actors = no_shuffle_upstream_actor_map + .iter() + .filter(|(id, _)| curr_actor_ids.contains(id)) + .map(|(id, upstream_fragment_actors)| { + debug_assert!(upstream_fragment_actors.len() == 1); + ( + *id, + vec![*upstream_fragment_actors.get(&upstream_fragment_id).unwrap()], + ) + }); + let upstream_assignment = fragment_actor_splits.get(&upstream_fragment_id).unwrap(); + Ok(align_backfill_splits( + actors, + upstream_assignment, + fragment_id, + upstream_fragment_id, + )?) + } + /// Allocates splits to actors for a newly created source executor. pub async fn allocate_splits(&self, table_id: &TableId) -> MetaResult { let core = self.core.lock().await; From 0339b38e6044d1beaaf31ba55e85605c13bf1470 Mon Sep 17 00:00:00 2001 From: xxchan Date: Thu, 28 Mar 2024 17:25:38 +0800 Subject: [PATCH 23/29] Revert "support scaling source backfill" (For a new PR) This reverts commit 231b93c212976d299232dbfe79f500675dcf0e36. --- src/meta/src/barrier/command.rs | 3 +- src/meta/src/stream/scale.rs | 52 ++++----------------------- src/meta/src/stream/source_manager.rs | 48 +++++-------------------- 3 files changed, 17 insertions(+), 86 deletions(-) diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index ad0331a98e13..36476aabd4da 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -70,8 +70,7 @@ pub struct Reschedule { /// The downstream fragments of this fragment. pub downstream_fragment_ids: Vec, - /// Reassigned splits for source actors. - /// It becomes the `actor_splits` in `UpdateMutation`. + /// Reassigned splits for source actors pub actor_splits: HashMap>, /// Whether this fragment is injectable. The injectable means whether the fragment contains diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index 9566ff465954..f5d4a2ec5da0 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -194,10 +194,8 @@ pub struct RescheduleContext { worker_nodes: HashMap, /// Index of all `Actor` upstreams, specific to `Dispatcher` upstream_dispatchers: HashMap>, - /// Fragments with `StreamSource` + /// Fragments with stream source stream_source_fragment_ids: HashSet, - /// Fragments with `StreamSourceBackfill` - stream_source_backfill_fragment_ids: HashSet, /// Target fragments in `NoShuffle` relation no_shuffle_target_fragment_ids: HashSet, /// Source fragments in `NoShuffle` relation @@ -665,7 +663,6 @@ impl ScaleController { } let mut stream_source_fragment_ids = HashSet::new(); - let mut stream_source_backfill_fragment_ids = HashSet::new(); let mut no_shuffle_reschedule = HashMap::new(); for ( fragment_id, @@ -739,12 +736,6 @@ impl ScaleController { stream_source_fragment_ids.insert(*fragment_id); } } - if (fragment.get_fragment_type_mask() & FragmentTypeFlag::SourceScan as u32) != 0 { - let stream_node = fragment.actor_template.nodes.as_ref().unwrap(); - if stream_node.find_source_backfill().is_some() { - stream_source_backfill_fragment_ids.insert(*fragment_id); - } - } // Check if the reschedule plan is valid. let current_parallel_units = fragment @@ -820,7 +811,6 @@ impl ScaleController { worker_nodes, upstream_dispatchers, stream_source_fragment_ids, - stream_source_backfill_fragment_ids, no_shuffle_target_fragment_ids, no_shuffle_source_fragment_ids, fragment_dispatcher_map, @@ -1218,9 +1208,9 @@ impl ScaleController { .await?; } - // For stream source & source backfill fragments, we need to reallocate the splits. + // For stream source fragments, we need to reallocate the splits. // Because we are in the Pause state, so it's no problem to reallocate - let mut fragment_actor_splits = HashMap::new(); + let mut fragment_stream_source_actor_splits = HashMap::new(); for fragment_id in reschedules.keys() { let actors_after_reschedule = fragment_actors_after_reschedule.get(fragment_id).unwrap(); @@ -1238,39 +1228,11 @@ impl ScaleController { let actor_splits = self .source_manager - .migrate_splits_for_source_actors( - *fragment_id, - &prev_actor_ids, - &curr_actor_ids, - ) + .migrate_splits(*fragment_id, &prev_actor_ids, &curr_actor_ids) .await?; - fragment_actor_splits.insert(*fragment_id, actor_splits); - } - } - // Loop another round to make sure source actors are migrated first, and then align backfill actors - if !ctx.stream_source_backfill_fragment_ids.is_empty() { - for fragment_id in reschedules.keys() { - let actors_after_reschedule = - fragment_actors_after_reschedule.get(fragment_id).unwrap(); - - if ctx - .stream_source_backfill_fragment_ids - .contains(fragment_id) - { - let fragment = ctx.fragment_map.get(fragment_id).unwrap(); - - let curr_actor_ids = actors_after_reschedule.keys().cloned().collect_vec(); - - let actor_splits = self.source_manager.migrate_splits_for_backfill_actors( - *fragment_id, - &fragment.upstream_fragment_ids, - &curr_actor_ids, - &fragment_actor_splits, - &no_shuffle_upstream_actor_map, - )?; - fragment_actor_splits.insert(*fragment_id, actor_splits); - } + fragment_stream_source_actor_splits.insert(*fragment_id, actor_splits); + todo!("migrate_splits_backfill"); } } @@ -1424,7 +1386,7 @@ impl ScaleController { let upstream_fragment_dispatcher_ids = upstream_fragment_dispatcher_set.into_iter().collect_vec(); - let actor_splits = fragment_actor_splits + let actor_splits = fragment_stream_source_actor_splits .get(&fragment_id) .cloned() .unwrap_or_default(); diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index c40d8679b4ab..5bd931b18050 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -471,13 +471,13 @@ impl Default for SplitDiffOptions { } /// Reassigns splits if there are new splits or dropped splits, -/// i.e., `actor_splits` and `discovered_splits` differ, or actors are rescheduled. +/// i.e., `actor_splits` and `discovered_splits` differ. /// /// 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.** +/// 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 /// @@ -774,10 +774,11 @@ impl SourceManager { /// Migrates splits from previous actors to the new actors for a rescheduled fragment. /// - /// Very occasionally split removal may happen during scaling, in which case we need to - /// use the old splits for reallocation instead of the latest splits (which may be missing), - /// so that we can resolve the split removal in the next command. - pub async fn migrate_splits_for_source_actors( + /// Very occasionally split removal may happen + /// during scaling, in which case we need to use the old splits for reallocation instead of the + /// latest splits (which may be missing), so that we can resolve the split removal in the next + /// command. + pub async fn migrate_splits( &self, fragment_id: FragmentId, prev_actor_ids: &[ActorId], @@ -800,7 +801,7 @@ impl SourceManager { fragment_id, empty_actor_splits, &prev_splits, - // pre-allocate splits is the first time getting splits and it does not have scale-in scene + // pre-allocate splits is the first time getting splits and it does not have scale in scene SplitDiffOptions::default(), ) .unwrap_or_default(); @@ -808,37 +809,6 @@ impl SourceManager { Ok(diff) } - /// Migrates splits from previous actors to the new actors for a rescheduled fragment. - pub fn migrate_splits_for_backfill_actors( - &self, - fragment_id: FragmentId, - upstream_fragment_ids: &Vec, - curr_actor_ids: &[ActorId], - fragment_actor_splits: &HashMap>>, - no_shuffle_upstream_actor_map: &HashMap>, - ) -> MetaResult>> { - // align splits for backfill fragments with its upstream source fragment - debug_assert!(upstream_fragment_ids.len() == 1); - let upstream_fragment_id = upstream_fragment_ids[0]; - let actors = no_shuffle_upstream_actor_map - .iter() - .filter(|(id, _)| curr_actor_ids.contains(id)) - .map(|(id, upstream_fragment_actors)| { - debug_assert!(upstream_fragment_actors.len() == 1); - ( - *id, - vec![*upstream_fragment_actors.get(&upstream_fragment_id).unwrap()], - ) - }); - let upstream_assignment = fragment_actor_splits.get(&upstream_fragment_id).unwrap(); - Ok(align_backfill_splits( - actors, - upstream_assignment, - fragment_id, - upstream_fragment_id, - )?) - } - /// Allocates splits to actors for a newly created source executor. pub async fn allocate_splits(&self, table_id: &TableId) -> MetaResult { let core = self.core.lock().await; From 17e3779d8559a7fd88091acd596b57d2db463463 Mon Sep 17 00:00:00 2001 From: xxchan Date: Thu, 28 Mar 2024 18:05:40 +0800 Subject: [PATCH 24/29] refine --- .../optimizer/plan_node/stream_source_scan.rs | 1 - src/meta/src/controller/fragment.rs | 18 ++++++++++++++++++ src/meta/src/manager/catalog/fragment.rs | 3 ++- src/meta/src/manager/metadata.rs | 16 +++++++++++++--- src/meta/src/stream/scale.rs | 2 +- src/meta/src/stream/stream_manager.rs | 2 -- 6 files changed, 34 insertions(+), 8 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_source_scan.rs b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs index 7016ca4ccb56..dc1e1d33a7aa 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs @@ -151,7 +151,6 @@ impl StreamSourceScan { .to_internal_table_prost(), ), info: Some(source_catalog.info.clone()), - // XXX: what's the usage of this? row_id_index: self.core.row_id_index.map(|index| index as _), columns: self .core diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index 1ede0941ba98..842a9d75a095 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -1089,6 +1089,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() diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index 01ac5379bab5..56873c2829a7 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -1027,7 +1027,8 @@ impl FragmentManager { bail!("fragment not found: {}", fragment_id) } - pub async fn get_running_actors_and_upstream_fragment_of_fragment( + /// 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)>> { diff --git a/src/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs index 6417343f7927..39b74d42d4c6 100644 --- a/src/meta/src/manager/metadata.rs +++ b/src/meta/src/manager/metadata.rs @@ -621,11 +621,21 @@ impl MetadataManager { match self { MetadataManager::V1(mgr) => { mgr.fragment_manager - .get_running_actors_and_upstream_fragment_of_fragment(id) + .get_running_actors_and_upstream_of_fragment(id) .await } - MetadataManager::V2(_mgr) => { - todo!() + 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() + })) } } } diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index f5d4a2ec5da0..baa10428a084 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -1232,9 +1232,9 @@ impl ScaleController { .await?; fragment_stream_source_actor_splits.insert(*fragment_id, actor_splits); - todo!("migrate_splits_backfill"); } } + // TODO: support migrate splits for SourceBackfill // Generate fragment reschedule plan let mut reschedule_fragment: HashMap = diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 7cb85d91af7d..5b1720c9edad 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -462,7 +462,6 @@ impl GlobalStreamManager { } let dummy_table_id = table_fragments.table_id(); - // TODO: does this need change? for replace_table let init_split_assignment = self.source_manager.allocate_splits(&dummy_table_id).await?; @@ -534,7 +533,6 @@ impl GlobalStreamManager { .await?; let dummy_table_id = table_fragments.table_id(); - // TODO: does this need change? for replace_table let init_split_assignment = self.source_manager.allocate_splits(&dummy_table_id).await?; if let Err(err) = self From 2e5c1c0ed3a9aa4539dab1099e7eccbcb4c1750e Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 29 Mar 2024 00:53:33 +0800 Subject: [PATCH 25/29] fix --- src/meta/src/manager/metadata.rs | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/src/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs index 39b74d42d4c6..9176a0e4077d 100644 --- a/src/meta/src/manager/metadata.rs +++ b/src/meta/src/manager/metadata.rs @@ -629,13 +629,15 @@ impl MetadataManager { .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() - })) + Ok(actor_ids + .into_iter() + .map(|(id, actors)| { + ( + id as ActorId, + actors.into_iter().map(|id| id as ActorId).collect(), + ) + }) + .collect()) } } } From e174c1279bd38f49111ff7c35d83da0cb59ffd83 Mon Sep 17 00:00:00 2001 From: xxchan Date: Mon, 1 Apr 2024 09:06:41 +0800 Subject: [PATCH 26/29] don't deprecate cdc_source_job for compatibility --- proto/catalog.proto | 16 +++++++++------- src/connector/src/source/cdc/mod.rs | 2 +- .../src/binder/relation/table_or_source.rs | 2 +- src/frontend/src/handler/create_source.rs | 8 +++++--- .../src/optimizer/plan_node/logical_source.rs | 4 +++- .../src/optimizer/plan_node/stream_source.rs | 2 +- .../optimizer/plan_node/stream_source_scan.rs | 2 +- src/frontend/src/stream_fragmenter/mod.rs | 3 +-- src/meta/src/controller/catalog.rs | 2 +- src/meta/src/controller/streaming_job.rs | 2 +- src/meta/src/manager/catalog/database.rs | 7 ++++++- src/meta/src/manager/catalog/mod.rs | 2 +- src/prost/src/lib.rs | 7 +++++++ 13 files changed, 38 insertions(+), 21 deletions(-) diff --git a/proto/catalog.proto b/proto/catalog.proto index 74e6267bd1d4..2596f53b7828 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -62,16 +62,18 @@ message StreamSourceInfo { SchemaRegistryNameStrategy name_strategy = 10; optional string key_message_name = 11; plan_common.ExternalTableDesc external_table = 12; - // Whether the stream source has a streaming job. + // Whether the stream source is a cdc source streaming job. + // We need this field to differentiate the cdc source job, because CDC source is a singleton fragment, while Kafka source is not. + // If we have other singleton source jobs in the future, we may consider deprecating this field and add a new field `is_distributed`. + bool cdc_source_job = 13; + // 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 have streaming jobs: - // - Direct CDC sources (mysql & postgresql) + // - Direct CDC sources (mysql & postgresql). Backwards compat note: For old CDC job it's `None`; For new CDC job it's `Some(true)`. // - MQ sources (Kafka, Pulsar, Kinesis, etc.) - bool is_shared = 13; - // Only used when `is_shared` is `true`. - // If `false`, `requires_singleton` will be set in the stream plan. - bool is_distributed = 15; - reserved "cdc_source_job"; // deprecated + // + // **Should also test `cdc_source_job` for backwards compatibility. Use `is_shared_compatible()` instead of this field directly.** + optional bool is_shared = 15; // Options specified by user in the FORMAT ENCODE clause. map format_encode_options = 14; } diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index c8512118bd5a..af61f4aa07f9 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.is_shared; + self.is_cdc_source_job = info.is_shared_compatible(); } } diff --git a/src/frontend/src/binder/relation/table_or_source.rs b/src/frontend/src/binder/relation/table_or_source.rs index babf3fb73478..aa49797b92eb 100644 --- a/src/frontend/src/binder/relation/table_or_source.rs +++ b/src/frontend/src/binder/relation/table_or_source.rs @@ -60,7 +60,7 @@ impl BoundSource { } pub fn is_shared(&self) -> bool { - self.catalog.info.is_shared + self.catalog.info.is_shared_compatible() } } diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index d7a8a3cd2c4e..2b363821c885 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -511,7 +511,7 @@ fn bind_columns_from_source_for_cdc( row_encode: row_encode_to_prost(&source_schema.row_encode) as i32, format_encode_options, use_schema_registry: json_schema_infer_use_schema_registry(&schema_config), - is_shared: true, + is_shared: Some(true), ..Default::default() }; if !format_encode_options_to_consume.is_empty() { @@ -1321,9 +1321,11 @@ pub async fn handle_create_source( } else { bind_columns_from_source(&session, &source_schema, &with_properties).await? }; + if create_cdc_source_job { + source_info.cdc_source_job = true; + } if is_shared { - source_info.is_shared = true; - source_info.is_distributed = !create_cdc_source_job; + source_info.is_shared = Some(true); } let columns_from_sql = bind_sql_columns(&stmt.columns)?; diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 7010e99867ce..9bfe28b2d55c 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -532,7 +532,9 @@ impl ToStream for LogicalSource { } SourceNodeKind::CreateMViewOrBatch => { // Create MV on source. - let use_shared_source = self.source_catalog().is_some_and(|c| c.info.is_shared) + let use_shared_source = self + .source_catalog() + .is_some_and(|c| c.info.is_shared_compatible()) && self.ctx().session_ctx().config().rw_enable_shared_source(); if use_shared_source { plan = StreamSourceScan::new(self.core.clone()).into(); diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index a4df97c5bc4c..41f336800669 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -44,7 +44,7 @@ impl StreamSource { // 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 + && source_catalog.info.is_shared_compatible() { let (columns_exist, additional_columns) = add_partition_offset_cols(&core.column_catalog, &source_catalog.connector_name()); diff --git a/src/frontend/src/optimizer/plan_node/stream_source_scan.rs b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs index dc1e1d33a7aa..827a1714d449 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs @@ -56,7 +56,7 @@ impl StreamSourceScan { // 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 + && source_catalog.info.is_shared_compatible() { let (columns_exist, additional_columns) = add_partition_offset_cols(&core.column_catalog, &source_catalog.connector_name()); diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index 95c77423c487..cc8aa6785ab0 100644 --- a/src/frontend/src/stream_fragmenter/mod.rs +++ b/src/frontend/src/stream_fragmenter/mod.rs @@ -264,8 +264,7 @@ fn build_fragment( if let Some(source) = node.source_inner.as_ref() && let Some(source_info) = source.info.as_ref() - && source_info.is_shared - && !source_info.is_distributed + && source_info.cdc_source_job { current_fragment.requires_singleton = true; } diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index ecc716c5aa5b..8595072ca109 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -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().is_shared + && source_info.into_inner().is_shared_compatible() { to_drop_streaming_jobs.push(object_id); } diff --git a/src/meta/src/controller/streaming_job.rs b/src/meta/src/controller/streaming_job.rs index f108fc33ac98..9e3a4bad406f 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().is_shared + && source_info.inner_ref().is_shared_compatible() { vec![source_id] } else { diff --git a/src/meta/src/manager/catalog/database.rs b/src/meta/src/manager/catalog/database.rs index e6c2da61018b..5fcba49f6ce6 100644 --- a/src/meta/src/manager/catalog/database.rs +++ b/src/meta/src/manager/catalog/database.rs @@ -404,7 +404,12 @@ impl DatabaseManager { .chain( self.sources .iter() - .filter(|(_, source)| source.info.as_ref().is_some_and(|info| info.is_shared)) + .filter(|(_, source)| { + source + .info + .as_ref() + .is_some_and(|info| info.is_shared_compatible()) + }) .map(|(id, _)| id) .copied(), ) diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 1b9709398fbe..e8f4e7104414 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -1454,7 +1454,7 @@ impl CatalogManager { } if let Some(info) = source.info - && info.is_shared + && info.is_shared_compatible() { all_streaming_job_source_ids.insert(source.id); let source_table_fragments = fragment_manager diff --git a/src/prost/src/lib.rs b/src/prost/src/lib.rs index ac7fc0b1c4a4..fb4746f270d6 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -251,6 +251,13 @@ impl stream_plan::StreamNode { } } +impl catalog::StreamSourceInfo { + /// Also tested `cdc_source_job` for backwards compatibility. Use this instead of `is_shared` directly. + pub fn is_shared_compatible(&self) -> bool { + self.cdc_source_job || self.is_shared.is_some_and(|b| b) + } +} + #[cfg(test)] mod tests { use crate::data::{data_type, DataType}; From 98cfb7206640651fdaee29b08fd6d86724798ffc Mon Sep 17 00:00:00 2001 From: xxchan Date: Mon, 1 Apr 2024 09:25:13 +0800 Subject: [PATCH 27/29] minor fix --- src/frontend/src/optimizer/plan_node/stream_source_scan.rs | 4 ++-- src/frontend/src/planner/relation.rs | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_source_scan.rs b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs index 827a1714d449..75a4203818c8 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs @@ -91,7 +91,7 @@ impl StreamSourceScan { self.core .catalog .clone() - .expect("source backfill should have source cataglog") + .expect("source scan should have source cataglog") } pub fn infer_internal_table_catalog() -> TableCatalog { @@ -203,6 +203,6 @@ impl ExprVisitable for StreamSourceScan {} impl StreamNode for StreamSourceScan { fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> NodeBody { - unreachable!("stream source backfill cannot be converted into a prost body -- call `adhoc_to_stream_prost` instead.") + 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 bc75bdda5c35..74060b484e9c 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -97,10 +97,10 @@ impl Planner { pub(super) fn plan_source(&mut self, source: BoundSource) -> Result { if source.is_backfillable_cdc_connector() { - return Err(ErrorCode::InternalError( + Err(ErrorCode::InternalError( "Should not create MATERIALIZED VIEW or SELECT directly on shared CDC source. HINT: create TABLE from the source instead.".to_string(), ) - .into()); + .into()) } else { let as_of = source.as_of.clone(); match as_of { From f169dec8cdf759842a5cd02d0bca6cc5f4bae8f1 Mon Sep 17 00:00:00 2001 From: xxchan Date: Tue, 2 Apr 2024 14:46:03 +0800 Subject: [PATCH 28/29] proto: rename cdc_source_job without renaming --- proto/catalog.proto | 22 +++++++++++-------- src/connector/src/source/cdc/mod.rs | 2 +- .../src/binder/relation/table_or_source.rs | 2 +- src/frontend/src/handler/create_source.rs | 10 ++++----- .../src/optimizer/plan_node/logical_source.rs | 4 +--- .../src/optimizer/plan_node/stream_source.rs | 2 +- .../optimizer/plan_node/stream_source_scan.rs | 2 +- src/frontend/src/stream_fragmenter/mod.rs | 3 ++- src/meta/src/controller/catalog.rs | 2 +- src/meta/src/controller/streaming_job.rs | 2 +- src/meta/src/manager/catalog/database.rs | 7 +----- src/meta/src/manager/catalog/mod.rs | 2 +- src/prost/src/lib.rs | 6 ++--- 13 files changed, 32 insertions(+), 34 deletions(-) diff --git a/proto/catalog.proto b/proto/catalog.proto index 2596f53b7828..c6d83f51612c 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -62,18 +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, because CDC source is a singleton fragment, while Kafka source is not. - // If we have other singleton source jobs in the future, we may consider deprecating this field and add a new field `is_distributed`. - bool cdc_source_job = 13; + // **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 have streaming jobs: - // - Direct CDC sources (mysql & postgresql). Backwards compat note: For old CDC job it's `None`; For new CDC job it's `Some(true)`. - // - MQ sources (Kafka, Pulsar, Kinesis, etc.) // - // **Should also test `cdc_source_job` for backwards compatibility. Use `is_shared_compatible()` instead of this field directly.** - optional bool is_shared = 15; + // 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/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index af61f4aa07f9..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.is_shared_compatible(); + self.is_cdc_source_job = info.is_shared(); } } diff --git a/src/frontend/src/binder/relation/table_or_source.rs b/src/frontend/src/binder/relation/table_or_source.rs index aa49797b92eb..a0591b4859c4 100644 --- a/src/frontend/src/binder/relation/table_or_source.rs +++ b/src/frontend/src/binder/relation/table_or_source.rs @@ -60,7 +60,7 @@ impl BoundSource { } pub fn is_shared(&self) -> bool { - self.catalog.info.is_shared_compatible() + self.catalog.info.is_shared() } } diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 2b363821c885..60d9dd284bf1 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -511,7 +511,8 @@ fn bind_columns_from_source_for_cdc( row_encode: row_encode_to_prost(&source_schema.row_encode) as i32, format_encode_options, use_schema_registry: json_schema_infer_use_schema_registry(&schema_config), - is_shared: Some(true), + cdc_source_job: true, + is_distributed: false, ..Default::default() }; if !format_encode_options_to_consume.is_empty() { @@ -1321,11 +1322,10 @@ pub async fn handle_create_source( } else { bind_columns_from_source(&session, &source_schema, &with_properties).await? }; - if create_cdc_source_job { - source_info.cdc_source_job = true; - } if is_shared { - source_info.is_shared = Some(true); + // 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)?; diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 9bfe28b2d55c..4cca0ec35486 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -532,9 +532,7 @@ impl ToStream for LogicalSource { } SourceNodeKind::CreateMViewOrBatch => { // Create MV on source. - let use_shared_source = self - .source_catalog() - .is_some_and(|c| c.info.is_shared_compatible()) + 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(); diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index 41f336800669..7b918a6312a6 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -44,7 +44,7 @@ impl StreamSource { // 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_compatible() + && source_catalog.info.is_shared() { let (columns_exist, additional_columns) = add_partition_offset_cols(&core.column_catalog, &source_catalog.connector_name()); diff --git a/src/frontend/src/optimizer/plan_node/stream_source_scan.rs b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs index 75a4203818c8..597517ce1e3f 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs @@ -56,7 +56,7 @@ impl StreamSourceScan { // 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_compatible() + && source_catalog.info.is_shared() { let (columns_exist, additional_columns) = add_partition_offset_cols(&core.column_catalog, &source_catalog.connector_name()); diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index cc8aa6785ab0..009449ec9228 100644 --- a/src/frontend/src/stream_fragmenter/mod.rs +++ b/src/frontend/src/stream_fragmenter/mod.rs @@ -264,7 +264,8 @@ 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 { current_fragment.requires_singleton = true; } diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index 8595072ca109..97fd0e081421 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -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().is_shared_compatible() + && source_info.into_inner().is_shared() { to_drop_streaming_jobs.push(object_id); } diff --git a/src/meta/src/controller/streaming_job.rs b/src/meta/src/controller/streaming_job.rs index 9e3a4bad406f..439c23c80134 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().is_shared_compatible() + && source_info.inner_ref().is_shared() { vec![source_id] } else { diff --git a/src/meta/src/manager/catalog/database.rs b/src/meta/src/manager/catalog/database.rs index 5fcba49f6ce6..54a0bea85dd1 100644 --- a/src/meta/src/manager/catalog/database.rs +++ b/src/meta/src/manager/catalog/database.rs @@ -404,12 +404,7 @@ impl DatabaseManager { .chain( self.sources .iter() - .filter(|(_, source)| { - source - .info - .as_ref() - .is_some_and(|info| info.is_shared_compatible()) - }) + .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/mod.rs b/src/meta/src/manager/catalog/mod.rs index e8f4e7104414..7b8fa6bff760 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -1454,7 +1454,7 @@ impl CatalogManager { } if let Some(info) = source.info - && info.is_shared_compatible() + && info.is_shared() { all_streaming_job_source_ids.insert(source.id); let source_table_fragments = fragment_manager diff --git a/src/prost/src/lib.rs b/src/prost/src/lib.rs index fb4746f270d6..825e721891fe 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -252,9 +252,9 @@ impl stream_plan::StreamNode { } impl catalog::StreamSourceInfo { - /// Also tested `cdc_source_job` for backwards compatibility. Use this instead of `is_shared` directly. - pub fn is_shared_compatible(&self) -> bool { - self.cdc_source_job || self.is_shared.is_some_and(|b| b) + /// Refer to [`Self::cdc_source_job`] for details. + pub fn is_shared(&self) -> bool { + self.cdc_source_job } } From aafba453084b593c927c169bfe20afd7aad4b1ee Mon Sep 17 00:00:00 2001 From: xxchan Date: Tue, 2 Apr 2024 14:54:15 +0800 Subject: [PATCH 29/29] rename backfillable -> shareable --- proto/ddl_service.proto | 2 +- src/connector/src/with_options.rs | 3 ++- src/frontend/src/binder/relation/table_or_source.rs | 4 ++-- src/frontend/src/handler/create_source.rs | 2 +- src/frontend/src/optimizer/plan_node/logical_source.rs | 2 +- src/frontend/src/planner/relation.rs | 2 +- src/meta/src/manager/metadata.rs | 2 +- src/stream/src/executor/source/source_executor.rs | 2 +- 8 files changed, 10 insertions(+), 9 deletions(-) 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/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/src/binder/relation/table_or_source.rs b/src/frontend/src/binder/relation/table_or_source.rs index a0591b4859c4..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,8 @@ 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 { diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 60d9dd284bf1..6691b457da61 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -1313,7 +1313,7 @@ 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()); diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 4cca0ec35486..fd4aaba5b7d2 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -522,7 +522,7 @@ impl ToStream for LogicalSource { match self.core.kind { SourceNodeKind::CreateTable | SourceNodeKind::CreateSharedSource => { // Note: for create table, row_id and generated columns is created in plan_root.gen_table_plan. - // for backfill-able source, row_id and generated columns is created after SourceBackfill node. + // 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(); 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/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs index b0f70ed1dddd..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( 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);