diff --git a/.git-blame-ignore-revs b/.git-blame-ignore-revs index 6efd862273624..b8ca322d767a8 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 ec7c68a3802ba..f376d4dc3bed7 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/proto/stream_plan.proto b/proto/stream_plan.proto index 0ccf6718ecad3..408c14b2b6044 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -758,6 +758,7 @@ message StreamNode { StreamFsFetchNode stream_fs_fetch = 138; StreamCdcScanNode stream_cdc_scan = 139; CdcFilterNode cdc_filter = 140; + SourceNode source_backfill = 141; } // The id for the operator. This is local per mview. // TODO: should better be a uint32. @@ -852,6 +853,7 @@ enum FragmentTypeFlag { FRAGMENT_TYPE_FLAG_VALUES = 64; FRAGMENT_TYPE_FLAG_DML = 128; FRAGMENT_TYPE_FLAG_CDC_FILTER = 256; + FRAGMENT_TYPE_FLAG_SOURCE_BACKFILL = 512; } // The streaming context associated with a stream plan diff --git a/src/common/src/util/iter_util.rs b/src/common/src/util/iter_util.rs index 92f19a0ee46fc..7003c80e926e6 100644 --- a/src/common/src/util/iter_util.rs +++ b/src/common/src/util/iter_util.rs @@ -12,6 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::borrow::Borrow; +use std::ops::Deref; + pub trait ZipEqFast: ExactSizeIterator + Sized where B::IntoIter: ExactSizeIterator, @@ -54,3 +57,29 @@ where { a.into_iter().zip_eq_fast(b) } + +pub trait IntoIteratorExt +where + for<'a> &'a Self: IntoIterator, +{ + /// Shorter version of `self.iter().map(f).collect()`. + fn map_collect(&self, f: F) -> BCollection + where + F: FnMut(&A) -> B, + for<'a> &'a Self: IntoIterator, + BCollection: FromIterator, + { + self.into_iter().map(f).collect() + } + + /// Shorter version of `self.iter().map(f).collect_vec()`. + fn map_to_vec(&self, f: F) -> Vec + where + F: FnMut(&A) -> B, + for<'a> &'a Self: IntoIterator, + { + self.map_collect(f) + } +} + +impl IntoIteratorExt for T where for<'a> &'a Self: IntoIterator {} diff --git a/src/common/src/util/stream_graph_visitor.rs b/src/common/src/util/stream_graph_visitor.rs index ce2820752f120..ab5aed59fca2e 100644 --- a/src/common/src/util/stream_graph_visitor.rs +++ b/src/common/src/util/stream_graph_visitor.rs @@ -187,6 +187,12 @@ pub fn visit_stream_node_tables_inner( always!(source.state_table, "FsFetch"); } } + NodeBody::SourceBackfill(node) => { + always!( + node.source_inner.as_mut().unwrap().state_table, + "SourceBackfill" + ) + } // Sink NodeBody::Sink(node) => { diff --git a/src/compute/src/rpc/service/stream_service.rs b/src/compute/src/rpc/service/stream_service.rs index 22da99e58a414..fdce2d6bb27f2 100644 --- a/src/compute/src/rpc/service/stream_service.rs +++ b/src/compute/src/rpc/service/stream_service.rs @@ -191,6 +191,7 @@ impl StreamService for StreamServiceImpl { .inspect_err( |err| tracing::error!(error = %err.as_report(), "failed to collect barrier"), )?; + tracing::trace!(?create_mview_progress, ?kind, "barrier_complete"); let (synced_sstables, table_watermarks) = match kind { BarrierKind::Unspecified => unreachable!(), diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index a6ddb359bacdb..6583aa5a49d1c 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -72,8 +72,10 @@ pub trait SourceProperties: TryFromHashmap + Clone + WithOptions { type SplitEnumerator: SplitEnumerator; type SplitReader: SplitReader; + /// Load additional info from `PbSource`. Currently only used by CDC. fn init_from_pb_source(&mut self, _source: &PbSource) {} + /// Load additional info from `ExternalTableDesc`. Currently only used by CDC. fn init_from_pb_cdc_table_desc(&mut self, _table_desc: &ExternalTableDesc) {} } @@ -447,10 +449,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 b3a2bc6554c60..03dc99ec6a9fd 100644 --- a/src/connector/src/source/cdc/mod.rs +++ b/src/connector/src/source/cdc/mod.rs @@ -136,7 +136,7 @@ where }; self.table_schema = table_schema; if let Some(info) = source.info.as_ref() { - self.is_multi_table_shared = info.cdc_source_job; + self.is_multi_table_shared = info.has_streaming_job; } } diff --git a/src/frontend/src/binder/relation/table_or_source.rs b/src/frontend/src/binder/relation/table_or_source.rs index 8c16f14d7ce71..17c16b16dd5fc 100644 --- a/src/frontend/src/binder/relation/table_or_source.rs +++ b/src/frontend/src/binder/relation/table_or_source.rs @@ -58,6 +58,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/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index fbf9248f1bfb9..3c690691ee752 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -474,7 +474,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() { @@ -1130,18 +1130,22 @@ 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 is_cdc_connector(&with_properties) { CdcTableType::from_properties(&with_properties).can_backfill() } else { false }; + let has_streaming_job = create_cdc_source_job || is_kafka_connector(&with_properties); - 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( @@ -1235,21 +1239,18 @@ 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::new( Some(Rc::new(SourceCatalog::from(&source))), columns.clone(), row_id_index, - false, + false, // Do not gen RowID. Gen RowID after backfill node instead. false, 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 f86f31c1e0765..08c8ae1059c0e 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -565,6 +565,8 @@ impl ToStream for LogicalSource { { plan_prefix = Some(self.rewrite_new_s3_plan()?); } + + // TODO: after SourceBackfill is added, we shouldn't put generated columns/row id here, and put them after backfill instead. plan = if self.core.for_table { dispatch_new_s3_plan(self.rewrite_to_stream_batch_source(), plan_prefix) } else { 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 0000000000000..3cd3e08986f74 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/logical_source_backfill.rs @@ -0,0 +1,588 @@ +// 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::cmp::{max, min}; +use std::ops::Bound; +use std::ops::Bound::{Excluded, Included, Unbounded}; +use std::rc::Rc; + +use fixedbitset::FixedBitSet; +use itertools::Itertools; +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::bail_not_implemented; +use risingwave_common::catalog::{ + ColumnCatalog, ColumnDesc, Field, Schema, KAFKA_TIMESTAMP_COLUMN_NAME, +}; +use risingwave_common::error::Result; +use risingwave_connector::source::{ConnectorProperties, DataType}; +use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; +use risingwave_pb::plan_common::GeneratedColumnDesc; + +use super::generic::GenericPlanRef; +use super::stream_watermark_filter::StreamWatermarkFilter; +use super::utils::{childless_record, Distill}; +use super::{ + generic, BatchProject, BatchSource, ColPrunable, ExprRewritable, Logical, LogicalFilter, + LogicalProject, PlanBase, PlanRef, PredicatePushdown, StreamProject, StreamRowIdGen, + StreamSource, ToBatch, ToStream, +}; +use crate::catalog::source_catalog::SourceCatalog; +use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, ExprVisitor, InputRef}; +use crate::optimizer::optimizer_context::OptimizerContextRef; +use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::plan_node::stream_fs_fetch::StreamFsFetch; +use crate::optimizer::plan_node::utils::column_names_pretty; +use crate::optimizer::plan_node::{ + ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, StreamDedup, + StreamSourceBackfill, ToStreamContext, +}; +use crate::optimizer::property::Distribution::HashShard; +use crate::optimizer::property::{Distribution, Order, RequiredDist}; +use crate::utils::{ColIndexMapping, Condition, IndexRewriter}; + +/// `LogicalSourceBackfill` returns contents of a table or other equivalent object +#[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>, +} + +impl LogicalSourceBackfill { + pub fn new( + source_catalog: Option>, + column_catalog: Vec, + row_id_index: Option, + gen_row_id: bool, + for_table: bool, + ctx: OptimizerContextRef, + ) -> Result { + let kafka_timestamp_range = (Bound::Unbounded, Bound::Unbounded); + let core = generic::Source { + catalog: source_catalog, + column_catalog, + row_id_index, + gen_row_id, + for_table, + ctx, + kafka_timestamp_range, + }; + + let base = PlanBase::new_logical_with_core(&core); + + let output_exprs = Self::derive_output_exprs_from_generated_columns(&core.column_catalog)?; + + Ok(LogicalSourceBackfill { + base, + core, + output_exprs, + }) + } + + pub fn with_catalog( + source_catalog: Rc, + for_table: bool, + ctx: OptimizerContextRef, + ) -> Result { + let column_catalogs = source_catalog.columns.clone(); + let row_id_index = source_catalog.row_id_index; + let gen_row_id = source_catalog.append_only; + + Self::new( + Some(source_catalog), + column_catalogs, + row_id_index, + gen_row_id, + for_table, + ctx, + ) + } + + pub fn derive_output_exprs_from_generated_columns( + columns: &[ColumnCatalog], + ) -> Result>> { + if !columns.iter().any(|c| c.is_generated()) { + return Ok(None); + } + + let col_mapping = { + let mut mapping = vec![None; columns.len()]; + let mut cur = 0; + for (idx, column) in columns.iter().enumerate() { + if !column.is_generated() { + mapping[idx] = Some(cur); + cur += 1; + } else { + mapping[idx] = None; + } + } + ColIndexMapping::new(mapping, columns.len()) + }; + + let mut rewriter = IndexRewriter::new(col_mapping); + let mut exprs = Vec::with_capacity(columns.len()); + let mut cur = 0; + for column in columns { + let column_desc = &column.column_desc; + let ret_data_type = column_desc.data_type.clone(); + + if let Some(GeneratedOrDefaultColumn::GeneratedColumn(generated_column)) = + &column_desc.generated_or_default_column + { + let GeneratedColumnDesc { expr } = generated_column; + // TODO(yuhao): avoid this `from_expr_proto`. + let proj_expr = + rewriter.rewrite_expr(ExprImpl::from_expr_proto(expr.as_ref().unwrap())?); + let casted_expr = proj_expr.cast_assign(ret_data_type)?; + exprs.push(casted_expr); + } else { + let input_ref = InputRef { + data_type: ret_data_type, + index: cur, + }; + cur += 1; + exprs.push(ExprImpl::InputRef(Box::new(input_ref))); + } + } + + Ok(Some(exprs)) + } + + fn rewrite_new_s3_plan(&self) -> Result { + let logical_source = generic::Source { + catalog: self.core.catalog.clone(), + column_catalog: vec![ + ColumnCatalog { + column_desc: ColumnDesc::from_field_with_column_id( + &Field { + name: "filename".to_string(), + data_type: DataType::Varchar, + sub_fields: vec![], + type_name: "".to_string(), + }, + 0, + ), + is_hidden: false, + }, + ColumnCatalog { + column_desc: ColumnDesc::from_field_with_column_id( + &Field { + name: "last_edit_time".to_string(), + data_type: DataType::Timestamptz, + sub_fields: vec![], + type_name: "".to_string(), + }, + 1, + ), + is_hidden: false, + }, + ColumnCatalog { + column_desc: ColumnDesc::from_field_with_column_id( + &Field { + name: "file_size".to_string(), + data_type: DataType::Int64, + sub_fields: vec![], + type_name: "".to_string(), + }, + 0, + ), + is_hidden: false, + }, + ], + row_id_index: None, + gen_row_id: false, + ..self.core.clone() + }; + let mut new_s3_plan: PlanRef = StreamSource { + base: PlanBase::new_stream_with_core( + &logical_source, + Distribution::Single, + true, // `list` will keep listing all objects, it must be append-only + false, + FixedBitSet::with_capacity(logical_source.column_catalog.len()), + ), + core: logical_source, + } + .into(); + new_s3_plan = RequiredDist::shard_by_key(3, &[0]) + .enforce_if_not_satisfies(new_s3_plan, &Order::any())?; + new_s3_plan = StreamDedup::new(generic::Dedup { + input: new_s3_plan, + dedup_cols: vec![0], + }) + .into(); + + Ok(new_s3_plan) + } + + /// `row_id_index` in source node should rule out generated column + #[must_use] + fn rewrite_row_id_idx(columns: &[ColumnCatalog], row_id_index: Option) -> Option { + row_id_index.map(|idx| { + let mut cnt = 0; + for col in columns.iter().take(idx + 1) { + if col.is_generated() { + cnt += 1; + } + } + idx - cnt + }) + } + + pub fn source_catalog(&self) -> Rc { + self.core + .catalog + .clone() + .expect("source catalog should exist for LogicalSourceBackfill") + } + + fn clone_with_kafka_timestamp_range(&self, range: (Bound, Bound)) -> Self { + let mut core = self.core.clone(); + core.kafka_timestamp_range = range; + Self { + base: self.base.clone(), + core, + output_exprs: self.output_exprs.clone(), + } + } + + /// The columns in stream/batch source node indicate the actual columns it will produce, + /// instead of the columns defined in source catalog. The difference is generated columns. + #[must_use] + fn rewrite_to_stream_batch_source(&self) -> generic::Source { + let column_catalog = self.core.column_catalog.clone(); + // Filter out the generated columns. + let row_id_index = Self::rewrite_row_id_idx(&column_catalog, self.core.row_id_index); + let source_column_catalogs = column_catalog + .into_iter() + .filter(|c| !c.is_generated()) + .collect_vec(); + generic::Source { + catalog: self.core.catalog.clone(), + column_catalog: source_column_catalogs, + row_id_index, + ctx: self.core.ctx.clone(), + ..self.core + } + } + + fn wrap_with_optional_generated_columns_stream_proj( + &self, + input: Option, + ) -> Result { + if let Some(exprs) = &self.output_exprs { + let source: PlanRef = + dispatch_new_s3_plan(self.rewrite_to_stream_batch_source(), input); + let logical_project = generic::Project::new(exprs.to_vec(), source); + Ok(StreamProject::new(logical_project).into()) + } else { + let source = dispatch_new_s3_plan(self.core.clone(), input); + Ok(source) + } + } + + fn wrap_with_optional_generated_columns_batch_proj(&self) -> Result { + if let Some(exprs) = &self.output_exprs { + let source = BatchSource::new(self.rewrite_to_stream_batch_source()); + let logical_project = generic::Project::new(exprs.to_vec(), source.into()); + Ok(BatchProject::new(logical_project).into()) + } else { + let source = BatchSource::new(self.core.clone()); + Ok(source.into()) + } + } +} + +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)); + } +} + +/// A util function to extract kafka offset timestamp range. +/// +/// Currently we only support limiting kafka offset timestamp range using literals, e.g. we only +/// support expressions like `_rw_kafka_timestamp <= '2022-10-11 1:00:00+00:00'`. +/// +/// # Parameters +/// +/// * `expr`: Expression to be consumed. +/// * `range`: Original timestamp range, if `expr` can be recognized, we will update `range`. +/// * `schema`: Input schema. +/// +/// # Return Value +/// +/// If `expr` can be recognized and consumed by this function, then we return `None`. +/// Otherwise `expr` is returned. +fn expr_to_kafka_timestamp_range( + expr: ExprImpl, + range: &mut (Bound, Bound), + schema: &Schema, +) -> Option { + let merge_upper_bound = |first, second| -> Bound { + match (first, second) { + (first, Unbounded) => first, + (Unbounded, second) => second, + (Included(f1), Included(f2)) => Included(min(f1, f2)), + (Included(f1), Excluded(f2)) => { + if f1 < f2 { + Included(f1) + } else { + Excluded(f2) + } + } + (Excluded(f1), Included(f2)) => { + if f2 < f1 { + Included(f2) + } else { + Excluded(f1) + } + } + (Excluded(f1), Excluded(f2)) => Excluded(min(f1, f2)), + } + }; + + let merge_lower_bound = |first, second| -> Bound { + match (first, second) { + (first, Unbounded) => first, + (Unbounded, second) => second, + (Included(f1), Included(f2)) => Included(max(f1, f2)), + (Included(f1), Excluded(f2)) => { + if f1 > f2 { + Included(f1) + } else { + Excluded(f2) + } + } + (Excluded(f1), Included(f2)) => { + if f2 > f1 { + Included(f2) + } else { + Excluded(f1) + } + } + (Excluded(f1), Excluded(f2)) => Excluded(max(f1, f2)), + } + }; + + let extract_timestampz_literal = |expr: &ExprImpl| -> Result> { + match expr { + ExprImpl::FunctionCall(function_call) if function_call.inputs().len() == 2 => { + match (&function_call.inputs()[0], &function_call.inputs()[1]) { + (ExprImpl::InputRef(input_ref), literal) + if let Some(datum) = literal.try_fold_const().transpose()? + && schema.fields[input_ref.index].name + == KAFKA_TIMESTAMP_COLUMN_NAME + && literal.return_type() == DataType::Timestamptz => + { + Ok(Some(( + datum.unwrap().into_timestamptz().timestamp_millis(), + false, + ))) + } + (literal, ExprImpl::InputRef(input_ref)) + if let Some(datum) = literal.try_fold_const().transpose()? + && schema.fields[input_ref.index].name + == KAFKA_TIMESTAMP_COLUMN_NAME + && literal.return_type() == DataType::Timestamptz => + { + Ok(Some(( + datum.unwrap().into_timestamptz().timestamp_millis(), + true, + ))) + } + _ => Ok(None), + } + } + _ => Ok(None), + } + }; + + match &expr { + ExprImpl::FunctionCall(function_call) => { + if let Ok(Some((timestampz_literal, reverse))) = extract_timestampz_literal(&expr) { + match function_call.func_type() { + ExprType::GreaterThan => { + if reverse { + range.1 = merge_upper_bound(range.1, Excluded(timestampz_literal)); + } else { + range.0 = merge_lower_bound(range.0, Excluded(timestampz_literal)); + } + + None + } + ExprType::GreaterThanOrEqual => { + if reverse { + range.1 = merge_upper_bound(range.1, Included(timestampz_literal)); + } else { + range.0 = merge_lower_bound(range.0, Included(timestampz_literal)); + } + None + } + ExprType::Equal => { + range.0 = merge_lower_bound(range.0, Included(timestampz_literal)); + range.1 = merge_upper_bound(range.1, Included(timestampz_literal)); + None + } + ExprType::LessThan => { + if reverse { + range.0 = merge_lower_bound(range.0, Excluded(timestampz_literal)); + } else { + range.1 = merge_upper_bound(range.1, Excluded(timestampz_literal)); + } + None + } + ExprType::LessThanOrEqual => { + if reverse { + range.0 = merge_lower_bound(range.0, Included(timestampz_literal)); + } else { + range.1 = merge_upper_bound(range.1, Included(timestampz_literal)); + } + None + } + _ => Some(expr), + } + } else { + Some(expr) + } + } + _ => Some(expr), + } +} + +impl PredicatePushdown for LogicalSourceBackfill { + fn predicate_pushdown( + &self, + predicate: Condition, + _ctx: &mut PredicatePushdownContext, + ) -> PlanRef { + let mut range = self.core.kafka_timestamp_range; + + let mut new_conjunctions = Vec::with_capacity(predicate.conjunctions.len()); + for expr in predicate.conjunctions { + if let Some(e) = expr_to_kafka_timestamp_range(expr, &mut range, self.base.schema()) { + // Not recognized, so push back + new_conjunctions.push(e); + } + } + + let new_source = self.clone_with_kafka_timestamp_range(range).into(); + + if new_conjunctions.is_empty() { + new_source + } else { + LogicalFilter::create( + new_source, + Condition { + conjunctions: new_conjunctions, + }, + ) + } + } +} + +impl ToBatch for LogicalSourceBackfill { + fn to_batch(&self) -> Result { + // TODO: + let source = BatchSource::new(self.core.clone()); + Ok(source.into()) + } +} + +impl ToStream for LogicalSourceBackfill { + fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result { + let mut plan = StreamSourceBackfill::new(self.rewrite_to_stream_batch_source()).into(); + + let catalog = self.source_catalog(); + if !catalog.watermark_descs.is_empty() && !self.core.for_table { + plan = StreamWatermarkFilter::new(plan, catalog.watermark_descs.clone()).into(); + } + + assert!(!(self.core.gen_row_id && self.core.for_table)); + if let Some(row_id_index) = self.core.row_id_index + && self.core.gen_row_id + { + 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()), + )) + } +} + +#[inline] +fn dispatch_new_s3_plan(source: generic::Source, input: Option) -> PlanRef { + if let Some(input) = input { + StreamFsFetch::new(input, source).into() + } else { + StreamSource::new(source).into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index fd05a4c7ecc9a..0aa4940e396cc 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) -> StreamPlanPb { use stream::prelude::*; @@ -687,6 +687,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); } @@ -821,6 +824,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; @@ -850,6 +854,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; @@ -912,6 +917,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; @@ -943,6 +949,7 @@ pub use stream_simple_agg::StreamSimpleAgg; pub use stream_sink::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; @@ -983,6 +990,7 @@ macro_rules! for_all_plan_nodes { , { Logical, CdcScan } , { Logical, SysScan } , { Logical, Source } + , { Logical, SourceBackfill } , { Logical, Insert } , { Logical, Delete } , { Logical, Update } @@ -1036,6 +1044,7 @@ macro_rules! for_all_plan_nodes { , { Stream, CdcTableScan } , { Stream, Sink } , { Stream, Source } + , { Stream, SourceBackfill } , { Stream, HashJoin } , { Stream, Exchange } , { Stream, HashAgg } @@ -1079,6 +1088,7 @@ macro_rules! for_logical_plan_nodes { , { Logical, CdcScan } , { Logical, SysScan } , { Logical, Source } + , { Logical, SourceBackfill } , { Logical, Insert } , { Logical, Delete } , { Logical, Update } @@ -1152,6 +1162,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 da4e8dc228ae1..76cd8baa02739 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) -> PbStreamNode { use risingwave_pb::stream_plan::*; @@ -237,10 +238,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 0000000000000..ce3bea1f03f0b --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/stream_source_backfill.rs @@ -0,0 +1,148 @@ +// 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_pb::stream_plan::stream_node::{NodeBody, PbNodeBody}; +use risingwave_pb::stream_plan::{PbStreamNode, PbStreamSource, SourceNode}; + +use super::stream::prelude::*; +use super::{PlanBase, PlanRef, PlanTreeNodeUnary}; +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::stream_fragmenter::BuildFragmentGraphState; +use crate::Explain; + +#[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) -> Option> { + self.core.catalog.clone() + } + + pub fn adhoc_to_stream_prost(&self, state: &mut BuildFragmentGraphState) -> PbStreamNode { + 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 = source_catalog.map(|source_catalog| PbStreamSource { + source_id: source_catalog.id, + source_name: source_catalog.name.clone(), + // FIXME: this is not the same as source. + state_table: Some( + generic::Source::infer_internal_table_catalog() + .with_id(state.gen_table_id_wrapped()) + .to_internal_table_prost(), + ), + info: Some(source_catalog.info.clone()), + row_id_index: self.core.row_id_index.map(|index| index as _), + columns: self + .core + .column_catalog + .iter() + .map(|c| c.to_protobuf()) + .collect_vec(), + with_properties: source_catalog.with_properties.clone().into_iter().collect(), + rate_limit: self.base.ctx().overwrite_options().streaming_rate_limit, + }); + + let stream_scan_body = PbNodeBody::SourceBackfill(SourceNode { source_inner }); + + let fields = self.schema().to_prost(); + // plan: merge -> backfill + 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 42fdc83a3f933..62414cb2ab005 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -27,7 +27,8 @@ use crate::binder::{ use crate::expr::{Expr, ExprImpl, ExprType, FunctionCall, InputRef}; 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; @@ -85,7 +86,14 @@ impl Planner { } pub(super) fn plan_source(&mut self, source: BoundSource) -> Result { - Ok(LogicalSource::with_catalog(Rc::new(source.catalog), false, self.ctx())?.into()) + if source.can_backfill() { + Ok( + LogicalSourceBackfill::with_catalog(Rc::new(source.catalog), false, self.ctx())? + .into(), + ) + } else { + Ok(LogicalSource::with_catalog(Rc::new(source.catalog), false, self.ctx())?.into()) + } } pub(super) fn plan_join(&mut self, join: BoundJoin) -> Result { diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index 291d72b6a8f22..ded1f86540eda 100644 --- a/src/frontend/src/stream_fragmenter/mod.rs +++ b/src/frontend/src/stream_fragmenter/mod.rs @@ -263,9 +263,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; } } @@ -293,6 +293,7 @@ fn build_fragment( } NodeBody::StreamCdcScan(_) => { + // TODO: 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; @@ -308,6 +309,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_inner.as_ref().unwrap().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 68ce0d2228b53..8d83763cd91cb 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -665,21 +665,10 @@ impl CommandContext { 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 + .tracking_progress_actor_ids() + .into_iter() + .collect(), _ => Default::default(), } } @@ -949,7 +938,7 @@ impl CommandContext { // Extract the fragments that include source operators. let source_fragments = table_fragments.stream_source_fragments(); - + // TODO: handle backfill? self.barrier_manager_context .source_manager .apply_source_change( diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index fc84edd77417f..45a044a0b2b90 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -1028,6 +1028,7 @@ impl GlobalBarrierManager { } commands }; + tracing::trace!("finished_commands: {}", finished_commands.len()); for command in finished_commands { self.checkpoint_control.stash_command_to_finish(command); diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index 0c753a3c3f025..8c2b9d5e32641 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -206,6 +206,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() @@ -368,6 +373,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)); @@ -426,6 +432,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. @@ -450,6 +457,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 ccfd2ea03d08e..277f3ce999931 100644 --- a/src/meta/src/barrier/schedule.rs +++ b/src/meta/src/barrier/schedule.rs @@ -278,21 +278,25 @@ 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 .map_err(|e| anyhow!("failed to inject barrier: {}", e))?; infos.push(info); + tracing::trace!("injected_rx finished"); // Throw the error if it occurs when collecting this barrier. collect_rx .await .map_err(|e| anyhow!("failed to collect barrier: {}", e))??; + tracing::trace!("collect_rx finished"); // Wait for this command to be finished. finish_rx .await .map_err(|e| anyhow!("failed to finish command: {}", e))?; + tracing::trace!("finish_rx finished"); } Ok(infos) diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index 70fc1c52a0fce..d472161b30021 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -1422,7 +1422,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() @@ -1432,7 +1432,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 bec61ff0f2166..bbd1b17e83a44 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -1122,23 +1122,6 @@ impl CatalogController { Ok(chain_fragments) } - /// Find the external stream source info inside the stream node, if any. - fn find_stream_source(stream_node: &PbStreamNode) -> Option<&StreamSource> { - if let Some(NodeBody::Source(source)) = &stream_node.node_body { - 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 - } - pub async fn load_source_fragment_ids( &self, ) -> MetaResult>> { @@ -1157,9 +1140,9 @@ impl CatalogController { let mut source_fragment_ids = HashMap::new(); for (fragment_id, _, stream_node) in fragments { - if let Some(source) = Self::find_stream_source(stream_node.inner_ref()) { + if let Some(source_id) = stream_node.inner_ref().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); } @@ -1167,31 +1150,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) = Self::find_stream_source(stream_node.inner_ref()) { + for (fragment_id, upstream_fragment_id, _, stream_node) in fragments { + if let Some(source_id) = stream_node.inner_ref().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 bcd86e28c5287..d5898347bd623 100644 --- a/src/meta/src/controller/streaming_job.rs +++ b/src/meta/src/controller/streaming_job.rs @@ -426,7 +426,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 { @@ -457,6 +457,7 @@ impl CatalogController { .all(&txn) .await?; + // 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 a9c7459d745d9..7cd623badbd26 100644 --- a/src/meta/src/manager/catalog/database.rs +++ b/src/meta/src/manager/catalog/database.rs @@ -361,11 +361,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 076afe09a7363..72551f590f674 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -1013,7 +1013,7 @@ impl FragmentManager { pub async fn get_running_actors_and_upstream_fragment_of_fragment( &self, fragment_id: FragmentId, - ) -> MetaResult<(Vec, Vec)> { + ) -> MetaResult)>> { let map = &self.core.read().await.table_fragments; for table_fragment in map.values() { @@ -1024,9 +1024,9 @@ impl FragmentManager { .filter(|a| { table_fragment.actor_status[&a.actor_id].state == ActorState::Running as i32 }) - .cloned() + .map(|a| (a.actor_id, a.upstream_actor_id.clone())) .collect(); - return Ok((running_actors, fragment.upstream_fragment_ids.clone())); + return Ok(running_actors); } } diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index cace146d7fb62..b63704e3bdd72 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -1145,7 +1145,7 @@ impl CatalogManager { let mut all_sink_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 @@ -1408,11 +1408,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?; @@ -1669,7 +1668,7 @@ impl CatalogManager { .into_iter() .map(|id| id.into()) .chain(all_sink_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 df2451b553319..39c2962444984 100644 --- a/src/meta/src/manager/metadata.rs +++ b/src/meta/src/manager/metadata.rs @@ -318,10 +318,10 @@ impl MetadataManager { } } - pub async fn get_running_actors_and_upstream_fragment_of_fragment( + pub async fn get_running_actors_and_upstream_actors_of_fragment( &self, id: FragmentId, - ) -> MetaResult<(Vec, Vec)> { + ) -> MetaResult)>> { match self { MetadataManager::V1(mgr) => { mgr.fragment_manager diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index 139581d46f43f..3d9f829214278 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -327,7 +327,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())) @@ -354,10 +354,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 }) } @@ -398,23 +400,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> { @@ -422,10 +407,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()) @@ -438,6 +420,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 1ab3aad2ddf3d..5ff8d6a883501 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -485,6 +485,7 @@ impl DdlController { unimplemented!("support drop source in v2"); }; // 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( @@ -1282,8 +1283,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, @@ -1360,7 +1365,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 fc0a7ef55b8f1..c2246d5070b93 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -606,7 +606,7 @@ impl ScaleController { if (fragment.get_fragment_type_mask() & FragmentTypeFlag::Source as u32) != 0 { let stream_node = fragment.actors.first().unwrap().get_nodes().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); } } diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index 95f5c219351a3..5743dddd84b30 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -16,6 +16,7 @@ use std::borrow::BorrowMut; use std::cmp::Ordering; use std::collections::hash_map::Entry; use std::collections::{BTreeMap, BTreeSet, BinaryHeap, HashMap, HashSet}; +use std::hash::Hash; use std::ops::Deref; use std::sync::Arc; use std::time::Duration; @@ -30,6 +31,7 @@ use risingwave_connector::source::{ }; use risingwave_pb::catalog::Source; use risingwave_pb::source::{ConnectorSplit, ConnectorSplits}; +use risingwave_pb::stream_plan::{Dispatcher, StreamActor}; use risingwave_rpc_client::ConnectorClient; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; use tokio::sync::{oneshot, Mutex}; @@ -228,8 +230,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>, @@ -240,20 +242,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, } } @@ -267,12 +263,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 { continue; @@ -281,21 +278,18 @@ impl SourceManagerCore { tracing::warn!("No splits discovered for source {}", source_id); } - let mut source_fragments = vec![]; - let mut backfill_fragments = vec![]; - - for fragment_id in fragment_ids { - let (actors, upstream_fragment_ids) = match self + for &fragment_id in source_fragment_ids { + let actors = match self .metadata_manager - .get_running_actors_and_upstream_fragment_of_fragment(*fragment_id) + .get_running_actors_of_fragment(fragment_id) .await { - Ok((actors, upstream_fragment_ids)) => { + Ok(actors) => { if actors.is_empty() { tracing::warn!("No actors found for fragment {}", fragment_id); continue; } - (actors, upstream_fragment_ids) + actors } Err(err) => { tracing::warn!("Failed to get the actor of the fragment {}, maybe the fragment doesn't exist anymore", err.to_string()); @@ -303,35 +297,9 @@ impl SourceManagerCore { } }; - if !upstream_fragment_ids.is_empty() { - debug_assert!( - upstream_fragment_ids.len() == 1, - "source backfill fragment should have exactly one upstream fragment, fragment_id: {fragment_id}, upstream_fragment_ids: {upstream_fragment_ids:?}" - ); - for actor in &actors { - debug_assert!( - actor.upstream_actor_id.len() == 1, - "source backfill actor should have exactly one upstream actor, fragment_id: {fragment_id}, actor: {actor:?}" - ); - } - backfill_fragments.push((*fragment_id, upstream_fragment_ids[0], actors)); - } else { - for actor in &actors { - debug_assert!( - actor.upstream_actor_id.is_empty(), - "source actor should not have upstream actors, fragment_id: {fragment_id}, actor: {actor:?}" - ); - } - source_fragments.push((*fragment_id, actors)); - } - } - - // assign splits for source fragments first - for (fragment_id, actors) in source_fragments { let prev_actor_splits: HashMap<_, _> = actors .into_iter() - .map(|actor| { - let actor_id = actor.actor_id; + .map(|actor_id| { ( actor_id, self.actor_splits @@ -354,28 +322,40 @@ impl SourceManagerCore { } } - // align splits for backfill fragments with its upstream source fragment - for (fragment_id, upstream_fragment_id, actors) in backfill_fragments { - let upstream_assignment = split_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 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; + } + }; + let upstream_assignment = split_assignment .get(&upstream_fragment_id) .unwrap_or_else(||panic!( "source backfill fragment's upstream fragment should have assignment, fragment_id: {fragment_id}, upstream_fragment_id: {upstream_fragment_id}, split_assignment: {split_assignment:?}")); - split_assignment.insert( - fragment_id, - actors - .into_iter() - .map(|a| { - let actor_id = a.actor_id; - ( - actor_id, - upstream_assignment - .get(&actor_id) - .cloned() - .unwrap_or_else(||panic!("source backfill actor should have upstream actor, fragment_id: {fragment_id}, upstream_fragment_id: {upstream_fragment_id}, actor: {a:?}, upstream_assignment: {upstream_assignment:?}")), - ) - }) - .collect(), - ); + split_assignment.insert( + *fragment_id, + align_backfill_splits( + actors, + upstream_assignment, + *fragment_id, + *upstream_fragment_id, + )?, + ); + } } } @@ -390,10 +370,6 @@ impl SourceManagerCore { ) { if let Some(source_fragments) = 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() @@ -432,10 +408,6 @@ impl SourceManagerCore { entry.remove(); } } - - for fragment_id in &fragment_ids { - self.fragment_sources.remove(fragment_id); - } } for actor_id in actor_splits { @@ -491,6 +463,7 @@ impl Default for SplitDiffOptions { /// /// 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 fn reassign_splits( @@ -586,6 +559,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); @@ -611,6 +610,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) => { @@ -622,6 +622,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()); } } @@ -638,6 +639,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() @@ -662,6 +678,7 @@ impl SourceManager { metadata_manager, managed_sources, source_fragments, + backfill_fragments, actor_splits, )); @@ -806,6 +823,68 @@ 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()?; + tracing::debug!( + ?source_backfill_fragments, + ?table_fragments, + "allocate_splits_for_backfill 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) -> anyhow::Result<()> { 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 c42c2f5a51425..d667212d9c9ec 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_inner.as_ref().unwrap().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 937dadda21fb8..51f817c768302 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -24,7 +24,7 @@ use risingwave_common::bail; use risingwave_common::catalog::{ generate_internal_table_name_with_type, TableId, CDC_SOURCE_COLUMN_NUM, }; -use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_common::util::iter_util::{IntoIteratorExt, ZipEqFast}; use risingwave_common::util::stream_graph_visitor; use risingwave_pb::catalog::Table; use risingwave_pb::ddl_service::TableJobType; @@ -54,7 +54,8 @@ pub(super) struct BuildingFragment { /// The ID of the job if it's materialized in this fragment. 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 source on table, its `vec![]`, since the output is fixed. upstream_table_columns: HashMap>, @@ -177,6 +178,17 @@ impl BuildingFragment { stream_scan.upstream_column_ids.clone(), ), NodeBody::CdcFilter(cdc_filter) => (cdc_filter.upstream_source_id.into(), vec![]), + NodeBody::SourceBackfill(backfill) => ( + backfill.get_source_inner().unwrap().source_id.into(), + // FIXME: only pass required columns instead of all columns here + backfill + .get_source_inner() + .unwrap() + .columns + .iter() + .map(|c| c.column_desc.as_ref().unwrap().column_id) + .collect(), + ), _ => return, }; table_columns @@ -187,7 +199,7 @@ impl BuildingFragment { assert_eq!( table_columns.len(), fragment.upstream_table_ids.len(), - "fragment type: {}", + "fragment type: {:b}", fragment.fragment_type_mask ); @@ -286,7 +298,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, @@ -513,7 +525,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), } @@ -525,6 +537,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, @@ -655,50 +668,96 @@ impl CompleteStreamFragmentGraph { (source_job_id, edge) } DdlType::MaterializedView | DdlType::Sink | DdlType::Index => { - // 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 + .map_collect::<_, _, _, Option>>(|c| { + all_column_ids + .iter() + .position(|&id| id == *c) + .map(|i| i as u32) + }) + .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 + .map_collect::<_, _, _, Option>>(|c| { + all_column_ids + .iter() + .position(|&id| id == *c) + .map(|i| i as u32) + }) + .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 ed2dac5be0e06..1ae24ec1b7d51 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 f10738d3f5cab..da9bc42af3449 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -354,6 +354,7 @@ impl GlobalStreamManager { } } CreatingState::Created => { + tracing::debug!(id=?table_id, "streaming job created"); self.creating_job_info.delete_job(table_id).await; return Ok(()); } @@ -435,6 +436,7 @@ impl GlobalStreamManager { actor_id: actors.clone(), }) .await?; + tracing::debug!("build actors finished"); Ok(()) as MetaResult<()> }) @@ -507,7 +509,7 @@ impl GlobalStreamManager { .await?; let dummy_table_id = table_fragments.table_id(); - + // TODO: need change? let init_split_assignment = self.source_manager.allocate_splits(&dummy_table_id).await?; @@ -524,7 +526,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, @@ -535,7 +546,7 @@ impl GlobalStreamManager { ddl_type, replace_table: replace_table_command, }; - + tracing::trace!(?command, "sending first barrier for creating streaming job"); if let Err(err) = self.barrier_scheduler.run_command(command).await { if create_type == CreateType::Foreground { let mut table_ids = HashSet::from_iter(std::iter::once(table_id)); @@ -569,7 +580,7 @@ impl GlobalStreamManager { .await?; let dummy_table_id = table_fragments.table_id(); - + // TODO: need change? 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 dafa3a568780f..b2548f7f3f401 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -190,6 +190,63 @@ 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() + { + if let Some(inner) = &source.source_inner { + return Some(inner.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/exchange/output.rs b/src/stream/src/executor/exchange/output.rs index 7b85633bdd18c..d94e4cfbac841 100644 --- a/src/stream/src/executor/exchange/output.rs +++ b/src/stream/src/executor/exchange/output.rs @@ -67,6 +67,13 @@ impl LocalOutput { } } +impl Drop for LocalOutput { + fn drop(&mut self) { + let backtrace = std::backtrace::Backtrace::capture(); + tracing::debug!(actor_id=?self.actor_id, "dropping LocalOutput, backtrace: {}", backtrace); + } +} + #[async_trait] impl Output for LocalOutput { async fn send(&mut self, message: Message) -> StreamResult<()> { @@ -76,7 +83,7 @@ impl Output for LocalOutput { .await .map_err(|SendError(message)| { anyhow!( - "failed to send message to actor {}: {:?}", + "failed to send message to actor {}, message: {:?}", self.actor_id, message ) @@ -130,7 +137,7 @@ impl Output for RemoteOutput { .await .map_err(|SendError(message)| { anyhow!( - "failed to send message to actor {}: {:#?}", + "failed to send message to actor {}, message: {:?}", self.actor_id, message ) diff --git a/src/stream/src/executor/source/kafka_backfill_executor.rs b/src/stream/src/executor/source/kafka_backfill_executor.rs index d1aef5a6bbf2e..a6924a6ea5d87 100644 --- a/src/stream/src/executor/source/kafka_backfill_executor.rs +++ b/src/stream/src/executor/source/kafka_backfill_executor.rs @@ -25,13 +25,14 @@ use futures::StreamExt; use futures_async_stream::try_stream; use risingwave_common::buffer::BitmapBuilder; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; -use risingwave_common::row::Row; +use risingwave_common::row::{Row, RowExt}; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_connector::source::{ BoxSourceWithStateStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitId, SplitMetaData, StreamChunkWithState, }; use risingwave_connector::ConnectorParams; +use risingwave_pb::plan_common::AdditionalColumnType; use risingwave_source::source_desc::{SourceDesc, SourceDescBuilder}; use risingwave_storage::StateStore; use thiserror_ext::AsReport; @@ -50,9 +51,9 @@ type ExecutorSplitId = String; const WAIT_BARRIER_MULTIPLE_TIMES: u128 = 5; pub struct KafkaBackfillExecutorWrapper { - inner: KafkaBackfillExecutor, + pub inner: KafkaBackfillExecutor, /// Upstream changelog stream which may contain metadata columns, e.g. `_rw_offset` - input: Box, + pub input: Box, } pub struct KafkaBackfillExecutor { @@ -369,13 +370,12 @@ impl KafkaBackfillExecutor { #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute(mut self, input: BoxedExecutor) { // TODO: these can be inferred in frontend by checking additional_column_type - let split_column_idx = 1; - let offset_column_idx = 2; let mut input = input.execute(); // Poll the upstream to get the first barrier. let barrier = expect_first_barrier(&mut input).await?; + tracing::debug!("KafkaBackfillExecutor got first barrier: {barrier:?}"); let mut core = self.stream_source_core.unwrap(); @@ -384,6 +384,17 @@ impl KafkaBackfillExecutor { let source_desc = source_desc_builder .build() .map_err(StreamExecutorError::connector_error)?; + let split_column_idx = source_desc + .columns + .iter() + .position(|c| matches!(c.additional_column_type, AdditionalColumnType::Partition)) + .expect("kafka source should have partition column"); + let offset_column_idx = source_desc + .columns + .iter() + .position(|c| matches!(c.additional_column_type, AdditionalColumnType::Offset)) + .expect("kafka source should have offset column"); + tracing::debug!(?split_column_idx, ?offset_column_idx); let mut boot_state = Vec::default(); if let Some(mutation) = barrier.mutation.as_ref() { @@ -405,7 +416,8 @@ impl KafkaBackfillExecutor { _ => {} } } - let mut latest_split_info = boot_state.clone(); + let latest_split_info = boot_state.clone(); + tracing::debug!("latest_split_info: {:?}", latest_split_info); core.split_state_store.init_epoch(barrier.epoch); @@ -415,6 +427,7 @@ impl KafkaBackfillExecutor { .try_recover_from_state_store(ele) .await? { + tracing::debug!("recover_state: {:?}", recover_state); *ele = recover_state; } } @@ -473,10 +486,13 @@ impl KafkaBackfillExecutor { let need_backfill = backfill_state .values() .any(|state| !matches!(state, BackfillState::Finished)); + tracing::debug!("KafkaBackfillExecutor backfill_state: {backfill_state:?}"); if need_backfill { #[for_await] 'backfill_loop: for either in &mut backfill_stream { + tracing::debug!("KafkaBackfillExecutor got msg either: {either:?}"); + match either { // Upstream Either::Left(msg) => { @@ -490,12 +506,11 @@ impl KafkaBackfillExecutor { // Note: We assume offset from the source is monotonically increasing for the algorithm to work correctly. let mut new_vis = BitmapBuilder::zeroed(chunk.visibility().len()); for (i, (_, row)) in chunk.rows().enumerate() { - let split = - row.datum_at(split_column_idx).unwrap().into_int64(); + tracing::debug!(row = %row.display()); + let split = row.datum_at(split_column_idx).unwrap().into_utf8(); let offset = - row.datum_at(offset_column_idx).unwrap().into_int64(); - let backfill_state = - backfill_state.get_mut(&split.to_string()).unwrap(); + row.datum_at(offset_column_idx).unwrap().into_utf8(); + let backfill_state = backfill_state.get_mut(split).unwrap(); match backfill_state { BackfillState::Backfilling(backfill_offset) => { new_vis.set(i, false); @@ -629,7 +644,11 @@ impl KafkaBackfillExecutor { // All splits finished backfilling. Now we only forward the source data. #[for_await] for msg in input { - match msg? { + let msg = msg?; + + tracing::debug!("KafkaBackfillExecutor got msg: {msg:?}"); + + match msg { Message::Barrier(barrier) => { // TODO: How to handle a split change here? // We might need to persist its state. Is is possible that we need to backfill? @@ -646,10 +665,11 @@ impl KafkaBackfillExecutor { } } -fn compare_kafka_offset(a: Option<&String>, b: i64) -> Ordering { +fn compare_kafka_offset(a: Option<&String>, b: &str) -> Ordering { match a { Some(a) => { let a = a.parse::().unwrap(); + let b = b.parse::().unwrap(); a.cmp(&b) } None => Ordering::Less, diff --git a/src/stream/src/from_proto/mod.rs b/src/stream/src/from_proto/mod.rs index 9a9e83c0a328f..5f3f710341336 100644 --- a/src/stream/src/from_proto/mod.rs +++ b/src/stream/src/from_proto/mod.rs @@ -42,6 +42,7 @@ mod simple_agg; mod sink; mod sort; mod source; +mod source_backfill; mod stateless_simple_agg; mod stream_cdc_scan; mod stream_scan; @@ -84,6 +85,7 @@ use self::simple_agg::*; use self::sink::*; use self::sort::*; use self::source::*; +use self::source_backfill::*; use self::stateless_simple_agg::*; use self::stream_cdc_scan::*; use self::stream_scan::*; @@ -172,5 +174,6 @@ pub async fn create_executor( NodeBody::EowcOverWindow => EowcOverWindowExecutorBuilder, NodeBody::OverWindow => OverWindowExecutorBuilder, NodeBody::StreamFsFetch => FsFetchExecutorBuilder, + NodeBody::SourceBackfill => KafkaBackfillExecutorBuilder, } } diff --git a/src/stream/src/from_proto/source_backfill.rs b/src/stream/src/from_proto/source_backfill.rs new file mode 100644 index 0000000000000..757d19688df30 --- /dev/null +++ b/src/stream/src/from_proto/source_backfill.rs @@ -0,0 +1,165 @@ +// 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 risingwave_common::catalog::{ + default_key_column_name_version_mapping, ColumnId, TableId, KAFKA_TIMESTAMP_COLUMN_NAME, +}; +use risingwave_connector::source::{ConnectorProperties, SourceCtrlOpts}; +use risingwave_pb::data::data_type::TypeName as PbTypeName; +use risingwave_pb::plan_common::{ + AdditionalColumnType, ColumnDescVersion, FormatType, PbEncodeType, +}; +use risingwave_pb::stream_plan::SourceNode; +use risingwave_source::source_desc::SourceDescBuilder; +use risingwave_storage::panic_store::PanicStateStore; +use tokio::sync::mpsc::unbounded_channel; + +use super::*; +use crate::executor::kafka_backfill_executor::{ + KafkaBackfillExecutor, KafkaBackfillExecutorWrapper, +}; +use crate::executor::source::{FsListExecutor, StreamSourceCore}; +use crate::executor::state_table_handler::SourceStateTableHandler; + +pub struct KafkaBackfillExecutorBuilder; + +impl ExecutorBuilder for KafkaBackfillExecutorBuilder { + type Node = SourceNode; + + async fn new_boxed_executor( + params: ExecutorParams, + node: &Self::Node, + store: impl StateStore, + stream: &mut LocalStreamManagerCore, + ) -> StreamResult { + let [input]: [_; 1] = params.input.try_into().unwrap(); + + // let (sender, barrier_receiver) = unbounded_channel(); + // stream + // .context + // .barrier_manager() + // .register_sender(params.actor_context.id, sender); + let system_params = params.env.system_params_manager_ref().get_params(); + + let source = node.source_inner.as_ref().unwrap(); + let source_id = TableId::new(source.source_id); + let source_name = source.source_name.clone(); + let source_info = source.get_info()?; + + let mut source_columns = source.columns.clone(); + + { + // compatible code: introduced in https://github.com/risingwavelabs/risingwave/pull/13707 + // for upsert and (avro | protobuf) overwrite the `_rw_key` column's ColumnDesc.additional_column_type to Key + if source_info.format() == FormatType::Upsert + && (source_info.row_encode() == PbEncodeType::Avro + || source_info.row_encode() == PbEncodeType::Protobuf) + { + let _ = source_columns.iter_mut().map(|c| { + let _ = c.column_desc.as_mut().map(|desc| { + let is_bytea = desc + .get_column_type() + .map(|col_type| col_type.type_name == PbTypeName::Bytea as i32) + .unwrap(); + if desc.name == default_key_column_name_version_mapping( + &desc.version() + ) + && is_bytea + // the column is from a legacy version + && desc.version == ColumnDescVersion::Unspecified as i32 + { + desc.additional_column_type = AdditionalColumnType::Key as i32; + } + }); + }); + } + } + + { + // compatible code: handle legacy column `_rw_kafka_timestamp` + // the column is auto added for all kafka source to empower batch query on source + // solution: rewrite the column `additional_column_type` to Timestamp + + let _ = source_columns.iter_mut().map(|c| { + let _ = c.column_desc.as_mut().map(|desc| { + let is_timestamp = desc + .get_column_type() + .map(|col_type| col_type.type_name == PbTypeName::Timestamptz as i32) + .unwrap(); + if desc.name == KAFKA_TIMESTAMP_COLUMN_NAME + && is_timestamp + // the column is from a legacy version + && desc.version == ColumnDescVersion::Unspecified as i32 + { + desc.additional_column_type = AdditionalColumnType::Timestamp as i32; + } + }); + }); + } + + let source_desc_builder = SourceDescBuilder::new( + source_columns.clone(), + params.env.source_metrics(), + source.row_id_index.map(|x| x as _), + source.with_properties.clone(), + source_info.clone(), + params.env.connector_params(), + params.env.config().developer.connector_message_buffer_size, + // `pk_indices` is used to ensure that a message will be skipped instead of parsed + // with null pk when the pk column is missing. + // + // Currently pk_indices for source is always empty since pk information is not + // passed via `StreamSource` so null pk may be emitted to downstream. + // + // TODO: use the correct information to fill in pk_dicies. + // We should consdier add back the "pk_column_ids" field removed by #8841 in + // StreamSource + params.info.pk_indices.clone(), + ); + + let source_ctrl_opts = SourceCtrlOpts { + chunk_size: params.env.config().developer.chunk_size, + }; + + let source_column_ids: Vec<_> = source_columns + .iter() + .map(|column| ColumnId::from(column.get_column_desc().unwrap().column_id)) + .collect(); + + let state_table_handler = SourceStateTableHandler::from_table_catalog( + source.state_table.as_ref().unwrap(), + store.clone(), + ) + .await; + let stream_source_core = StreamSourceCore::new( + source_id, + source_name, + source_column_ids, + source_desc_builder, + state_table_handler, + ); + + let exec = KafkaBackfillExecutor::new( + params.actor_context.clone(), + params.info.clone(), + Some(stream_source_core), + params.executor_stats.clone(), + // barrier_receiver, + system_params, + source_ctrl_opts.clone(), + params.env.connector_params(), + ); + Ok(KafkaBackfillExecutorWrapper { inner: exec, input }.boxed()) + } +}