From 71a86b4623bf56b32a7d65dd1a3f64fb3b9d9d48 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Thu, 18 Sep 2025 16:06:53 +0800 Subject: [PATCH 01/36] planner --- proto/stream_plan.proto | 10 + .../plan_node/generic/locality_provider.rs | 85 +++++ .../src/optimizer/plan_node/generic/mod.rs | 3 + .../src/optimizer/plan_node/logical_join.rs | 15 +- .../plan_node/logical_locality_provider.rs | 167 +++++++++ src/frontend/src/optimizer/plan_node/mod.rs | 6 + .../plan_node/stream_locality_provider.rs | 173 +++++++++ src/stream/src/executor/locality_provider.rs | 335 ++++++++++++++++++ src/stream/src/executor/mod.rs | 1 + .../src/from_proto/locality_provider.rs | 82 +++++ src/stream/src/from_proto/mod.rs | 3 + 11 files changed, 876 insertions(+), 4 deletions(-) create mode 100644 src/frontend/src/optimizer/plan_node/generic/locality_provider.rs create mode 100644 src/frontend/src/optimizer/plan_node/logical_locality_provider.rs create mode 100644 src/frontend/src/optimizer/plan_node/stream_locality_provider.rs create mode 100644 src/stream/src/executor/locality_provider.rs create mode 100644 src/stream/src/from_proto/locality_provider.rs diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index dfd881a9aab5d..f6c462382715b 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -997,6 +997,15 @@ message UpstreamSinkUnionNode { repeated UpstreamSinkInfo init_upstreams = 1; } +message LocalityProviderNode { + // Column indices that define locality + repeated uint32 locality_columns = 1; + // State table for buffering input data + optional catalog.Table state_table = 2; + // Progress table for tracking backfill progress + optional catalog.Table progress_table = 3; +} + message StreamNode { // This field used to be a `bool append_only`. // Enum variants are ordered for backwards compatibility. @@ -1059,6 +1068,7 @@ message StreamNode { MaterializedExprsNode materialized_exprs = 149; VectorIndexWriteNode vector_index_write = 150; UpstreamSinkUnionNode upstream_sink_union = 151; + LocalityProviderNode locality_provider = 152; } // The id for the operator. This is local per mview. // TODO: should better be a uint32. diff --git a/src/frontend/src/optimizer/plan_node/generic/locality_provider.rs b/src/frontend/src/optimizer/plan_node/generic/locality_provider.rs new file mode 100644 index 0000000000000..017c52a7318cc --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/generic/locality_provider.rs @@ -0,0 +1,85 @@ +// Copyright 2025 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt; + +use risingwave_common::catalog::Schema; + +use super::{GenericPlanNode, GenericPlanRef, impl_distill_unit_from_fields}; +use crate::expr::ExprRewriter; +use crate::optimizer::optimizer_context::OptimizerContextRef; +use crate::optimizer::property::FunctionalDependencySet; + +/// `LocalityProvider` provides locality for operators during backfilling. +/// It buffers input data into a state table using locality columns as primary key prefix. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct LocalityProvider { + pub input: PlanRef, + /// Columns that define the locality + pub locality_columns: Vec, +} + +impl fmt::Display for LocalityProvider { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!( + f, + "LocalityProvider {{ locality_columns: {:?} }}", + self.locality_columns + ) + } +} + +impl LocalityProvider { + pub fn new(input: PlanRef, locality_columns: Vec) -> Self { + Self { + input, + locality_columns, + } + } + + pub fn fields_pretty<'a>(&self) -> Vec<(&'a str, pretty_xmlish::Pretty<'a>)> { + let locality_columns_str = format!("{:?}", self.locality_columns); + vec![("locality_columns", locality_columns_str.into())] + } +} + +impl GenericPlanNode for LocalityProvider { + fn schema(&self) -> Schema { + self.input.schema().clone() + } + + fn stream_key(&self) -> Option> { + Some(self.input.stream_key()?.to_vec()) + } + + fn ctx(&self) -> OptimizerContextRef { + self.input.ctx() + } + + fn functional_dependency(&self) -> FunctionalDependencySet { + self.input.functional_dependency().clone() + } +} + +impl LocalityProvider { + pub fn rewrite_exprs(&mut self, _r: &mut dyn ExprRewriter) { + // LocalityProvider doesn't contain expressions to rewrite + } + + pub fn visit_exprs(&self, _v: &mut dyn crate::expr::ExprVisitor) { + // LocalityProvider doesn't contain expressions to visit + } +} + +impl_distill_unit_from_fields!(LocalityProvider, GenericPlanRef); diff --git a/src/frontend/src/optimizer/plan_node/generic/mod.rs b/src/frontend/src/optimizer/plan_node/generic/mod.rs index 6394e8e9348b6..dbea1ae29b2ac 100644 --- a/src/frontend/src/optimizer/plan_node/generic/mod.rs +++ b/src/frontend/src/optimizer/plan_node/generic/mod.rs @@ -95,6 +95,9 @@ pub use postgres_query::*; mod mysql_query; pub use mysql_query::*; +mod locality_provider; +pub use locality_provider::*; + pub trait DistillUnit { fn distill_with_name<'a>(&self, name: impl Into>) -> XmlNode<'a>; } diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index 48884018bb2f7..ea755fef94a70 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -28,9 +28,9 @@ use super::generic::{ }; use super::utils::{Distill, childless_record}; use super::{ - BatchPlanRef, ColPrunable, ExprRewritable, Logical, LogicalPlanRef as PlanRef, PlanBase, - PlanTreeNodeBinary, PredicatePushdown, StreamHashJoin, StreamPlanRef, StreamProject, ToBatch, - ToStream, generic, + BatchPlanRef, ColPrunable, ExprRewritable, Logical, LogicalLocalityProvider, + LogicalPlanRef as PlanRef, PlanBase, PlanTreeNodeBinary, PredicatePushdown, StreamHashJoin, + StreamPlanRef, StreamProject, ToBatch, ToStream, generic, }; use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{CollectInputRef, Expr, ExprImpl, ExprRewriter, ExprType, ExprVisitor, InputRef}; @@ -1770,7 +1770,14 @@ impl ToStream for LogicalJoin { ); } } - None + + Some( + LogicalLocalityProvider::new( + self.clone_with_left_right(self.left(), self.right()).into(), + columns.to_owned(), + ) + .into(), + ) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs b/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs new file mode 100644 index 0000000000000..f9db3b04c41ed --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs @@ -0,0 +1,167 @@ +// Copyright 2025 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use itertools::Itertools; + +use super::generic::GenericPlanRef; +use super::utils::impl_distill_by_unit; +use super::{ + BatchPlanRef, ColPrunable, ExprRewritable, Logical, LogicalPlanRef as PlanRef, LogicalProject, + PlanBase, PlanTreeNodeUnary, PredicatePushdown, StreamPlanRef, ToBatch, ToStream, generic, +}; +use crate::error::Result; +use crate::expr::{ExprRewriter, ExprVisitor}; +use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::plan_node::{ + ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, +}; +use crate::utils::{ColIndexMapping, Condition}; + +/// `LogicalLocalityProvider` provides locality for operators during backfilling. +/// It buffers input data into a state table using locality columns as primary key prefix. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct LogicalLocalityProvider { + pub base: PlanBase, + core: generic::LocalityProvider, +} + +impl LogicalLocalityProvider { + pub fn new(input: PlanRef, locality_columns: Vec) -> Self { + let core = generic::LocalityProvider::new(input, locality_columns); + let base = PlanBase::new_logical_with_core(&core); + LogicalLocalityProvider { base, core } + } + + /// Create a `LogicalLocalityProvider` with the given input and locality columns + pub fn create(input: PlanRef, locality_columns: Vec) -> PlanRef { + LogicalLocalityProvider::new(input, locality_columns).into() + } + + /// Get the locality columns of the locality provider. + pub fn locality_columns(&self) -> &[usize] { + &self.core.locality_columns + } +} + +impl PlanTreeNodeUnary for LogicalLocalityProvider { + fn input(&self) -> PlanRef { + self.core.input.clone() + } + + fn clone_with_input(&self, input: PlanRef) -> Self { + Self::new(input, self.locality_columns().to_vec()) + } + + fn rewrite_with_input( + &self, + input: PlanRef, + input_col_change: ColIndexMapping, + ) -> (Self, ColIndexMapping) { + let locality_columns = self + .locality_columns() + .iter() + .map(|&i| input_col_change.map(i)) + .collect(); + + (Self::new(input, locality_columns), input_col_change) + } +} + +impl_plan_tree_node_for_unary! { Logical, LogicalLocalityProvider} +impl_distill_by_unit!(LogicalLocalityProvider, core, "LogicalLocalityProvider"); + +impl ColPrunable for LogicalLocalityProvider { + fn prune_col(&self, required_cols: &[usize], ctx: &mut ColumnPruningContext) -> PlanRef { + // No pruning. + let input_required_cols = (0..self.input().schema().len()).collect_vec(); + LogicalProject::with_out_col_idx( + self.clone_with_input(self.input().prune_col(&input_required_cols, ctx)) + .into(), + required_cols.iter().cloned(), + ) + .into() + } +} + +impl PredicatePushdown for LogicalLocalityProvider { + fn predicate_pushdown( + &self, + predicate: Condition, + ctx: &mut PredicatePushdownContext, + ) -> PlanRef { + let new_input = self.input().predicate_pushdown(predicate, ctx); + let new_provider = self.clone_with_input(new_input); + new_provider.into() + } +} + +impl ToBatch for LogicalLocalityProvider { + fn to_batch(&self) -> Result { + // LocalityProvider is a streaming-only operator + Err(crate::error::ErrorCode::NotSupported( + "LocalityProvider in batch mode".to_string(), + "LocalityProvider is only supported in streaming mode for backfilling".to_string(), + ) + .into()) + } +} + +impl ToStream for LogicalLocalityProvider { + fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { + use super::StreamLocalityProvider; + let input = self.input().to_stream(ctx)?; + let stream_core = generic::LocalityProvider::new(input, self.locality_columns().to_vec()); + Ok(StreamLocalityProvider::new(stream_core).into()) + } + + fn logical_rewrite_for_stream( + &self, + ctx: &mut RewriteStreamContext, + ) -> Result<(PlanRef, ColIndexMapping)> { + let (input, input_col_change) = self.input().logical_rewrite_for_stream(ctx)?; + let (locality_provider, out_col_change) = self.rewrite_with_input(input, input_col_change); + Ok((locality_provider.into(), out_col_change)) + } +} + +impl ExprRewritable for LogicalLocalityProvider { + fn has_rewritable_expr(&self) -> bool { + false + } + + fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { + self.clone().into() + } +} + +impl ExprVisitable for LogicalLocalityProvider { + fn visit_exprs(&self, _v: &mut dyn ExprVisitor) { + // No expressions to visit + } +} + +impl LogicalLocalityProvider { + /// Try to provide better locality by transforming input + pub fn try_better_locality(&self, columns: &[usize]) -> Option { + if columns == self.locality_columns() { + Some(self.clone().into()) + } else { + if let Some(better_input) = self.input().try_better_locality(columns) { + Some(better_input) + } else { + Some(Self::new(self.input(), columns.to_owned()).into()) + } + } + } +} diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index c123ca06f91f0..92b6fdf7807f7 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -1011,6 +1011,7 @@ mod logical_intersect; mod logical_join; mod logical_kafka_scan; mod logical_limit; +mod logical_locality_provider; mod logical_max_one_row; mod logical_multi_join; mod logical_now; @@ -1045,6 +1046,7 @@ mod stream_hash_join; mod stream_hop_window; mod stream_join_common; mod stream_local_approx_percentile; +mod stream_locality_provider; mod stream_materialize; mod stream_materialized_exprs; mod stream_now; @@ -1139,6 +1141,7 @@ pub use logical_intersect::LogicalIntersect; pub use logical_join::LogicalJoin; pub use logical_kafka_scan::LogicalKafkaScan; pub use logical_limit::LogicalLimit; +pub use logical_locality_provider::LogicalLocalityProvider; pub use logical_max_one_row::LogicalMaxOneRow; pub use logical_multi_join::{LogicalMultiJoin, LogicalMultiJoinBuilder}; pub use logical_mysql_query::LogicalMySqlQuery; @@ -1177,6 +1180,7 @@ pub use stream_hash_join::StreamHashJoin; pub use stream_hop_window::StreamHopWindow; use stream_join_common::StreamJoinCommon; pub use stream_local_approx_percentile::StreamLocalApproxPercentile; +pub use stream_locality_provider::StreamLocalityProvider; pub use stream_materialize::StreamMaterialize; pub use stream_materialized_exprs::StreamMaterializedExprs; pub use stream_now::StreamNow; @@ -1264,6 +1268,7 @@ macro_rules! for_all_plan_nodes { , { Logical, PostgresQuery } , { Logical, MySqlQuery } , { Logical, VectorSearch } + , { Logical, LocalityProvider } , { Batch, SimpleAgg } , { Batch, HashAgg } , { Batch, SortAgg } @@ -1340,6 +1345,7 @@ macro_rules! for_all_plan_nodes { , { Stream, MaterializedExprs } , { Stream, VectorIndexWrite } , { Stream, UpstreamSinkUnion } + , { Stream, LocalityProvider } $(,$rest)* } }; diff --git a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs new file mode 100644 index 0000000000000..e27823471340c --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs @@ -0,0 +1,173 @@ +// Copyright 2025 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use risingwave_common::catalog::Field; +use risingwave_common::hash::VirtualNode; +use risingwave_common::types::DataType; +use risingwave_common::util::sort_util::OrderType; +use risingwave_pb::stream_plan::LocalityProviderNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; + +use super::stream::prelude::*; +use super::utils::{TableCatalogBuilder, impl_distill_by_unit}; +use super::{ExprRewritable, PlanTreeNodeUnary, StreamNode, StreamPlanRef as PlanRef, generic}; +use crate::TableCatalog; +use crate::expr::{ExprRewriter, ExprVisitor}; +use crate::optimizer::plan_node::PlanBase; +use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::stream_fragmenter::BuildFragmentGraphState; + +/// `StreamLocalityProvider` implements [`super::LogicalLocalityProvider`] +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct StreamLocalityProvider { + pub base: PlanBase, + core: generic::LocalityProvider, +} + +impl StreamLocalityProvider { + pub fn new(core: generic::LocalityProvider) -> Self { + let input = core.input.clone(); + let dist = input.distribution().clone(); + + // LocalityProvider maintains the append-only behavior if input is append-only + let base = PlanBase::new_stream_with_core( + &core, + dist, + input.stream_kind(), + input.emit_on_window_close(), + input.watermark_columns().clone(), + input.columns_monotonicity().clone(), + ); + StreamLocalityProvider { base, core } + } + + pub fn locality_columns(&self) -> &[usize] { + &self.core.locality_columns + } +} + +impl PlanTreeNodeUnary for StreamLocalityProvider { + fn input(&self) -> PlanRef { + self.core.input.clone() + } + + fn clone_with_input(&self, input: PlanRef) -> Self { + let mut core = self.core.clone(); + core.input = input; + Self::new(core) + } +} + +impl_plan_tree_node_for_unary! { Stream, StreamLocalityProvider } +impl_distill_by_unit!(StreamLocalityProvider, core, "StreamLocalityProvider"); + +impl StreamNode for StreamLocalityProvider { + fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> PbNodeBody { + let state_table = self.build_state_catalog(state); + let progress_table = self.build_progress_catalog(state); + + let locality_provider_node = LocalityProviderNode { + locality_columns: self.locality_columns().iter().map(|&i| i as u32).collect(), + // State table for buffering input data + state_table: Some(state_table.to_prost()), + // Progress table for tracking backfill progress + progress_table: Some(progress_table.to_prost()), + }; + + PbNodeBody::LocalityProvider(locality_provider_node) + } +} + +impl ExprRewritable for StreamLocalityProvider { + fn has_rewritable_expr(&self) -> bool { + false + } + + fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { + self.clone().into() + } +} + +impl ExprVisitable for StreamLocalityProvider { + fn visit_exprs(&self, _v: &mut dyn ExprVisitor) { + // No expressions to visit + } +} + +impl StreamLocalityProvider { + /// Build the state table catalog for buffering input data + /// Schema: same as input schema (locality handled by primary key ordering) + /// Key: locality_columns (vnode handled internally by StateTable) + fn build_state_catalog(&self, state: &mut BuildFragmentGraphState) -> TableCatalog { + let mut catalog_builder = TableCatalogBuilder::default(); + let input = self.input(); + let input_schema = input.schema(); + + // Add all input columns in original order + for field in &input_schema.fields { + catalog_builder.add_column(field); + } + + // Set locality columns as primary key (vnode will be handled internally) + for locality_col_idx in self.locality_columns() { + catalog_builder.add_order_column(*locality_col_idx, OrderType::ascending()); + } + // add streaming key of the input as the rest of the primary key + if let Some(stream_key) = input.stream_key() { + for &key_col_idx in stream_key { + if !self.locality_columns().contains(&key_col_idx) { + catalog_builder.add_order_column(key_col_idx, OrderType::ascending()); + } + } + } + + // All non-locality columns are value columns + let value_indices: Vec = (0..input_schema.len()) + .filter(|&idx| !self.locality_columns().contains(&idx)) + .collect(); + catalog_builder.set_value_indices(value_indices); + + catalog_builder + .build( + self.locality_columns().to_vec(), + self.locality_columns().len(), + ) + .with_id(state.gen_table_id_wrapped()) + } + + /// Build the progress table catalog for tracking backfill progress + /// Schema: | vnode | backfill_finished | + /// Key: | vnode | + fn build_progress_catalog(&self, state: &mut BuildFragmentGraphState) -> TableCatalog { + let mut catalog_builder = TableCatalogBuilder::default(); + + // Add vnode column as primary key + catalog_builder.add_column(&Field::with_name(VirtualNode::RW_TYPE, "vnode")); + catalog_builder.add_order_column(0, OrderType::ascending()); + + // Add backfill_finished column + catalog_builder.add_column(&Field::with_name(DataType::Boolean, "backfill_finished")); + + // Set vnode column index and distribution key + catalog_builder.set_vnode_col_idx(0); + catalog_builder.set_dist_key_in_pk(vec![0]); + + // backfill_finished is the value column + catalog_builder.set_value_indices(vec![1]); + + catalog_builder + .build(vec![0], 1) + .with_id(state.gen_table_id_wrapped()) + } +} diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs new file mode 100644 index 0000000000000..0dcd219a20ede --- /dev/null +++ b/src/stream/src/executor/locality_provider.rs @@ -0,0 +1,335 @@ +// Copyright 2025 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use futures::{TryStreamExt, pin_mut}; +use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::catalog::Schema; +use risingwave_common::hash::VnodeBitmapExt; +use risingwave_common::util::epoch::EpochPair; +use risingwave_storage::StateStore; +use risingwave_storage::store::PrefetchOptions; + +use crate::common::table::state_table::StateTable; +use crate::executor::prelude::*; +use crate::task::{ActorId, FragmentId}; + +/// The `LocalityProviderExecutor` provides locality for operators during backfilling. +/// It buffers input data into a state table using locality columns as primary key prefix. +/// +/// The executor has two phases: +/// 1. Backfill phase: Buffer incoming data into state table +/// 2. Serve phase: Provide buffered data with locality after receiving backfill completion signal +pub struct LocalityProviderExecutor { + /// Upstream input + upstream: Executor, + + /// Locality columns (indices in input schema) + locality_columns: Vec, + + /// State table for buffering input data + state_table: StateTable, + + /// Progress table for tracking backfill progress + progress_table: StateTable, + + /// Schema of the input + input_schema: Schema, + + /// Actor ID + actor_id: ActorId, + + /// Fragment ID + fragment_id: FragmentId, + + /// Metrics + metrics: Arc, + + /// Chunk size for output + chunk_size: usize, +} + +impl LocalityProviderExecutor { + #[allow(clippy::too_many_arguments)] + pub fn new( + upstream: Executor, + locality_columns: Vec, + state_table: StateTable, + progress_table: StateTable, + input_schema: Schema, + actor_id: ActorId, + metrics: Arc, + chunk_size: usize, + fragment_id: FragmentId, + ) -> Self { + Self { + upstream, + locality_columns, + state_table, + progress_table, + input_schema, + actor_id, + fragment_id, + metrics, + chunk_size, + } + } + + /// Provide buffered data with locality (static method) + async fn provide_locality_data( + state_table: &StateTable, + input_schema: &Schema, + chunk_size: usize, + _epoch: EpochPair, + ) -> StreamExecutorResult> { + // Iterate through state table which is already ordered by locality columns + // Use iter_with_prefix to get all rows (empty prefix = all rows) + let empty_prefix: &[risingwave_common::types::Datum] = &[]; + let iter = state_table + .iter_with_prefix( + empty_prefix, + &( + std::ops::Bound::<&[risingwave_common::types::Datum]>::Unbounded, + std::ops::Bound::<&[risingwave_common::types::Datum]>::Unbounded, + ), + PrefetchOptions::default(), + ) + .await?; + pin_mut!(iter); + + let mut output_rows = Vec::new(); + while let Some(keyed_row) = iter.try_next().await? { + output_rows.push((Op::Insert, keyed_row)); + + // If we've collected enough rows, emit a chunk + if output_rows.len() >= chunk_size { + let chunk = StreamChunk::from_rows(&output_rows, &input_schema.data_types()); + return Ok(Some(chunk)); + } + } + + // Emit remaining rows if any + if !output_rows.is_empty() { + let chunk = StreamChunk::from_rows(&output_rows, &input_schema.data_types()); + Ok(Some(chunk)) + } else { + Ok(None) + } + } + + /// Update progress and persist state (static method) + fn update_progress( + progress_table: &mut StateTable, + _epoch: EpochPair, + ) -> StreamExecutorResult<()> { + // For LocalityProvider, we use a simple boolean flag to indicate completion + // Insert a single row into progress table to mark backfill as finished + let vnodes: Vec<_> = progress_table.vnodes().iter_vnodes().collect(); + for vnode in vnodes { + let row = [ + Some(vnode.to_scalar().into()), + Some(risingwave_common::types::ScalarImpl::Bool(true)), + ]; + progress_table.insert(&row); + } + Ok(()) + } + + /// Check progress state by reading progress table (static method) + /// Returns (`has_progress_state`, `is_backfill_finished`) + /// - `has_progress_state`: true if we have any progress state recorded + /// - `is_backfill_finished`: true if backfill is completed (only valid when `has_progress_state` is true) + async fn check_backfill_progress( + progress_table: &StateTable, + ) -> StreamExecutorResult<(bool, bool)> { + let mut vnodes = progress_table.vnodes().iter_vnodes_scalar(); + let first_vnode = vnodes.next().unwrap(); + let key: &[risingwave_common::types::Datum] = &[Some(first_vnode.into())]; + + if let Some(row) = progress_table.get_row(key).await? { + // Row exists, check the finished flag + let is_finished: bool = row.datum_at(1).unwrap().into_bool(); + Ok((true, is_finished)) + } else { + // No row exists, backfill not started yet + Ok((false, false)) + } + } +} + +impl Execute for LocalityProviderExecutor { + fn execute(self: Box) -> BoxedMessageStream { + self.execute_inner().boxed() + } +} + +impl LocalityProviderExecutor { + #[try_stream(ok = Message, error = StreamExecutorError)] + async fn execute_inner(mut self) { + // Extract actor_id before we consume self + let actor_id = self.actor_id; + + let mut upstream = self.upstream.execute(); + + // Wait for first barrier to initialize + let first_barrier = expect_first_barrier(&mut upstream).await?; + let first_epoch = first_barrier.epoch; + let is_newly_added = first_barrier.is_newly_added(actor_id); + + // Initialize state tables + self.state_table.init_epoch(first_epoch).await?; + self.progress_table.init_epoch(first_epoch).await?; + + // Propagate the first barrier + yield Message::Barrier(first_barrier); + + // Check progress state using static method to avoid borrowing issues + let (has_progress_state, is_backfill_finished) = + Self::check_backfill_progress(&self.progress_table).await?; + + // Determine what to do based on progress state: + // - If no progress state exists: need to buffer chunks (backfill not started) + // - If progress state exists but not finished: backfill in progress, no buffering anymore + // - If progress state exists and finished: pass-through mode (backfill completed) + let need_buffering = !has_progress_state; + let is_completely_finished = has_progress_state && is_backfill_finished; + + if is_completely_finished { + assert!(!is_newly_added); + } + + tracing::info!( + actor_id = actor_id, + has_progress_state = has_progress_state, + is_backfill_finished = is_backfill_finished, + need_buffering = need_buffering, + "LocalityProvider initialized" + ); + + if need_buffering { + // Enter buffering phase - buffer data until backfill completion signal + let mut backfill_complete = false; + + #[for_await] + for msg in upstream.by_ref() { + let msg = msg?; + + match msg { + Message::Watermark(_) => { + // Forward watermarks + yield msg; + } + Message::Chunk(chunk) => { + for (op, row_ref) in chunk.rows() { + match op { + Op::Insert | Op::UpdateInsert => { + self.state_table.insert(row_ref); + } + Op::Delete | Op::UpdateDelete => { + self.state_table.delete(row_ref); + } + } + } + } + Message::Barrier(barrier) => { + let epoch = barrier.epoch; + + // Commit state tables + let post_commit = self.state_table.commit(epoch).await?; + + // Check if this is a backfill completion signal + // For now, use a simple heuristic (in practice, this should be a proper signal) + if !backfill_complete { + // TODO: Replace with actual backfill completion detection + // For now, assume backfill completes after receiving some data + backfill_complete = true; // Simplified for demo + + if backfill_complete { + tracing::info!( + actor_id = actor_id, + "LocalityProvider backfill completed, updating progress" + ); + + // Update progress to completed + Self::update_progress(&mut self.progress_table, epoch)?; + let progress_post_commit = + self.progress_table.commit(epoch).await?; + + // Provide buffered data with locality + // if let Some(locality_chunk) = + // self.provide_locality_data(epoch).await? + // { + // yield Message::Chunk(locality_chunk); + // } + + yield Message::Barrier(barrier); + progress_post_commit.post_yield_barrier(None).await?; + break; // Exit buffering phase + } + } + + yield Message::Barrier(barrier); + post_commit.post_yield_barrier(None).await?; + } + } + } + + tracing::debug!( + actor_id = actor_id, + "LocalityProvider backfill finished, entering passthrough mode" + ); + } + + // After backfill completion (or if already completed), forward messages directly + #[for_await] + for msg in upstream { + let msg = msg?; + + match msg { + Message::Barrier(barrier) => { + // Commit state tables but don't modify them + self.state_table + .commit_assert_no_update_vnode_bitmap(barrier.epoch) + .await?; + self.progress_table + .commit_assert_no_update_vnode_bitmap(barrier.epoch) + .await?; + yield Message::Barrier(barrier); + } + _ => { + // Forward all other messages directly + yield msg; + } + } + } + } +} + +#[cfg(test)] +mod tests { + use risingwave_common::array::StreamChunk; + use risingwave_common::catalog::{Field, Schema}; + use risingwave_common::types::DataType; + use risingwave_storage::memory::MemoryStateStore; + + use super::*; + use crate::executor::test_utils::MockSource; + + #[tokio::test] + async fn test_locality_provider_basic() { + // This is a basic test structure + // TODO: Implement comprehensive tests + } +} diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index b474fb16bb737..1fc6641a0c936 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -86,6 +86,7 @@ mod filter; pub mod hash_join; mod hop_window; mod join; +pub mod locality_provider; mod lookup; mod lookup_union; mod merge; diff --git a/src/stream/src/from_proto/locality_provider.rs b/src/stream/src/from_proto/locality_provider.rs new file mode 100644 index 0000000000000..4f55bdaa54b03 --- /dev/null +++ b/src/stream/src/from_proto/locality_provider.rs @@ -0,0 +1,82 @@ +// Copyright 2025 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use risingwave_pb::stream_plan::LocalityProviderNode; +use risingwave_storage::StateStore; + +use super::*; +use crate::common::table::state_table::StateTableBuilder; +use crate::executor::Executor; +use crate::executor::locality_provider::LocalityProviderExecutor; + +pub struct LocalityProviderBuilder; + +impl ExecutorBuilder for LocalityProviderBuilder { + type Node = LocalityProviderNode; + + async fn new_boxed_executor( + params: ExecutorParams, + node: &Self::Node, + store: impl StateStore, + ) -> StreamResult { + let [input]: [_; 1] = params.input.try_into().unwrap(); + + let locality_columns = node + .locality_columns + .iter() + .map(|&i| i as usize) + .collect::>(); + + let input_schema = input.schema().clone(); + + let vnodes = Some(Arc::new( + params + .vnode_bitmap + .expect("vnodes not set for locality provider"), + )); + + // Create state table for buffering input data + let state_table = StateTableBuilder::new( + node.get_state_table().unwrap(), + store.clone(), + vnodes.clone(), + ) + .enable_preload_all_rows_by_config(¶ms.actor_context.streaming_config) + .build() + .await; + + // Create progress table for tracking backfill progress + let progress_table = + StateTableBuilder::new(node.get_progress_table().unwrap(), store, vnodes) + .enable_preload_all_rows_by_config(¶ms.actor_context.streaming_config) + .build() + .await; + + let exec = LocalityProviderExecutor::new( + input, + locality_columns, + state_table, + progress_table, + input_schema, + params.info.id.try_into().unwrap(), // Use executor id as actor id + params.executor_stats.clone(), + 1024, // default chunk size + params.fragment_id, + ); + + Ok((params.info, exec).into()) + } +} diff --git a/src/stream/src/from_proto/mod.rs b/src/stream/src/from_proto/mod.rs index 90f2eeb3a6eee..25c0bfa8d5169 100644 --- a/src/stream/src/from_proto/mod.rs +++ b/src/stream/src/from_proto/mod.rs @@ -30,6 +30,7 @@ mod group_top_n; mod hash_agg; mod hash_join; mod hop_window; +mod locality_provider; mod lookup; mod lookup_union; mod materialized_exprs; @@ -85,6 +86,7 @@ use self::group_top_n::GroupTopNExecutorBuilder; use self::hash_agg::*; use self::hash_join::*; use self::hop_window::*; +use self::locality_provider::*; use self::lookup::*; use self::lookup_union::*; use self::materialized_exprs::MaterializedExprsExecutorBuilder; @@ -201,5 +203,6 @@ pub async fn create_executor( NodeBody::MaterializedExprs => MaterializedExprsExecutorBuilder, NodeBody::VectorIndexWrite => VectorIndexWriteExecutorBuilder, NodeBody::UpstreamSinkUnion => UpstreamSinkUnionExecutorBuilder, + NodeBody::LocalityProvider => LocalityProviderBuilder, } } From b8a76df36290194527023d78967b4545f3cfa582 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Thu, 18 Sep 2025 17:06:07 +0800 Subject: [PATCH 02/36] fix locality state --- .../plan_node/stream_locality_provider.rs | 44 +++++++++--- src/stream/src/executor/locality_provider.rs | 69 +++++++++++-------- 2 files changed, 72 insertions(+), 41 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs index e27823471340c..548f6e7b7d546 100644 --- a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs +++ b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use itertools::Itertools; use risingwave_common::catalog::Field; use risingwave_common::hash::VirtualNode; use risingwave_common::types::DataType; @@ -132,39 +133,60 @@ impl StreamLocalityProvider { } } - // All non-locality columns are value columns - let value_indices: Vec = (0..input_schema.len()) - .filter(|&idx| !self.locality_columns().contains(&idx)) - .collect(); - catalog_builder.set_value_indices(value_indices); + catalog_builder.set_value_indices((0..input_schema.len()).into_iter().collect()); catalog_builder .build( - self.locality_columns().to_vec(), - self.locality_columns().len(), + self.input().distribution().dist_column_indices().to_vec(), + 0, ) .with_id(state.gen_table_id_wrapped()) } /// Build the progress table catalog for tracking backfill progress - /// Schema: | vnode | backfill_finished | - /// Key: | vnode | + /// Schema: | vnode | pk(locality columns + input stream keys) | backfill_finished | row_count | + /// Key: | vnode | pk(locality columns + input stream keys) | fn build_progress_catalog(&self, state: &mut BuildFragmentGraphState) -> TableCatalog { let mut catalog_builder = TableCatalogBuilder::default(); + let input = self.input(); + let input_schema = input.schema(); // Add vnode column as primary key catalog_builder.add_column(&Field::with_name(VirtualNode::RW_TYPE, "vnode")); catalog_builder.add_order_column(0, OrderType::ascending()); + // Add locality columns as part of primary key + for &locality_col_idx in self.locality_columns() { + let field = &input_schema.fields[locality_col_idx]; + catalog_builder.add_column(field); + catalog_builder + .add_order_column(catalog_builder.columns().len() - 1, OrderType::ascending()); + } + + // Add stream key columns as part of primary key (excluding those already added as locality columns) + if let Some(stream_key) = input.stream_key() { + for &key_col_idx in stream_key { + if !self.locality_columns().contains(&key_col_idx) { + let field = &input_schema.fields[key_col_idx]; + catalog_builder.add_column(field); + catalog_builder + .add_order_column(catalog_builder.columns().len() - 1, OrderType::ascending()); + } + } + } + // Add backfill_finished column catalog_builder.add_column(&Field::with_name(DataType::Boolean, "backfill_finished")); + // Add row_count column + catalog_builder.add_column(&Field::with_name(DataType::Int64, "row_count")); + // Set vnode column index and distribution key catalog_builder.set_vnode_col_idx(0); catalog_builder.set_dist_key_in_pk(vec![0]); - // backfill_finished is the value column - catalog_builder.set_value_indices(vec![1]); + let num_of_columns = catalog_builder.columns().len(); + catalog_builder.set_value_indices((1..num_of_columns).collect_vec()); catalog_builder .build(vec![0], 1) diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index 0dcd219a20ede..4174140d34f1b 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -18,6 +18,7 @@ use futures::{TryStreamExt, pin_mut}; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::catalog::Schema; use risingwave_common::hash::VnodeBitmapExt; +use risingwave_common::row::OwnedRow; use risingwave_common::util::epoch::EpochPair; use risingwave_storage::StateStore; use risingwave_storage::store::PrefetchOptions; @@ -132,17 +133,30 @@ impl LocalityProviderExecutor { /// Update progress and persist state (static method) fn update_progress( progress_table: &mut StateTable, + locality_columns: &[usize], + input_schema: &Schema, _epoch: EpochPair, ) -> StreamExecutorResult<()> { // For LocalityProvider, we use a simple boolean flag to indicate completion // Insert a single row into progress table to mark backfill as finished let vnodes: Vec<_> = progress_table.vnodes().iter_vnodes().collect(); for vnode in vnodes { - let row = [ - Some(vnode.to_scalar().into()), - Some(risingwave_common::types::ScalarImpl::Bool(true)), - ]; - progress_table.insert(&row); + // Build the full primary key: vnode + locality columns (defaulted to NULL for now) + let mut row_data = vec![Some(vnode.to_scalar().into())]; + + // Add locality column values (NULL for now since this is just marking completion) + for _ in locality_columns { + row_data.push(None); // NULL value for the locality column + } + + // Add backfill_finished = true + row_data.push(Some(risingwave_common::types::ScalarImpl::Bool(true))); + + // Add row_count = 0 (we don't track actual row count for now) + row_data.push(Some(risingwave_common::types::ScalarImpl::Int64(0))); + + let row = OwnedRow::new(row_data); + progress_table.insert(row); } Ok(()) } @@ -153,14 +167,22 @@ impl LocalityProviderExecutor { /// - `is_backfill_finished`: true if backfill is completed (only valid when `has_progress_state` is true) async fn check_backfill_progress( progress_table: &StateTable, + locality_columns: &[usize], ) -> StreamExecutorResult<(bool, bool)> { let mut vnodes = progress_table.vnodes().iter_vnodes_scalar(); let first_vnode = vnodes.next().unwrap(); - let key: &[risingwave_common::types::Datum] = &[Some(first_vnode.into())]; - if let Some(row) = progress_table.get_row(key).await? { - // Row exists, check the finished flag - let is_finished: bool = row.datum_at(1).unwrap().into_bool(); + // Build key with vnode + NULL values for locality columns (to check any progress entry) + let mut key_data = vec![Some(first_vnode.into())]; + for _ in locality_columns { + key_data.push(None); // NULL value for locality column + } + let key = OwnedRow::new(key_data); + + if let Some(row) = progress_table.get_row(&key).await? { + // Row exists, check the finished flag (it's at position 1 + locality_columns.len()) + let finished_col_idx = 1 + locality_columns.len(); + let is_finished: bool = row.datum_at(finished_col_idx).unwrap().into_bool(); Ok((true, is_finished)) } else { // No row exists, backfill not started yet @@ -197,7 +219,7 @@ impl LocalityProviderExecutor { // Check progress state using static method to avoid borrowing issues let (has_progress_state, is_backfill_finished) = - Self::check_backfill_progress(&self.progress_table).await?; + Self::check_backfill_progress(&self.progress_table, &self.locality_columns).await?; // Determine what to do based on progress state: // - If no progress state exists: need to buffer chunks (backfill not started) @@ -228,8 +250,7 @@ impl LocalityProviderExecutor { match msg { Message::Watermark(_) => { - // Forward watermarks - yield msg; + // Ignore watermarks during backfill } Message::Chunk(chunk) => { for (op, row_ref) in chunk.rows() { @@ -263,7 +284,12 @@ impl LocalityProviderExecutor { ); // Update progress to completed - Self::update_progress(&mut self.progress_table, epoch)?; + Self::update_progress( + &mut self.progress_table, + &self.locality_columns, + &self.input_schema, + epoch, + )?; let progress_post_commit = self.progress_table.commit(epoch).await?; @@ -316,20 +342,3 @@ impl LocalityProviderExecutor { } } } - -#[cfg(test)] -mod tests { - use risingwave_common::array::StreamChunk; - use risingwave_common::catalog::{Field, Schema}; - use risingwave_common::types::DataType; - use risingwave_storage::memory::MemoryStateStore; - - use super::*; - use crate::executor::test_utils::MockSource; - - #[tokio::test] - async fn test_locality_provider_basic() { - // This is a basic test structure - // TODO: Implement comprehensive tests - } -} From 68851361a0c6e1978bfff948edc20089bd5151a7 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 19 Sep 2025 10:49:47 +0800 Subject: [PATCH 03/36] fix --- src/common/src/util/stream_graph_visitor.rs | 4 + .../plan_node/stream_locality_provider.rs | 10 +- src/stream/src/executor/locality_provider.rs | 131 +++++++----------- .../src/from_proto/locality_provider.rs | 2 - 4 files changed, 55 insertions(+), 92 deletions(-) diff --git a/src/common/src/util/stream_graph_visitor.rs b/src/common/src/util/stream_graph_visitor.rs index f901b47c49112..c4115e4e31b24 100644 --- a/src/common/src/util/stream_graph_visitor.rs +++ b/src/common/src/util/stream_graph_visitor.rs @@ -309,6 +309,10 @@ pub fn visit_stream_node_tables_inner( always!(node.table, "StreamVectorIndexWrite"); } + NodeBody::LocalityProvider(node) => { + always!(node.state_table, "LocalityProviderState"); + always!(node.progress_table, "LocalityProviderProgress"); + } _ => {} } }; diff --git a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs index 548f6e7b7d546..4f1bb35634fbf 100644 --- a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs +++ b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs @@ -159,19 +159,13 @@ impl StreamLocalityProvider { for &locality_col_idx in self.locality_columns() { let field = &input_schema.fields[locality_col_idx]; catalog_builder.add_column(field); - catalog_builder - .add_order_column(catalog_builder.columns().len() - 1, OrderType::ascending()); } // Add stream key columns as part of primary key (excluding those already added as locality columns) if let Some(stream_key) = input.stream_key() { for &key_col_idx in stream_key { - if !self.locality_columns().contains(&key_col_idx) { - let field = &input_schema.fields[key_col_idx]; - catalog_builder.add_column(field); - catalog_builder - .add_order_column(catalog_builder.columns().len() - 1, OrderType::ascending()); - } + let field = &input_schema.fields[key_col_idx]; + catalog_builder.add_column(field); } } diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index 4174140d34f1b..73235510445d5 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -25,7 +25,7 @@ use risingwave_storage::store::PrefetchOptions; use crate::common::table::state_table::StateTable; use crate::executor::prelude::*; -use crate::task::{ActorId, FragmentId}; + /// The `LocalityProviderExecutor` provides locality for operators during backfilling. /// It buffers input data into a state table using locality columns as primary key prefix. @@ -49,12 +49,6 @@ pub struct LocalityProviderExecutor { /// Schema of the input input_schema: Schema, - /// Actor ID - actor_id: ActorId, - - /// Fragment ID - fragment_id: FragmentId, - /// Metrics metrics: Arc, @@ -70,10 +64,8 @@ impl LocalityProviderExecutor { state_table: StateTable, progress_table: StateTable, input_schema: Schema, - actor_id: ActorId, metrics: Arc, chunk_size: usize, - fragment_id: FragmentId, ) -> Self { Self { upstream, @@ -81,8 +73,6 @@ impl LocalityProviderExecutor { state_table, progress_table, input_schema, - actor_id, - fragment_id, metrics, chunk_size, } @@ -167,21 +157,17 @@ impl LocalityProviderExecutor { /// - `is_backfill_finished`: true if backfill is completed (only valid when `has_progress_state` is true) async fn check_backfill_progress( progress_table: &StateTable, - locality_columns: &[usize], ) -> StreamExecutorResult<(bool, bool)> { let mut vnodes = progress_table.vnodes().iter_vnodes_scalar(); let first_vnode = vnodes.next().unwrap(); // Build key with vnode + NULL values for locality columns (to check any progress entry) let mut key_data = vec![Some(first_vnode.into())]; - for _ in locality_columns { - key_data.push(None); // NULL value for locality column - } let key = OwnedRow::new(key_data); if let Some(row) = progress_table.get_row(&key).await? { // Row exists, check the finished flag (it's at position 1 + locality_columns.len()) - let finished_col_idx = 1 + locality_columns.len(); + let finished_col_idx = row.len() - 2; // backfill_finished is second last column let is_finished: bool = row.datum_at(finished_col_idx).unwrap().into_bool(); Ok((true, is_finished)) } else { @@ -200,26 +186,23 @@ impl Execute for LocalityProviderExecutor { impl LocalityProviderExecutor { #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(mut self) { - // Extract actor_id before we consume self - let actor_id = self.actor_id; - let mut upstream = self.upstream.execute(); + // Wait for first barrier to initialize let first_barrier = expect_first_barrier(&mut upstream).await?; let first_epoch = first_barrier.epoch; - let is_newly_added = first_barrier.is_newly_added(actor_id); + + // Propagate the first barrier + yield Message::Barrier(first_barrier); // Initialize state tables self.state_table.init_epoch(first_epoch).await?; self.progress_table.init_epoch(first_epoch).await?; - // Propagate the first barrier - yield Message::Barrier(first_barrier); - // Check progress state using static method to avoid borrowing issues let (has_progress_state, is_backfill_finished) = - Self::check_backfill_progress(&self.progress_table, &self.locality_columns).await?; + Self::check_backfill_progress(&self.progress_table).await?; // Determine what to do based on progress state: // - If no progress state exists: need to buffer chunks (backfill not started) @@ -228,18 +211,15 @@ impl LocalityProviderExecutor { let need_buffering = !has_progress_state; let is_completely_finished = has_progress_state && is_backfill_finished; - if is_completely_finished { - assert!(!is_newly_added); - } - tracing::info!( - actor_id = actor_id, has_progress_state = has_progress_state, is_backfill_finished = is_backfill_finished, need_buffering = need_buffering, "LocalityProvider initialized" ); + let mut barrier_count = 0; + if need_buffering { // Enter buffering phase - buffer data until backfill completion signal let mut backfill_complete = false; @@ -253,71 +233,58 @@ impl LocalityProviderExecutor { // Ignore watermarks during backfill } Message::Chunk(chunk) => { - for (op, row_ref) in chunk.rows() { - match op { - Op::Insert | Op::UpdateInsert => { - self.state_table.insert(row_ref); - } - Op::Delete | Op::UpdateDelete => { - self.state_table.delete(row_ref); - } - } - } + self.state_table.write_chunk(chunk); + self.state_table.try_flush().await?; } Message::Barrier(barrier) => { let epoch = barrier.epoch; // Commit state tables - let post_commit = self.state_table.commit(epoch).await?; - - // Check if this is a backfill completion signal - // For now, use a simple heuristic (in practice, this should be a proper signal) - if !backfill_complete { - // TODO: Replace with actual backfill completion detection - // For now, assume backfill completes after receiving some data - backfill_complete = true; // Simplified for demo - - if backfill_complete { - tracing::info!( - actor_id = actor_id, - "LocalityProvider backfill completed, updating progress" - ); - - // Update progress to completed - Self::update_progress( - &mut self.progress_table, - &self.locality_columns, - &self.input_schema, - epoch, - )?; - let progress_post_commit = - self.progress_table.commit(epoch).await?; - - // Provide buffered data with locality - // if let Some(locality_chunk) = - // self.provide_locality_data(epoch).await? - // { - // yield Message::Chunk(locality_chunk); - // } - - yield Message::Barrier(barrier); - progress_post_commit.post_yield_barrier(None).await?; - break; // Exit buffering phase - } - } + let post_commit1 = self.state_table.commit(epoch).await?; + let post_commit2 = self.progress_table.commit(epoch).await?; yield Message::Barrier(barrier); - post_commit.post_yield_barrier(None).await?; + post_commit1.post_yield_barrier(None).await?; + post_commit2.post_yield_barrier(None).await?; + barrier_count += 1; + if barrier_count >= 100 { + break; + } } } } - - tracing::debug!( - actor_id = actor_id, - "LocalityProvider backfill finished, entering passthrough mode" - ); } + // TODO: implement backfill loop here + + // Arrangement Backfill Algorithm: + // + // backfill_stream + // / \ + // upstream snapshot + // + // We construct a backfill stream with upstream as its left input and mv snapshot read + // stream as its right input. When a chunk comes from upstream, we will buffer it. + // + // When a barrier comes from upstream: + // Immediately break out of backfill loop. + // - For each row of the upstream chunk buffer, compute vnode. + // - Get the `current_pos` corresponding to the vnode. Forward it to downstream if its pk + // <= `current_pos`, otherwise ignore it. + // - Flush all buffered upstream_chunks to replicated state table. + // - Update the `snapshot_read_epoch`. + // - Reconstruct the whole backfill stream with upstream and new mv snapshot read stream + // with the `snapshot_read_epoch`. + // + // When a chunk comes from snapshot, we forward it to the downstream and raise + // `current_pos`. + // + // When we reach the end of the snapshot read stream, it means backfill has been + // finished. + // + // Once the backfill loop ends, we forward the upstream directly to the downstream. + + // After backfill completion (or if already completed), forward messages directly #[for_await] for msg in upstream { diff --git a/src/stream/src/from_proto/locality_provider.rs b/src/stream/src/from_proto/locality_provider.rs index 4f55bdaa54b03..ef02a262d2961 100644 --- a/src/stream/src/from_proto/locality_provider.rs +++ b/src/stream/src/from_proto/locality_provider.rs @@ -71,10 +71,8 @@ impl ExecutorBuilder for LocalityProviderBuilder { state_table, progress_table, input_schema, - params.info.id.try_into().unwrap(), // Use executor id as actor id params.executor_stats.clone(), 1024, // default chunk size - params.fragment_id, ); Ok((params.info, exec).into()) From 58fe6d8db1718a8859a3b3ad439f483c9433a7fc Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 19 Sep 2025 11:30:44 +0800 Subject: [PATCH 04/36] add backfill --- src/stream/src/executor/locality_provider.rs | 650 ++++++++++++++----- 1 file changed, 499 insertions(+), 151 deletions(-) diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index 73235510445d5..437f6fdf2720c 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -12,27 +12,147 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::sync::Arc; -use futures::{TryStreamExt, pin_mut}; +use futures_async_stream::try_stream; +use either::Either; +use futures::stream::select_with_strategy; +use futures::{TryStreamExt, pin_mut, stream}; +use itertools::Itertools; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::catalog::Schema; -use risingwave_common::hash::VnodeBitmapExt; -use risingwave_common::row::OwnedRow; -use risingwave_common::util::epoch::EpochPair; +use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; +use risingwave_common::row::{OwnedRow, Row, RowExt}; +use risingwave_common::types::{Datum, ToOwnedDatum}; +use risingwave_common::util::sort_util::cmp_datum_iter; use risingwave_storage::StateStore; use risingwave_storage::store::PrefetchOptions; use crate::common::table::state_table::StateTable; use crate::executor::prelude::*; +/// Progress state for tracking backfill per vnode +#[derive(Clone, Debug, PartialEq, Eq)] +enum LocalityBackfillProgress { + /// Backfill not started for this vnode + NotStarted, + /// Backfill in progress, tracking current position + InProgress { + /// Current position in the locality-ordered scan + current_pos: OwnedRow, + /// Number of rows processed for this vnode + processed_rows: u64, + }, + /// Backfill completed for this vnode + Completed { + /// Final position reached + final_pos: OwnedRow, + /// Total rows processed for this vnode + total_rows: u64, + }, +} + +/// State management for locality provider backfill process +#[derive(Clone, Debug)] +struct LocalityBackfillState { + /// Progress per vnode + per_vnode: HashMap, + /// Total snapshot rows read across all vnodes + total_snapshot_rows: u64, +} + +impl LocalityBackfillState { + fn new(vnodes: impl Iterator) -> Self { + let per_vnode = vnodes + .map(|vnode| (vnode, LocalityBackfillProgress::NotStarted)) + .collect(); + Self { + per_vnode, + total_snapshot_rows: 0, + } + } + + fn is_completed(&self) -> bool { + self.per_vnode.values().all(|progress| { + matches!(progress, LocalityBackfillProgress::Completed { .. }) + }) + } + + fn has_progress(&self) -> bool { + self.per_vnode.values().any(|progress| { + matches!(progress, LocalityBackfillProgress::InProgress { .. }) + }) + } + + fn update_progress( + &mut self, + vnode: VirtualNode, + new_pos: OwnedRow, + row_count_delta: u64, + ) { + let progress = self.per_vnode.get_mut(&vnode).unwrap(); + match progress { + LocalityBackfillProgress::NotStarted => { + *progress = LocalityBackfillProgress::InProgress { + current_pos: new_pos, + processed_rows: row_count_delta, + }; + } + LocalityBackfillProgress::InProgress { processed_rows, .. } => { + *progress = LocalityBackfillProgress::InProgress { + current_pos: new_pos, + processed_rows: *processed_rows + row_count_delta, + }; + } + LocalityBackfillProgress::Completed { .. } => { + // Already completed, shouldn't update + } + } + self.total_snapshot_rows += row_count_delta; + } + + fn finish_vnode(&mut self, vnode: VirtualNode) { + let progress = self.per_vnode.get_mut(&vnode).unwrap(); + match progress { + LocalityBackfillProgress::NotStarted => { + *progress = LocalityBackfillProgress::Completed { + final_pos: OwnedRow::empty(), + total_rows: 0, + }; + } + LocalityBackfillProgress::InProgress { + current_pos, + processed_rows, + } => { + *progress = LocalityBackfillProgress::Completed { + final_pos: current_pos.clone(), + total_rows: *processed_rows, + }; + } + LocalityBackfillProgress::Completed { .. } => { + // Already completed + } + } + } + + fn get_progress(&self, vnode: &VirtualNode) -> &LocalityBackfillProgress { + self.per_vnode.get(vnode).unwrap() + } +} + /// The `LocalityProviderExecutor` provides locality for operators during backfilling. /// It buffers input data into a state table using locality columns as primary key prefix. /// -/// The executor has two phases: -/// 1. Backfill phase: Buffer incoming data into state table -/// 2. Serve phase: Provide buffered data with locality after receiving backfill completion signal +/// The executor implements a proper backfill process similar to arrangement backfill: +/// 1. Backfill phase: Buffer incoming data and provide locality-ordered snapshot reads +/// 2. Forward phase: Once backfill is complete, forward upstream messages directly +/// +/// Key improvements over the original implementation: +/// - Removes arbitrary 100-barrier buffer limit +/// - Implements proper upstream chunk tracking during backfill +/// - Uses per-vnode progress tracking for better state management pub struct LocalityProviderExecutor { /// Upstream input upstream: Executor, @@ -43,7 +163,7 @@ pub struct LocalityProviderExecutor { /// State table for buffering input data state_table: StateTable, - /// Progress table for tracking backfill progress + /// Progress table for tracking backfill progress per vnode progress_table: StateTable, /// Schema of the input @@ -54,6 +174,7 @@ pub struct LocalityProviderExecutor { /// Chunk size for output chunk_size: usize, + } impl LocalityProviderExecutor { @@ -78,72 +199,92 @@ impl LocalityProviderExecutor { } } - /// Provide buffered data with locality (static method) - async fn provide_locality_data( - state_table: &StateTable, - input_schema: &Schema, - chunk_size: usize, - _epoch: EpochPair, - ) -> StreamExecutorResult> { - // Iterate through state table which is already ordered by locality columns - // Use iter_with_prefix to get all rows (empty prefix = all rows) - let empty_prefix: &[risingwave_common::types::Datum] = &[]; - let iter = state_table - .iter_with_prefix( - empty_prefix, - &( - std::ops::Bound::<&[risingwave_common::types::Datum]>::Unbounded, - std::ops::Bound::<&[risingwave_common::types::Datum]>::Unbounded, - ), - PrefetchOptions::default(), - ) - .await?; - pin_mut!(iter); - - let mut output_rows = Vec::new(); - while let Some(keyed_row) = iter.try_next().await? { - output_rows.push((Op::Insert, keyed_row)); - - // If we've collected enough rows, emit a chunk - if output_rows.len() >= chunk_size { - let chunk = StreamChunk::from_rows(&output_rows, &input_schema.data_types()); - return Ok(Some(chunk)); + /// Creates a snapshot stream that reads from state table in locality order + #[try_stream(ok = Option<(VirtualNode, OwnedRow)>, error = StreamExecutorError)] + async fn make_snapshot_stream<'a>( + state_table: &'a StateTable, + backfill_state: LocalityBackfillState, + ) { + // Read from state table per vnode in locality order + for vnode in state_table.vnodes().iter_vnodes() { + let progress = backfill_state.get_progress(&vnode); + + let current_pos = match progress { + LocalityBackfillProgress::NotStarted => None, + LocalityBackfillProgress::Completed { .. } => { + // Skip completed vnodes + continue; + } + LocalityBackfillProgress::InProgress { current_pos, .. } => { + Some(current_pos.clone()) + } + }; + + // Compute range bounds for iteration based on current position + let range_bounds = if let Some(ref pos) = current_pos { + let start_bound = std::ops::Bound::Excluded(pos.as_inner()); + (start_bound, std::ops::Bound::<&[Datum]>::Unbounded) + } else { + ( + std::ops::Bound::<&[Datum]>::Unbounded, + std::ops::Bound::<&[Datum]>::Unbounded, + ) + }; + + // Iterate over rows for this vnode + let iter = state_table + .iter_with_vnode( + vnode, + &range_bounds, + PrefetchOptions::prefetch_for_small_range_scan(), + ) + .await?; + pin_mut!(iter); + + while let Some(row) = iter.try_next().await? { + yield Some((vnode, row)); } } - // Emit remaining rows if any - if !output_rows.is_empty() { - let chunk = StreamChunk::from_rows(&output_rows, &input_schema.data_types()); - Ok(Some(chunk)) - } else { - Ok(None) - } + // Signal end of stream + yield None; } - /// Update progress and persist state (static method) - fn update_progress( + /// Persist backfill state to progress table + async fn persist_backfill_state( progress_table: &mut StateTable, + backfill_state: &LocalityBackfillState, locality_columns: &[usize], - input_schema: &Schema, - _epoch: EpochPair, ) -> StreamExecutorResult<()> { - // For LocalityProvider, we use a simple boolean flag to indicate completion - // Insert a single row into progress table to mark backfill as finished - let vnodes: Vec<_> = progress_table.vnodes().iter_vnodes().collect(); - for vnode in vnodes { - // Build the full primary key: vnode + locality columns (defaulted to NULL for now) + for (vnode, progress) in &backfill_state.per_vnode { + let (is_finished, current_pos, row_count) = match progress { + LocalityBackfillProgress::NotStarted => continue, // Don't persist NotStarted + LocalityBackfillProgress::InProgress { + current_pos, + processed_rows, + } => (false, current_pos.clone(), *processed_rows), + LocalityBackfillProgress::Completed { + final_pos, + total_rows, + } => (true, final_pos.clone(), *total_rows), + }; + + // Build progress row: vnode + current_pos + is_finished + row_count let mut row_data = vec![Some(vnode.to_scalar().into())]; - // Add locality column values (NULL for now since this is just marking completion) - for _ in locality_columns { - row_data.push(None); // NULL value for the locality column + // Add current position values + row_data.extend_from_slice(current_pos.as_inner()); + + // Pad with NULLs if position is shorter than locality columns + while row_data.len() < 1 + locality_columns.len() { + row_data.push(None); } - // Add backfill_finished = true - row_data.push(Some(risingwave_common::types::ScalarImpl::Bool(true))); + // Add is_finished flag + row_data.push(Some(risingwave_common::types::ScalarImpl::Bool(is_finished))); - // Add row_count = 0 (we don't track actual row count for now) - row_data.push(Some(risingwave_common::types::ScalarImpl::Int64(0))); + // Add row count + row_data.push(Some(risingwave_common::types::ScalarImpl::Int64(row_count as i64))); let row = OwnedRow::new(row_data); progress_table.insert(row); @@ -151,29 +292,118 @@ impl LocalityProviderExecutor { Ok(()) } - /// Check progress state by reading progress table (static method) - /// Returns (`has_progress_state`, `is_backfill_finished`) - /// - `has_progress_state`: true if we have any progress state recorded - /// - `is_backfill_finished`: true if backfill is completed (only valid when `has_progress_state` is true) - async fn check_backfill_progress( + /// Load backfill state from progress table + async fn load_backfill_state( progress_table: &StateTable, - ) -> StreamExecutorResult<(bool, bool)> { - let mut vnodes = progress_table.vnodes().iter_vnodes_scalar(); - let first_vnode = vnodes.next().unwrap(); - - // Build key with vnode + NULL values for locality columns (to check any progress entry) - let mut key_data = vec![Some(first_vnode.into())]; - let key = OwnedRow::new(key_data); - - if let Some(row) = progress_table.get_row(&key).await? { - // Row exists, check the finished flag (it's at position 1 + locality_columns.len()) - let finished_col_idx = row.len() - 2; // backfill_finished is second last column - let is_finished: bool = row.datum_at(finished_col_idx).unwrap().into_bool(); - Ok((true, is_finished)) - } else { - // No row exists, backfill not started yet - Ok((false, false)) + locality_columns: &[usize], + ) -> StreamExecutorResult { + let mut backfill_state = LocalityBackfillState::new( + progress_table.vnodes().iter_vnodes() + ); + let mut total_snapshot_rows = 0; + + // Scan progress table to restore state + let empty_prefix: &[Datum] = &[]; + let iter = progress_table + .iter_with_prefix( + empty_prefix, + &( + std::ops::Bound::<&[Datum]>::Unbounded, + std::ops::Bound::<&[Datum]>::Unbounded, + ), + PrefetchOptions::default(), + ) + .await?; + pin_mut!(iter); + + while let Some(row) = iter.try_next().await? { + if row.len() < 3 { + continue; // Skip malformed rows + } + + // Parse vnode (first column) + let vnode_datum = row.datum_at(0); + if vnode_datum.is_none() { + continue; + } + let vnode = VirtualNode::from_scalar(vnode_datum.unwrap().into_int16()); + + // Parse is_finished flag (second to last column) + let finished_col_idx = row.len() - 2; + let is_finished = row.datum_at(finished_col_idx) + .map(|d| d.into_bool()) + .unwrap_or(false); + + // Parse row count (last column) + let row_count = row.datum_at(row.len() - 1) + .map(|d| d.into_int64() as u64) + .unwrap_or(0); + + // Extract current position (columns 1 to 1+locality_columns.len()) + let pos_end = std::cmp::min(1 + locality_columns.len(), finished_col_idx); + let current_pos_data: Vec = (1..pos_end) + .map(|i| row.datum_at(i).to_owned_datum()) + .collect(); + let current_pos = OwnedRow::new(current_pos_data); + + // Set progress based on is_finished flag + let progress = if is_finished { + LocalityBackfillProgress::Completed { + final_pos: current_pos, + total_rows: row_count, + } + } else { + LocalityBackfillProgress::InProgress { + current_pos, + processed_rows: row_count, + } + }; + + backfill_state.per_vnode.insert(vnode, progress); + total_snapshot_rows += row_count; + } + + backfill_state.total_snapshot_rows = total_snapshot_rows; + Ok(backfill_state) + } + + /// Mark chunk for forwarding based on backfill progress + fn mark_chunk_for_locality( + chunk: StreamChunk, + backfill_state: &LocalityBackfillState, + state_table: &StateTable, + ) -> StreamExecutorResult { + let chunk = chunk.compact(); + let (data, ops) = chunk.into_parts(); + let mut new_visibility = risingwave_common::bitmap::BitmapBuilder::with_capacity(ops.len()); + + let pk_indices = state_table.pk_indices(); + let pk_order = state_table.pk_serde().get_order_types(); + + for (_i, row) in data.rows().enumerate() { + // Project to primary key columns for comparison + let pk = row.project(pk_indices); + let vnode = state_table.compute_vnode_by_pk(pk); + + let visible = match backfill_state.get_progress(&vnode) { + LocalityBackfillProgress::Completed { .. } => true, + LocalityBackfillProgress::NotStarted => false, + LocalityBackfillProgress::InProgress { current_pos, .. } => { + // Compare primary key with current position + cmp_datum_iter( + pk.iter(), + current_pos.iter(), + pk_order.iter().copied(), + ).is_le() + } + }; + + new_visibility.append(visible); } + + let (columns, _) = data.into_parts(); + let chunk = StreamChunk::with_visibility(ops, columns, new_visibility.finish()); + Ok(chunk) } } @@ -185,96 +415,55 @@ impl Execute for LocalityProviderExecutor { impl LocalityProviderExecutor { #[try_stream(ok = Message, error = StreamExecutorError)] - async fn execute_inner(mut self) { + async fn execute_inner(self) { let mut upstream = self.upstream.execute(); - // Wait for first barrier to initialize let first_barrier = expect_first_barrier(&mut upstream).await?; let first_epoch = first_barrier.epoch; // Propagate the first barrier yield Message::Barrier(first_barrier); + + let mut state_table = self.state_table; + let mut progress_table = self.progress_table; // Initialize state tables - self.state_table.init_epoch(first_epoch).await?; - self.progress_table.init_epoch(first_epoch).await?; + state_table.init_epoch(first_epoch).await?; + progress_table.init_epoch(first_epoch).await?; + + // Load backfill state from progress table + let mut backfill_state = Self::load_backfill_state(&progress_table, &self.locality_columns).await?; - // Check progress state using static method to avoid borrowing issues - let (has_progress_state, is_backfill_finished) = - Self::check_backfill_progress(&self.progress_table).await?; + // Get pk info from state table + let pk_indices = state_table.pk_indices().iter().cloned().collect_vec(); - // Determine what to do based on progress state: - // - If no progress state exists: need to buffer chunks (backfill not started) - // - If progress state exists but not finished: backfill in progress, no buffering anymore - // - If progress state exists and finished: pass-through mode (backfill completed) - let need_buffering = !has_progress_state; - let is_completely_finished = has_progress_state && is_backfill_finished; + let is_completely_finished = backfill_state.is_completed(); + let to_backfill = !is_completely_finished; tracing::info!( - has_progress_state = has_progress_state, - is_backfill_finished = is_backfill_finished, - need_buffering = need_buffering, + is_completely_finished = is_completely_finished, + to_backfill = to_backfill, + total_snapshot_rows = backfill_state.total_snapshot_rows, "LocalityProvider initialized" ); - let mut barrier_count = 0; - - if need_buffering { - // Enter buffering phase - buffer data until backfill completion signal - let mut backfill_complete = false; - - #[for_await] - for msg in upstream.by_ref() { - let msg = msg?; - - match msg { - Message::Watermark(_) => { - // Ignore watermarks during backfill - } - Message::Chunk(chunk) => { - self.state_table.write_chunk(chunk); - self.state_table.try_flush().await?; - } - Message::Barrier(barrier) => { - let epoch = barrier.epoch; - - // Commit state tables - let post_commit1 = self.state_table.commit(epoch).await?; - let post_commit2 = self.progress_table.commit(epoch).await?; - - yield Message::Barrier(barrier); - post_commit1.post_yield_barrier(None).await?; - post_commit2.post_yield_barrier(None).await?; - barrier_count += 1; - if barrier_count >= 100 { - break; - } - } - } - } - } - - // TODO: implement backfill loop here - - // Arrangement Backfill Algorithm: + // Locality Provider Backfill Algorithm (adapted from Arrangement Backfill): // // backfill_stream // / \ - // upstream snapshot + // upstream snapshot (from state_table) // - // We construct a backfill stream with upstream as its left input and mv snapshot read - // stream as its right input. When a chunk comes from upstream, we will buffer it. + // We construct a backfill stream with upstream as its left input and locality-ordered + // snapshot read stream as its right input. When a chunk comes from upstream, we buffer it. // // When a barrier comes from upstream: - // Immediately break out of backfill loop. // - For each row of the upstream chunk buffer, compute vnode. - // - Get the `current_pos` corresponding to the vnode. Forward it to downstream if its pk - // <= `current_pos`, otherwise ignore it. - // - Flush all buffered upstream_chunks to replicated state table. - // - Update the `snapshot_read_epoch`. - // - Reconstruct the whole backfill stream with upstream and new mv snapshot read stream - // with the `snapshot_read_epoch`. + // - Get the `current_pos` corresponding to the vnode. Forward it to downstream if its + // locality key <= `current_pos`, otherwise ignore it. + // - Flush all buffered upstream_chunks to state table. + // - Persist backfill progress to progress table. + // - Reconstruct the whole backfill stream with upstream and new snapshot read stream. // // When a chunk comes from snapshot, we forward it to the downstream and raise // `current_pos`. @@ -284,8 +473,167 @@ impl LocalityProviderExecutor { // // Once the backfill loop ends, we forward the upstream directly to the downstream. + if to_backfill { + let mut upstream_chunk_buffer: Vec = vec![]; + let mut pending_barrier: Option = None; + + 'backfill_loop: loop { + let mut cur_barrier_snapshot_processed_rows: u64 = 0; + let mut cur_barrier_upstream_processed_rows: u64 = 0; + let mut snapshot_read_complete = false; + + // Create the backfill stream with upstream and snapshot + { + let left_upstream = upstream.by_ref().map(Either::Left); + let right_snapshot = pin!( + Self::make_snapshot_stream( + &state_table, + backfill_state.clone(), + ) + .map(Either::Right) + ); + + // Prefer to select upstream, so we can stop snapshot stream as soon as the + // barrier comes. + let mut backfill_stream = + select_with_strategy(left_upstream, right_snapshot, |_: &mut ()| { + stream::PollNext::Left + }); + + #[for_await] + for either in &mut backfill_stream { + match either { + // Upstream + Either::Left(msg) => { + match msg? { + Message::Barrier(barrier) => { + // We have to process the barrier outside of the loop. + pending_barrier = Some(barrier); + break; + } + Message::Chunk(chunk) => { + // Buffer the upstream chunk. + upstream_chunk_buffer.push(chunk.compact()); + } + Message::Watermark(_) => { + // Ignore watermark during backfill. + } + } + } + // Snapshot read + Either::Right(msg) => { + match msg? { + None => { + // End of the snapshot read stream. + // Consume remaining rows in the buffer. + for chunk in upstream_chunk_buffer.drain(..) { + let chunk_cardinality = chunk.cardinality() as u64; + cur_barrier_upstream_processed_rows += chunk_cardinality; + yield Message::Chunk(chunk); + } + break 'backfill_loop; + } + Some((vnode, row)) => { + // Extract primary key from row for progress tracking + let pk = row.clone().project(&pk_indices); + + // Convert projected row to OwnedRow for progress tracking + let pk_owned = pk.into_owned_row(); + + // Update progress for this vnode + backfill_state.update_progress( + vnode, + pk_owned, + 1, + ); + + cur_barrier_snapshot_processed_rows += 1; + + // Create chunk with single row + let chunk = StreamChunk::from_rows( + &[(Op::Insert, row)], + &self.input_schema.data_types(), + ); + yield Message::Chunk(chunk); + } + } + } + } + } + } + + // Process barrier + let barrier = match pending_barrier.take() { + Some(barrier) => barrier, + None => break 'backfill_loop, // Reached end of backfill + }; + + // Process upstream buffer chunks with marking + for chunk in upstream_chunk_buffer.drain(..) { + cur_barrier_upstream_processed_rows += chunk.cardinality() as u64; + + // Mark chunk based on backfill progress + if backfill_state.has_progress() { + let marked_chunk = Self::mark_chunk_for_locality( + chunk.clone(), + &backfill_state, + &state_table, + )?; + yield Message::Chunk(marked_chunk); + } + + // Buffer chunk to state table + state_table.write_chunk(chunk); + } + + // Commit state table + let post_commit1 = state_table.commit(barrier.epoch).await?; + + // Persist backfill progress + Self::persist_backfill_state( + &mut progress_table, + &backfill_state, + &self.locality_columns, + ).await?; + let post_commit2 = progress_table.commit(barrier.epoch).await?; + + yield Message::Barrier(barrier); + post_commit1.post_yield_barrier(None).await?; + post_commit2.post_yield_barrier(None).await?; + + // Check if all vnodes are complete + if backfill_state.is_completed() { + break 'backfill_loop; + } + } + } + + tracing::debug!("Locality provider backfill finished, forwarding upstream directly"); + + // Wait for first barrier after backfill completion to mark progress as finished + if to_backfill && !backfill_state.is_completed() { + if let Some(Ok(msg)) = upstream.next().await { + if let Message::Barrier(barrier) = msg { + // Mark all vnodes as completed + for vnode in state_table.vnodes().iter_vnodes() { + backfill_state.finish_vnode(vnode); + } + + // Persist final state + Self::persist_backfill_state( + &mut progress_table, + &backfill_state, + &self.locality_columns, + ).await?; + let post_commit = progress_table.commit(barrier.epoch).await?; + + yield Message::Barrier(barrier); + post_commit.post_yield_barrier(None).await?; + } + } + } - // After backfill completion (or if already completed), forward messages directly + // After backfill completion, forward messages directly #[for_await] for msg in upstream { let msg = msg?; @@ -293,10 +641,10 @@ impl LocalityProviderExecutor { match msg { Message::Barrier(barrier) => { // Commit state tables but don't modify them - self.state_table + state_table .commit_assert_no_update_vnode_bitmap(barrier.epoch) .await?; - self.progress_table + progress_table .commit_assert_no_update_vnode_bitmap(barrier.epoch) .await?; yield Message::Barrier(barrier); From 9b36d47d89f90d26b3ecc83fa52a6a96e22c05b1 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 19 Sep 2025 11:55:56 +0800 Subject: [PATCH 05/36] fix --- src/stream/src/executor/locality_provider.rs | 121 ++++++++----------- 1 file changed, 48 insertions(+), 73 deletions(-) diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index 437f6fdf2720c..fdeedc262f45c 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -112,12 +112,14 @@ impl LocalityBackfillState { self.total_snapshot_rows += row_count_delta; } - fn finish_vnode(&mut self, vnode: VirtualNode) { + fn finish_vnode(&mut self, vnode: VirtualNode, pk_len: usize) { let progress = self.per_vnode.get_mut(&vnode).unwrap(); match progress { LocalityBackfillProgress::NotStarted => { + // Create a final position with pk_len NULL values to indicate completion + let final_pos = OwnedRow::new(vec![None; pk_len]); *progress = LocalityBackfillProgress::Completed { - final_pos: OwnedRow::empty(), + final_pos, total_rows: 0, }; } @@ -150,7 +152,7 @@ impl LocalityBackfillState { /// 2. Forward phase: Once backfill is complete, forward upstream messages directly /// /// Key improvements over the original implementation: -/// - Removes arbitrary 100-barrier buffer limit +/// - Removes arbitrary barrier buffer limit /// - Implements proper upstream chunk tracking during backfill /// - Uses per-vnode progress tracking for better state management pub struct LocalityProviderExecutor { @@ -271,19 +273,8 @@ impl LocalityProviderExecutor { // Build progress row: vnode + current_pos + is_finished + row_count let mut row_data = vec![Some(vnode.to_scalar().into())]; - - // Add current position values - row_data.extend_from_slice(current_pos.as_inner()); - - // Pad with NULLs if position is shorter than locality columns - while row_data.len() < 1 + locality_columns.len() { - row_data.push(None); - } - - // Add is_finished flag + row_data.extend(current_pos); row_data.push(Some(risingwave_common::types::ScalarImpl::Bool(is_finished))); - - // Add row count row_data.push(Some(risingwave_common::types::ScalarImpl::Int64(row_count as i64))); let row = OwnedRow::new(row_data); @@ -302,65 +293,49 @@ impl LocalityProviderExecutor { ); let mut total_snapshot_rows = 0; - // Scan progress table to restore state - let empty_prefix: &[Datum] = &[]; - let iter = progress_table - .iter_with_prefix( - empty_prefix, - &( - std::ops::Bound::<&[Datum]>::Unbounded, - std::ops::Bound::<&[Datum]>::Unbounded, - ), - PrefetchOptions::default(), - ) - .await?; - pin_mut!(iter); - - while let Some(row) = iter.try_next().await? { - if row.len() < 3 { - continue; // Skip malformed rows - } + // For each vnode, try to get its progress state + for vnode in progress_table.vnodes().iter_vnodes() { + // Build key: vnode + NULL values for locality columns (to match progress table schema) + let key_data = vec![Some(vnode.to_scalar().into())]; + + let key = OwnedRow::new(key_data); + + if let Some(row) = progress_table.get_row(&key).await? { + // Parse is_finished flag (second to last column) + let finished_col_idx = row.len() - 2; + let is_finished = row.datum_at(finished_col_idx) + .map(|d| d.into_bool()) + .unwrap_or(false); + + // Parse row count (last column) + let row_count = row.datum_at(row.len() - 1) + .map(|d| d.into_int64() as u64) + .unwrap_or(0); + + // Extract current position (columns 1 to 1+locality_columns.len()) + let pos_end = std::cmp::min(1 + locality_columns.len(), finished_col_idx); + let current_pos_data: Vec = (1..pos_end) + .map(|i| row.datum_at(i).to_owned_datum()) + .collect(); + let current_pos = OwnedRow::new(current_pos_data); + + // Set progress based on is_finished flag + let progress = if is_finished { + LocalityBackfillProgress::Completed { + final_pos: current_pos, + total_rows: row_count, + } + } else { + LocalityBackfillProgress::InProgress { + current_pos, + processed_rows: row_count, + } + }; - // Parse vnode (first column) - let vnode_datum = row.datum_at(0); - if vnode_datum.is_none() { - continue; + backfill_state.per_vnode.insert(vnode, progress); + total_snapshot_rows += row_count; } - let vnode = VirtualNode::from_scalar(vnode_datum.unwrap().into_int16()); - - // Parse is_finished flag (second to last column) - let finished_col_idx = row.len() - 2; - let is_finished = row.datum_at(finished_col_idx) - .map(|d| d.into_bool()) - .unwrap_or(false); - - // Parse row count (last column) - let row_count = row.datum_at(row.len() - 1) - .map(|d| d.into_int64() as u64) - .unwrap_or(0); - - // Extract current position (columns 1 to 1+locality_columns.len()) - let pos_end = std::cmp::min(1 + locality_columns.len(), finished_col_idx); - let current_pos_data: Vec = (1..pos_end) - .map(|i| row.datum_at(i).to_owned_datum()) - .collect(); - let current_pos = OwnedRow::new(current_pos_data); - - // Set progress based on is_finished flag - let progress = if is_finished { - LocalityBackfillProgress::Completed { - final_pos: current_pos, - total_rows: row_count, - } - } else { - LocalityBackfillProgress::InProgress { - current_pos, - processed_rows: row_count, - } - }; - - backfill_state.per_vnode.insert(vnode, progress); - total_snapshot_rows += row_count; + // If no row found, keep the default NotStarted state } backfill_state.total_snapshot_rows = total_snapshot_rows; @@ -616,7 +591,7 @@ impl LocalityProviderExecutor { if let Message::Barrier(barrier) = msg { // Mark all vnodes as completed for vnode in state_table.vnodes().iter_vnodes() { - backfill_state.finish_vnode(vnode); + backfill_state.finish_vnode(vnode, pk_indices.len()); } // Persist final state From 97252b0acb2b9eaf49809d0a7b6f3290a861f155 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 19 Sep 2025 12:03:35 +0800 Subject: [PATCH 06/36] add back buffering stage --- src/stream/src/executor/locality_provider.rs | 56 ++++++++++++++++---- 1 file changed, 45 insertions(+), 11 deletions(-) diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index fdeedc262f45c..4f08e9b09d448 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -413,15 +413,49 @@ impl LocalityProviderExecutor { // Get pk info from state table let pk_indices = state_table.pk_indices().iter().cloned().collect_vec(); - let is_completely_finished = backfill_state.is_completed(); - let to_backfill = !is_completely_finished; - - tracing::info!( - is_completely_finished = is_completely_finished, - to_backfill = to_backfill, - total_snapshot_rows = backfill_state.total_snapshot_rows, - "LocalityProvider initialized" - ); + let need_backfill = ! backfill_state.is_completed(); + + let need_buffering = backfill_state.per_vnode.values().all(|progress| { + matches!(progress, LocalityBackfillProgress::NotStarted) + }); + + // Initial buffering phase before backfill (if needed) + if need_buffering { + // Enter buffering phase - buffer data until we have sufficient data for backfill + let mut barrier_count = 0; + + #[for_await] + for msg in upstream.by_ref() { + let msg = msg?; + + match msg { + Message::Watermark(_) => { + // Ignore watermarks during initial buffering + } + Message::Chunk(chunk) => { + state_table.write_chunk(chunk); + state_table.try_flush().await?; + } + Message::Barrier(barrier) => { + let epoch = barrier.epoch; + + // Commit state tables + let post_commit1 = state_table.commit(epoch).await?; + let post_commit2 = progress_table.commit(epoch).await?; + + yield Message::Barrier(barrier); + post_commit1.post_yield_barrier(None).await?; + post_commit2.post_yield_barrier(None).await?; + + barrier_count += 1; + // Start backfill after buffering some data + if barrier_count >= 10 { + break; + } + } + } + } + } // Locality Provider Backfill Algorithm (adapted from Arrangement Backfill): // @@ -448,7 +482,7 @@ impl LocalityProviderExecutor { // // Once the backfill loop ends, we forward the upstream directly to the downstream. - if to_backfill { + if need_backfill { let mut upstream_chunk_buffer: Vec = vec![]; let mut pending_barrier: Option = None; @@ -586,7 +620,7 @@ impl LocalityProviderExecutor { tracing::debug!("Locality provider backfill finished, forwarding upstream directly"); // Wait for first barrier after backfill completion to mark progress as finished - if to_backfill && !backfill_state.is_completed() { + if need_backfill && !backfill_state.is_completed() { if let Some(Ok(msg)) = upstream.next().await { if let Message::Barrier(barrier) = msg { // Mark all vnodes as completed From dc18e6c0d4defc818ae0893ae6b56edb1d61b6ee Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 19 Sep 2025 12:20:26 +0800 Subject: [PATCH 07/36] fix --- src/stream/src/executor/locality_provider.rs | 64 ++++++++++++-------- 1 file changed, 39 insertions(+), 25 deletions(-) diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index 4f08e9b09d448..876140d9a6a02 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -343,7 +343,7 @@ impl LocalityProviderExecutor { } /// Mark chunk for forwarding based on backfill progress - fn mark_chunk_for_locality( + fn mark_chunk( chunk: StreamChunk, backfill_state: &LocalityBackfillState, state_table: &StateTable, @@ -583,20 +583,19 @@ impl LocalityProviderExecutor { // Mark chunk based on backfill progress if backfill_state.has_progress() { - let marked_chunk = Self::mark_chunk_for_locality( + let marked_chunk = Self::mark_chunk( chunk.clone(), &backfill_state, &state_table, )?; yield Message::Chunk(marked_chunk); } - - // Buffer chunk to state table - state_table.write_chunk(chunk); } - // Commit state table - let post_commit1 = state_table.commit(barrier.epoch).await?; + // no-op commit state table + state_table + .commit_assert_no_update_vnode_bitmap(barrier.epoch) + .await?; // Persist backfill progress Self::persist_backfill_state( @@ -604,11 +603,10 @@ impl LocalityProviderExecutor { &backfill_state, &self.locality_columns, ).await?; - let post_commit2 = progress_table.commit(barrier.epoch).await?; + let post_commit = progress_table.commit(barrier.epoch).await?; yield Message::Barrier(barrier); - post_commit1.post_yield_barrier(None).await?; - post_commit2.post_yield_barrier(None).await?; + post_commit.post_yield_barrier(None).await?; // Check if all vnodes are complete if backfill_state.is_completed() { @@ -621,23 +619,39 @@ impl LocalityProviderExecutor { // Wait for first barrier after backfill completion to mark progress as finished if need_backfill && !backfill_state.is_completed() { - if let Some(Ok(msg)) = upstream.next().await { - if let Message::Barrier(barrier) = msg { - // Mark all vnodes as completed - for vnode in state_table.vnodes().iter_vnodes() { - backfill_state.finish_vnode(vnode, pk_indices.len()); - } + while let Some(Ok(msg)) = upstream.next().await { + match msg { + Message::Barrier(barrier) => { + // no-op commit state table + state_table + .commit_assert_no_update_vnode_bitmap(barrier.epoch) + .await?; + + // Mark all vnodes as completed + for vnode in state_table.vnodes().iter_vnodes() { + backfill_state.finish_vnode(vnode, pk_indices.len()); + } - // Persist final state - Self::persist_backfill_state( - &mut progress_table, - &backfill_state, - &self.locality_columns, - ).await?; - let post_commit = progress_table.commit(barrier.epoch).await?; + // Persist final state + Self::persist_backfill_state( + &mut progress_table, + &backfill_state, + &self.locality_columns, + ).await?; + let post_commit = progress_table.commit(barrier.epoch).await?; - yield Message::Barrier(barrier); - post_commit.post_yield_barrier(None).await?; + yield Message::Barrier(barrier); + post_commit.post_yield_barrier(None).await?; + break; // Exit the loop after processing the barrier + } + Message::Chunk(chunk) => { + // Forward chunks directly during completion phase + yield Message::Chunk(chunk); + } + Message::Watermark(watermark) => { + // Forward watermarks directly during completion phase + yield Message::Watermark(watermark); + } } } } From 027d22f29c8216de87da68a1c9778aa1da7c83ec Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 19 Sep 2025 13:55:07 +0800 Subject: [PATCH 08/36] fmt --- src/stream/src/executor/locality_provider.rs | 90 +++++++++----------- 1 file changed, 39 insertions(+), 51 deletions(-) diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index 876140d9a6a02..1170237771f99 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -15,10 +15,10 @@ use std::collections::HashMap; use std::sync::Arc; -use futures_async_stream::try_stream; use either::Either; use futures::stream::select_with_strategy; use futures::{TryStreamExt, pin_mut, stream}; +use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::catalog::Schema; @@ -74,23 +74,18 @@ impl LocalityBackfillState { } fn is_completed(&self) -> bool { - self.per_vnode.values().all(|progress| { - matches!(progress, LocalityBackfillProgress::Completed { .. }) - }) + self.per_vnode + .values() + .all(|progress| matches!(progress, LocalityBackfillProgress::Completed { .. })) } fn has_progress(&self) -> bool { - self.per_vnode.values().any(|progress| { - matches!(progress, LocalityBackfillProgress::InProgress { .. }) - }) + self.per_vnode + .values() + .any(|progress| matches!(progress, LocalityBackfillProgress::InProgress { .. })) } - fn update_progress( - &mut self, - vnode: VirtualNode, - new_pos: OwnedRow, - row_count_delta: u64, - ) { + fn update_progress(&mut self, vnode: VirtualNode, new_pos: OwnedRow, row_count_delta: u64) { let progress = self.per_vnode.get_mut(&vnode).unwrap(); match progress { LocalityBackfillProgress::NotStarted => { @@ -143,7 +138,6 @@ impl LocalityBackfillState { } } - /// The `LocalityProviderExecutor` provides locality for operators during backfilling. /// It buffers input data into a state table using locality columns as primary key prefix. /// @@ -176,7 +170,6 @@ pub struct LocalityProviderExecutor { /// Chunk size for output chunk_size: usize, - } impl LocalityProviderExecutor { @@ -274,8 +267,12 @@ impl LocalityProviderExecutor { // Build progress row: vnode + current_pos + is_finished + row_count let mut row_data = vec![Some(vnode.to_scalar().into())]; row_data.extend(current_pos); - row_data.push(Some(risingwave_common::types::ScalarImpl::Bool(is_finished))); - row_data.push(Some(risingwave_common::types::ScalarImpl::Int64(row_count as i64))); + row_data.push(Some(risingwave_common::types::ScalarImpl::Bool( + is_finished, + ))); + row_data.push(Some(risingwave_common::types::ScalarImpl::Int64( + row_count as i64, + ))); let row = OwnedRow::new(row_data); progress_table.insert(row); @@ -288,9 +285,7 @@ impl LocalityProviderExecutor { progress_table: &StateTable, locality_columns: &[usize], ) -> StreamExecutorResult { - let mut backfill_state = LocalityBackfillState::new( - progress_table.vnodes().iter_vnodes() - ); + let mut backfill_state = LocalityBackfillState::new(progress_table.vnodes().iter_vnodes()); let mut total_snapshot_rows = 0; // For each vnode, try to get its progress state @@ -303,12 +298,14 @@ impl LocalityProviderExecutor { if let Some(row) = progress_table.get_row(&key).await? { // Parse is_finished flag (second to last column) let finished_col_idx = row.len() - 2; - let is_finished = row.datum_at(finished_col_idx) + let is_finished = row + .datum_at(finished_col_idx) .map(|d| d.into_bool()) .unwrap_or(false); // Parse row count (last column) - let row_count = row.datum_at(row.len() - 1) + let row_count = row + .datum_at(row.len() - 1) .map(|d| d.into_int64() as u64) .unwrap_or(0); @@ -365,11 +362,7 @@ impl LocalityProviderExecutor { LocalityBackfillProgress::NotStarted => false, LocalityBackfillProgress::InProgress { current_pos, .. } => { // Compare primary key with current position - cmp_datum_iter( - pk.iter(), - current_pos.iter(), - pk_order.iter().copied(), - ).is_le() + cmp_datum_iter(pk.iter(), current_pos.iter(), pk_order.iter().copied()).is_le() } }; @@ -399,7 +392,7 @@ impl LocalityProviderExecutor { // Propagate the first barrier yield Message::Barrier(first_barrier); - + let mut state_table = self.state_table; let mut progress_table = self.progress_table; @@ -408,16 +401,18 @@ impl LocalityProviderExecutor { progress_table.init_epoch(first_epoch).await?; // Load backfill state from progress table - let mut backfill_state = Self::load_backfill_state(&progress_table, &self.locality_columns).await?; + let mut backfill_state = + Self::load_backfill_state(&progress_table, &self.locality_columns).await?; // Get pk info from state table let pk_indices = state_table.pk_indices().iter().cloned().collect_vec(); - let need_backfill = ! backfill_state.is_completed(); + let need_backfill = !backfill_state.is_completed(); - let need_buffering = backfill_state.per_vnode.values().all(|progress| { - matches!(progress, LocalityBackfillProgress::NotStarted) - }); + let need_buffering = backfill_state + .per_vnode + .values() + .all(|progress| matches!(progress, LocalityBackfillProgress::NotStarted)); // Initial buffering phase before backfill (if needed) if need_buffering { @@ -495,11 +490,8 @@ impl LocalityProviderExecutor { { let left_upstream = upstream.by_ref().map(Either::Left); let right_snapshot = pin!( - Self::make_snapshot_stream( - &state_table, - backfill_state.clone(), - ) - .map(Either::Right) + Self::make_snapshot_stream(&state_table, backfill_state.clone(),) + .map(Either::Right) ); // Prefer to select upstream, so we can stop snapshot stream as soon as the @@ -537,7 +529,8 @@ impl LocalityProviderExecutor { // Consume remaining rows in the buffer. for chunk in upstream_chunk_buffer.drain(..) { let chunk_cardinality = chunk.cardinality() as u64; - cur_barrier_upstream_processed_rows += chunk_cardinality; + cur_barrier_upstream_processed_rows += + chunk_cardinality; yield Message::Chunk(chunk); } break 'backfill_loop; @@ -550,11 +543,7 @@ impl LocalityProviderExecutor { let pk_owned = pk.into_owned_row(); // Update progress for this vnode - backfill_state.update_progress( - vnode, - pk_owned, - 1, - ); + backfill_state.update_progress(vnode, pk_owned, 1); cur_barrier_snapshot_processed_rows += 1; @@ -583,11 +572,8 @@ impl LocalityProviderExecutor { // Mark chunk based on backfill progress if backfill_state.has_progress() { - let marked_chunk = Self::mark_chunk( - chunk.clone(), - &backfill_state, - &state_table, - )?; + let marked_chunk = + Self::mark_chunk(chunk.clone(), &backfill_state, &state_table)?; yield Message::Chunk(marked_chunk); } } @@ -602,7 +588,8 @@ impl LocalityProviderExecutor { &mut progress_table, &backfill_state, &self.locality_columns, - ).await?; + ) + .await?; let post_commit = progress_table.commit(barrier.epoch).await?; yield Message::Barrier(barrier); @@ -637,7 +624,8 @@ impl LocalityProviderExecutor { &mut progress_table, &backfill_state, &self.locality_columns, - ).await?; + ) + .await?; let post_commit = progress_table.commit(barrier.epoch).await?; yield Message::Barrier(barrier); From f52a6607d59a84b7cd0ee668cee9f243b58955e2 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 19 Sep 2025 14:50:56 +0800 Subject: [PATCH 09/36] change locality provider distribution to upstreamhash and ensure its input has an shuffle. It is used to ensure locality provider is in its own fragment --- .../plan_node/logical_locality_provider.rs | 14 +++++++++++++- .../plan_node/stream_locality_provider.rs | 17 ++++++++++++++++- 2 files changed, 29 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs b/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs index f9db3b04c41ed..46859b2d3e35a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs +++ b/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs @@ -18,7 +18,8 @@ use super::generic::GenericPlanRef; use super::utils::impl_distill_by_unit; use super::{ BatchPlanRef, ColPrunable, ExprRewritable, Logical, LogicalPlanRef as PlanRef, LogicalProject, - PlanBase, PlanTreeNodeUnary, PredicatePushdown, StreamPlanRef, ToBatch, ToStream, generic, + PlanBase, PlanTreeNodeUnary, PredicatePushdown, StreamExchange, StreamPlanRef, ToBatch, + ToStream, generic, }; use crate::error::Result; use crate::expr::{ExprRewriter, ExprVisitor}; @@ -26,6 +27,7 @@ use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; +use crate::optimizer::property::RequiredDist; use crate::utils::{ColIndexMapping, Condition}; /// `LogicalLocalityProvider` provides locality for operators during backfilling. @@ -120,7 +122,17 @@ impl ToBatch for LogicalLocalityProvider { impl ToStream for LogicalLocalityProvider { fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { use super::StreamLocalityProvider; + let input = self.input().to_stream(ctx)?; + let required_dist = + RequiredDist::shard_by_key(self.input().schema().len(), self.locality_columns()); + let input = required_dist.streaming_enforce_if_not_satisfies(input)?; + let input = if input.as_stream_exchange().is_none() { + // force a no shuffle exchange to ensure locality provider is in its own fragment + StreamExchange::new_no_shuffle(input).into() + } else { + input + }; let stream_core = generic::LocalityProvider::new(input, self.locality_columns().to_vec()); Ok(StreamLocalityProvider::new(stream_core).into()) } diff --git a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs index 4f1bb35634fbf..ac5082159044e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs +++ b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs @@ -24,9 +24,11 @@ use super::stream::prelude::*; use super::utils::{TableCatalogBuilder, impl_distill_by_unit}; use super::{ExprRewritable, PlanTreeNodeUnary, StreamNode, StreamPlanRef as PlanRef, generic}; use crate::TableCatalog; +use crate::catalog::TableId; use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::PlanBase; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamLocalityProvider` implements [`super::LogicalLocalityProvider`] @@ -39,7 +41,20 @@ pub struct StreamLocalityProvider { impl StreamLocalityProvider { pub fn new(core: generic::LocalityProvider) -> Self { let input = core.input.clone(); - let dist = input.distribution().clone(); + + let dist = match input.distribution() { + Distribution::HashShard(keys) => { + // If the input is hash-distributed, we make it a UpstreamHashShard distribution + // just like a normal table scan. It is used to ensure locality provider is in its own fragment + Distribution::UpstreamHashShard(keys.clone(), TableId::placeholder()) + } + Distribution::UpstreamHashShard(keys, table_id) => { + Distribution::UpstreamHashShard(keys.clone(), *table_id) + } + _ => { + panic!("LocalityProvider input must be hash-distributed"); + } + }; // LocalityProvider maintains the append-only behavior if input is append-only let base = PlanBase::new_stream_with_core( From 2c4bb952fa01b124cfd02562d671f71c9d4ab245 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 19 Sep 2025 16:27:03 +0800 Subject: [PATCH 10/36] add fragment dependency --- .../plan_node/stream_locality_provider.rs | 4 +- src/meta/src/stream/stream_graph/fragment.rs | 126 ++++++++++++++++++ src/stream/src/executor/locality_provider.rs | 4 +- 3 files changed, 128 insertions(+), 6 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs index ac5082159044e..c8ceff25df3b7 100644 --- a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs +++ b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs @@ -142,9 +142,7 @@ impl StreamLocalityProvider { // add streaming key of the input as the rest of the primary key if let Some(stream_key) = input.stream_key() { for &key_col_idx in stream_key { - if !self.locality_columns().contains(&key_col_idx) { - catalog_builder.add_order_column(key_col_idx, OrderType::ascending()); - } + catalog_builder.add_order_column(key_col_idx, OrderType::ascending()); } } diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index 5c218b1fe2ee1..04bea581790cc 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -1020,6 +1020,8 @@ impl StreamFragmentGraph { pub fn create_fragment_backfill_ordering(&self) -> FragmentBackfillOrder { let mapping = self.collect_backfill_mapping(); let mut fragment_ordering: HashMap> = HashMap::new(); + + // 1. Add backfill dependencies for (rel_id, downstream_rel_ids) in &self.backfill_order.order { let fragment_ids = mapping.get(rel_id).unwrap(); for fragment_id in fragment_ids { @@ -1032,8 +1034,132 @@ impl StreamFragmentGraph { fragment_ordering.insert(*fragment_id, downstream_fragment_ids); } } + // println!("{:?}", fragment_ordering); + // + // // 2. Add dependencies: all backfill fragments should run before LocalityProvider fragments + // let locality_provider_dependencies = self.find_locality_provider_dependencies(); + // println!("{:?}", locality_provider_dependencies); + // + // let backfill_fragments: HashSet = mapping.values().flatten().copied().collect(); + // let locality_provider_root_fragments: Vec = locality_provider_dependencies.keys().copied().collect(); + // + // // For each backfill fragment, add only the root LocalityProvider fragments as dependents + // for &backfill_fragment_id in &backfill_fragments { + // fragment_ordering + // .entry(backfill_fragment_id) + // .or_default() + // .extend(locality_provider_root_fragments.iter().copied()); + // } + // + // // 3. Add LocalityProvider internal dependencies + // for (fragment_id, downstream_fragments) in locality_provider_dependencies { + // fragment_ordering + // .entry(fragment_id) + // .or_default() + // .extend(downstream_fragments); + // } + fragment_ordering } + + /// Find dependency relationships among fragments containing LocalityProvider nodes. + /// Returns a mapping where each fragment ID maps to a list of fragment IDs that should be processed after it. + /// Following the same semantics as FragmentBackfillOrder: + /// `G[10] -> [1, 2, 11]` means LocalityProvider in fragment 10 should be processed + /// before LocalityProviders in fragments 1, 2, and 11. + /// + /// This method assumes each fragment contains at most one LocalityProvider node. + pub fn find_locality_provider_dependencies(&self) -> HashMap> { + let mut locality_provider_fragments = HashSet::new(); + let mut dependencies: HashMap> = HashMap::new(); + + // First, identify all fragments that contain LocalityProvider nodes + for (fragment_id, fragment) in &self.fragments { + let fragment_id = fragment_id.as_global_id(); + let has_locality_provider = self.fragment_has_locality_provider(fragment); + + if has_locality_provider { + locality_provider_fragments.insert(fragment_id); + dependencies.entry(fragment_id).or_default(); + } + } + + // Build dependency relationships between LocalityProvider fragments + // For each LocalityProvider fragment, find all downstream LocalityProvider fragments + // The upstream fragment should be processed before the downstream fragments + for &provider_fragment_id in &locality_provider_fragments { + let provider_fragment_global_id = GlobalFragmentId::new(provider_fragment_id); + + // Find all fragments downstream from this LocalityProvider fragment + let mut visited = HashSet::new(); + let mut downstream_locality_providers = Vec::new(); + + self.collect_downstream_locality_providers( + provider_fragment_global_id, + &locality_provider_fragments, + &mut visited, + &mut downstream_locality_providers, + ); + + // This fragment should be processed before all its downstream LocalityProvider fragments + dependencies + .entry(provider_fragment_id) + .or_default() + .extend(downstream_locality_providers); + } + + dependencies + } + + /// Check if a fragment contains a LocalityProvider node + fn fragment_has_locality_provider(&self, fragment: &BuildingFragment) -> bool { + let mut has_locality_provider = false; + + if let Some(node) = fragment.node.as_ref() { + visit_stream_node_cont(node, |stream_node| { + if let Some(NodeBody::LocalityProvider(_)) = stream_node.node_body.as_ref() { + has_locality_provider = true; + false // Stop visiting once we find a LocalityProvider + } else { + true // Continue visiting + } + }); + } + + has_locality_provider + } + + /// Recursively collect downstream LocalityProvider fragments + fn collect_downstream_locality_providers( + &self, + current_fragment_id: GlobalFragmentId, + locality_provider_fragments: &HashSet, + visited: &mut HashSet, + downstream_providers: &mut Vec, + ) { + if visited.contains(¤t_fragment_id) { + return; + } + visited.insert(current_fragment_id); + + // Check all downstream fragments + for (&downstream_id, _edge) in self.get_downstreams(current_fragment_id) { + let downstream_fragment_id = downstream_id.as_global_id(); + + // If the downstream fragment is a LocalityProvider, add it to results + if locality_provider_fragments.contains(&downstream_fragment_id) { + downstream_providers.push(downstream_fragment_id); + } + + // Recursively check further downstream + self.collect_downstream_locality_providers( + downstream_id, + locality_provider_fragments, + visited, + downstream_providers, + ); + } + } } /// Fill snapshot epoch for `StreamScanNode` of `SnapshotBackfill`. diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index 1170237771f99..afc7e34cd5f33 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -309,9 +309,7 @@ impl LocalityProviderExecutor { .map(|d| d.into_int64() as u64) .unwrap_or(0); - // Extract current position (columns 1 to 1+locality_columns.len()) - let pos_end = std::cmp::min(1 + locality_columns.len(), finished_col_idx); - let current_pos_data: Vec = (1..pos_end) + let current_pos_data: Vec = (1..finished_col_idx) .map(|i| row.datum_at(i).to_owned_datum()) .collect(); let current_pos = OwnedRow::new(current_pos_data); From 2f50b5961d3f544530db8fd8ce9dcf65f5cdcbce Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 19 Sep 2025 16:50:31 +0800 Subject: [PATCH 11/36] report backfill progress --- src/stream/src/executor/locality_provider.rs | 66 +++++++++++++++++-- .../src/from_proto/locality_provider.rs | 6 ++ 2 files changed, 67 insertions(+), 5 deletions(-) diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index afc7e34cd5f33..6f0a53f59c861 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -31,6 +31,7 @@ use risingwave_storage::store::PrefetchOptions; use crate::common::table::state_table::StateTable; use crate::executor::prelude::*; +use crate::task::{CreateMviewProgressReporter, FragmentId}; /// Progress state for tracking backfill per vnode #[derive(Clone, Debug, PartialEq, Eq)] @@ -79,6 +80,10 @@ impl LocalityBackfillState { .all(|progress| matches!(progress, LocalityBackfillProgress::Completed { .. })) } + fn vnodes(&self) -> impl Iterator { + self.per_vnode.iter().map(|(&vnode, progress)| (vnode, progress)) + } + fn has_progress(&self) -> bool { self.per_vnode .values() @@ -165,11 +170,20 @@ pub struct LocalityProviderExecutor { /// Schema of the input input_schema: Schema, + /// Progress reporter for materialized view creation + progress: CreateMviewProgressReporter, + + /// Actor ID for this executor + actor_id: ActorId, + /// Metrics metrics: Arc, /// Chunk size for output chunk_size: usize, + + /// Fragment ID of the fragment this LocalityProvider belongs to + fragment_id: FragmentId, } impl LocalityProviderExecutor { @@ -180,8 +194,10 @@ impl LocalityProviderExecutor { state_table: StateTable, progress_table: StateTable, input_schema: Schema, + progress: CreateMviewProgressReporter, metrics: Arc, chunk_size: usize, + fragment_id: FragmentId, ) -> Self { Self { upstream, @@ -189,8 +205,11 @@ impl LocalityProviderExecutor { state_table, progress_table, input_schema, + actor_id: progress.actor_id(), + progress, metrics, chunk_size, + fragment_id, } } @@ -249,7 +268,7 @@ impl LocalityProviderExecutor { async fn persist_backfill_state( progress_table: &mut StateTable, backfill_state: &LocalityBackfillState, - locality_columns: &[usize], + _locality_columns: &[usize], ) -> StreamExecutorResult<()> { for (vnode, progress) in &backfill_state.per_vnode { let (is_finished, current_pos, row_count) = match progress { @@ -283,7 +302,7 @@ impl LocalityProviderExecutor { /// Load backfill state from progress table async fn load_backfill_state( progress_table: &StateTable, - locality_columns: &[usize], + _locality_columns: &[usize], ) -> StreamExecutorResult { let mut backfill_state = LocalityBackfillState::new(progress_table.vnodes().iter_vnodes()); let mut total_snapshot_rows = 0; @@ -381,7 +400,7 @@ impl Execute for LocalityProviderExecutor { impl LocalityProviderExecutor { #[try_stream(ok = Message, error = StreamExecutorError)] - async fn execute_inner(self) { + async fn execute_inner(mut self) { let mut upstream = self.upstream.execute(); // Wait for first barrier to initialize @@ -482,7 +501,7 @@ impl LocalityProviderExecutor { 'backfill_loop: loop { let mut cur_barrier_snapshot_processed_rows: u64 = 0; let mut cur_barrier_upstream_processed_rows: u64 = 0; - let mut snapshot_read_complete = false; + let _snapshot_read_complete = false; // Create the backfill stream with upstream and snapshot { @@ -581,6 +600,21 @@ impl LocalityProviderExecutor { .commit_assert_no_update_vnode_bitmap(barrier.epoch) .await?; + // Update progress with current epoch and snapshot read count + let total_snapshot_processed_rows: u64 = backfill_state.vnodes().map(|(_, progress)| { + match progress { + &LocalityBackfillProgress::InProgress { processed_rows, .. } => processed_rows, + &LocalityBackfillProgress::Completed { total_rows, .. } => total_rows, + &LocalityBackfillProgress::NotStarted => 0, + } + }).sum(); + + self.progress.update( + barrier.epoch, + barrier.epoch.curr, // Use barrier epoch as snapshot read epoch + total_snapshot_processed_rows, + ); + // Persist backfill progress Self::persist_backfill_state( &mut progress_table, @@ -588,13 +622,23 @@ impl LocalityProviderExecutor { &self.locality_columns, ) .await?; - let post_commit = progress_table.commit(barrier.epoch).await?; + let barrier_epoch = barrier.epoch; + let post_commit = progress_table.commit(barrier_epoch).await?; yield Message::Barrier(barrier); post_commit.post_yield_barrier(None).await?; // Check if all vnodes are complete if backfill_state.is_completed() { + // Backfill is complete, finish progress reporting + let total_snapshot_processed_rows: u64 = backfill_state.vnodes().map(|(_, progress)| { + match progress { + &LocalityBackfillProgress::Completed { total_rows, .. } => total_rows, + _ => 0, // Should all be completed at this point + } + }).sum(); + + self.progress.finish(barrier_epoch, total_snapshot_processed_rows); break 'backfill_loop; } } @@ -617,6 +661,18 @@ impl LocalityProviderExecutor { backfill_state.finish_vnode(vnode, pk_indices.len()); } + // Calculate final total processed rows + let total_snapshot_processed_rows: u64 = backfill_state.vnodes().map(|(_, progress)| { + match progress { + &LocalityBackfillProgress::Completed { total_rows, .. } => total_rows, + &LocalityBackfillProgress::InProgress { processed_rows, .. } => processed_rows, + &LocalityBackfillProgress::NotStarted => 0, + } + }).sum(); + + // Finish progress reporting + self.progress.finish(barrier.epoch, total_snapshot_processed_rows); + // Persist final state Self::persist_backfill_state( &mut progress_table, diff --git a/src/stream/src/from_proto/locality_provider.rs b/src/stream/src/from_proto/locality_provider.rs index ef02a262d2961..e9a30a4a86255 100644 --- a/src/stream/src/from_proto/locality_provider.rs +++ b/src/stream/src/from_proto/locality_provider.rs @@ -65,14 +65,20 @@ impl ExecutorBuilder for LocalityProviderBuilder { .build() .await; + let progress = params + .local_barrier_manager + .register_create_mview_progress(params.actor_context.id); + let exec = LocalityProviderExecutor::new( input, locality_columns, state_table, progress_table, input_schema, + progress, params.executor_stats.clone(), 1024, // default chunk size + params.actor_context.fragment_id, ); Ok((params.info, exec).into()) From 124e8c82605e79db25d38bbc3aced469a0c458e3 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 19 Sep 2025 17:12:49 +0800 Subject: [PATCH 12/36] backfill track locality provider --- src/common/src/catalog/mod.rs | 3 ++- src/frontend/src/stream_fragmenter/mod.rs | 6 ++++++ src/meta/src/barrier/progress.rs | 5 +++++ src/meta/src/model/stream.rs | 9 +++++++-- 4 files changed, 20 insertions(+), 3 deletions(-) diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index deda6f60712c3..3190d1ab8470d 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -637,7 +637,8 @@ macro_rules! for_all_fragment_type_flags { CrossDbSnapshotBackfillStreamScan, StreamCdcScan, VectorIndexWrite, - UpstreamSinkUnion + UpstreamSinkUnion, + LocalityProvider }, {}, 0 diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index acf85856c8c93..58055249ad2a2 100644 --- a/src/frontend/src/stream_fragmenter/mod.rs +++ b/src/frontend/src/stream_fragmenter/mod.rs @@ -482,6 +482,12 @@ fn build_fragment( .add(FragmentTypeFlag::UpstreamSinkUnion); } + NodeBody::LocalityProvider(_) => { + current_fragment + .fragment_type_mask + .add(FragmentTypeFlag::LocalityProvider); + } + _ => {} }; diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index 76018c3e84eac..9ef0814932bf5 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -149,6 +149,10 @@ impl Progress { BackfillUpstreamType::Values => { // do not consider progress for values } + BackfillUpstreamType::LocalityProvider => { + // Track LocalityProvider progress similar to MView + self.mv_backfill_consumed_rows += new - old; + } } self.states.insert(actor, new_state); next_backfill_nodes @@ -183,6 +187,7 @@ impl Progress { BackfillUpstreamType::MView => mv_count += 1, BackfillUpstreamType::Source => source_count += 1, BackfillUpstreamType::Values => (), + BackfillUpstreamType::LocalityProvider => mv_count += 1, // Count LocalityProvider as an MView for progress } } diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index e8735a0d708e4..e416cde7ebcba 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -503,6 +503,7 @@ impl StreamJobFragments { FragmentTypeFlag::Values, FragmentTypeFlag::StreamScan, FragmentTypeFlag::SourceScan, + FragmentTypeFlag::LocalityProvider, ]) { actor_ids.extend(fragment.actors.iter().map(|actor| { ( @@ -782,6 +783,7 @@ pub enum BackfillUpstreamType { MView, Values, Source, + LocalityProvider, } impl BackfillUpstreamType { @@ -789,12 +791,13 @@ impl BackfillUpstreamType { let is_mview = mask.contains(FragmentTypeFlag::StreamScan); let is_values = mask.contains(FragmentTypeFlag::Values); let is_source = mask.contains(FragmentTypeFlag::SourceScan); + let is_locality_provider = mask.contains(FragmentTypeFlag::LocalityProvider); // Note: in theory we can have multiple backfill executors in one fragment, but currently it's not possible. // See . debug_assert!( - is_mview as u8 + is_values as u8 + is_source as u8 == 1, - "a backfill fragment should either be mview, value or source, found {:?}", + is_mview as u8 + is_values as u8 + is_source as u8 + is_locality_provider as u8 == 1, + "a backfill fragment should either be mview, value, source, or locality provider, found {:?}", mask ); @@ -804,6 +807,8 @@ impl BackfillUpstreamType { BackfillUpstreamType::Values } else if is_source { BackfillUpstreamType::Source + } else if is_locality_provider { + BackfillUpstreamType::LocalityProvider } else { unreachable!("invalid fragment type mask: {:?}", mask); } From c806a4df01a6755225b404421e4c8c3ea933acca Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 19 Sep 2025 17:38:22 +0800 Subject: [PATCH 13/36] add backfill ordering for locality provider --- .../src/barrier/backfill_order_control.rs | 2 +- src/meta/src/stream/stream_graph/fragment.rs | 48 +++++++++---------- src/stream/src/executor/locality_provider.rs | 21 +++++--- 3 files changed, 40 insertions(+), 31 deletions(-) diff --git a/src/meta/src/barrier/backfill_order_control.rs b/src/meta/src/barrier/backfill_order_control.rs index a9c0ddc006f1f..0154ebdd3a289 100644 --- a/src/meta/src/barrier/backfill_order_control.rs +++ b/src/meta/src/barrier/backfill_order_control.rs @@ -69,7 +69,7 @@ impl BackfillOrderState { for fragment in stream_job_fragments.fragments() { if fragment .fragment_type_mask - .contains_any([FragmentTypeFlag::StreamScan, FragmentTypeFlag::SourceScan]) + .contains_any([FragmentTypeFlag::StreamScan, FragmentTypeFlag::SourceScan, FragmentTypeFlag::LocalityProvider]) { let fragment_id = fragment.fragment_id; backfill_nodes.insert( diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index 04bea581790cc..54e700df9f19d 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -1034,30 +1034,30 @@ impl StreamFragmentGraph { fragment_ordering.insert(*fragment_id, downstream_fragment_ids); } } - // println!("{:?}", fragment_ordering); - // - // // 2. Add dependencies: all backfill fragments should run before LocalityProvider fragments - // let locality_provider_dependencies = self.find_locality_provider_dependencies(); - // println!("{:?}", locality_provider_dependencies); - // - // let backfill_fragments: HashSet = mapping.values().flatten().copied().collect(); - // let locality_provider_root_fragments: Vec = locality_provider_dependencies.keys().copied().collect(); - // - // // For each backfill fragment, add only the root LocalityProvider fragments as dependents - // for &backfill_fragment_id in &backfill_fragments { - // fragment_ordering - // .entry(backfill_fragment_id) - // .or_default() - // .extend(locality_provider_root_fragments.iter().copied()); - // } - // - // // 3. Add LocalityProvider internal dependencies - // for (fragment_id, downstream_fragments) in locality_provider_dependencies { - // fragment_ordering - // .entry(fragment_id) - // .or_default() - // .extend(downstream_fragments); - // } + println!("{:?}", fragment_ordering); + + // 2. Add dependencies: all backfill fragments should run before LocalityProvider fragments + let locality_provider_dependencies = self.find_locality_provider_dependencies(); + println!("{:?}", locality_provider_dependencies); + + let backfill_fragments: HashSet = mapping.values().flatten().copied().collect(); + let locality_provider_root_fragments: Vec = locality_provider_dependencies.keys().copied().collect(); + + // For each backfill fragment, add only the root LocalityProvider fragments as dependents + for &backfill_fragment_id in &backfill_fragments { + fragment_ordering + .entry(backfill_fragment_id) + .or_default() + .extend(locality_provider_root_fragments.iter().copied()); + } + + // 3. Add LocalityProvider internal dependencies + for (fragment_id, downstream_fragments) in locality_provider_dependencies { + fragment_ordering + .entry(fragment_id) + .or_default() + .extend(downstream_fragments); + } fragment_ordering } diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index 6f0a53f59c861..9ca8faf7bf955 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -431,10 +431,10 @@ impl LocalityProviderExecutor { .values() .all(|progress| matches!(progress, LocalityBackfillProgress::NotStarted)); - // Initial buffering phase before backfill (if needed) + // Initial buffering phase before backfill - wait for StartFragmentBackfill mutation (if needed) if need_buffering { - // Enter buffering phase - buffer data until we have sufficient data for backfill - let mut barrier_count = 0; + // Enter buffering phase - buffer data until StartFragmentBackfill is received + let mut start_backfill = false; #[for_await] for msg in upstream.by_ref() { @@ -451,6 +451,16 @@ impl LocalityProviderExecutor { Message::Barrier(barrier) => { let epoch = barrier.epoch; + // Check for StartFragmentBackfill mutation + if let Some(mutation) = barrier.mutation.as_deref() { + use crate::executor::Mutation; + if let Mutation::StartFragmentBackfill { fragment_ids } = mutation { + if fragment_ids.contains(&self.fragment_id) { + start_backfill = true; + } + } + } + // Commit state tables let post_commit1 = state_table.commit(epoch).await?; let post_commit2 = progress_table.commit(epoch).await?; @@ -459,9 +469,8 @@ impl LocalityProviderExecutor { post_commit1.post_yield_barrier(None).await?; post_commit2.post_yield_barrier(None).await?; - barrier_count += 1; - // Start backfill after buffering some data - if barrier_count >= 10 { + // Start backfill when StartFragmentBackfill mutation is received + if start_backfill { break; } } From 3d5fc58cc722f77972741345e4eb20b7d52d0e38 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 19 Sep 2025 17:45:20 +0800 Subject: [PATCH 14/36] fix progress states --- src/stream/src/executor/locality_provider.rs | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index 9ca8faf7bf955..80a6f2133c0ff 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -293,8 +293,20 @@ impl LocalityProviderExecutor { row_count as i64, ))); - let row = OwnedRow::new(row_data); - progress_table.insert(row); + let new_row = OwnedRow::new(row_data); + + // Check if there's an existing row for this vnode to determine insert vs update + // This ensures state operation consistency - update existing rows, insert new ones + let key_data = vec![Some(vnode.to_scalar().into())]; + let key = OwnedRow::new(key_data); + + if let Some(existing_row) = progress_table.get_row(&key).await? { + // Update existing state - ensures proper state transition for recovery + progress_table.update(existing_row, new_row); + } else { + // Insert new state - first time persisting for this vnode + progress_table.insert(new_row); + } } Ok(()) } From 0d97562fd36c4d8b9b195bee75763d9bdee9453a Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 19 Sep 2025 18:15:54 +0800 Subject: [PATCH 15/36] improve locality provider ordering --- src/meta/src/stream/stream_graph/fragment.rs | 31 ++++++++++++++++++-- src/stream/src/executor/locality_provider.rs | 1 + 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index 54e700df9f19d..ae079961b9405 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -1034,16 +1034,40 @@ impl StreamFragmentGraph { fragment_ordering.insert(*fragment_id, downstream_fragment_ids); } } - println!("{:?}", fragment_ordering); + + // If no backfill order is specified, we still need to ensure that all backfill fragments + // run before LocalityProvider fragments. + if fragment_ordering.is_empty() { + for value in mapping.values() { + for &fragment_id in value { + fragment_ordering.entry(fragment_id).or_default(); + } + } + } + tracing::info!("Backfill fragment ordering from frontend: {fragment_ordering:?}"); // 2. Add dependencies: all backfill fragments should run before LocalityProvider fragments let locality_provider_dependencies = self.find_locality_provider_dependencies(); - println!("{:?}", locality_provider_dependencies); + tracing::info!("LocalityProvider fragment dependencies: {locality_provider_dependencies:?}"); let backfill_fragments: HashSet = mapping.values().flatten().copied().collect(); - let locality_provider_root_fragments: Vec = locality_provider_dependencies.keys().copied().collect(); + + // Calculate LocalityProvider root fragments (zero indegree) + // Root fragments are those that appear as keys but never appear as downstream dependencies + let all_locality_provider_fragments: HashSet = locality_provider_dependencies.keys().copied().collect(); + let downstream_locality_provider_fragments: HashSet = locality_provider_dependencies + .values() + .flatten() + .copied() + .collect(); + let locality_provider_root_fragments: Vec = all_locality_provider_fragments + .difference(&downstream_locality_provider_fragments) + .copied() + .collect(); + tracing::info!("LocalityProvider root fragments (zero indegree): {locality_provider_root_fragments:?}"); // For each backfill fragment, add only the root LocalityProvider fragments as dependents + // This ensures backfill completes before any LocalityProvider starts, while minimizing dependencies for &backfill_fragment_id in &backfill_fragments { fragment_ordering .entry(backfill_fragment_id) @@ -1058,6 +1082,7 @@ impl StreamFragmentGraph { .or_default() .extend(downstream_fragments); } + tracing::info!("Backfill fragments dependencies include scan backfill and locality provider: {fragment_ordering:?}"); fragment_ordering } diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index 80a6f2133c0ff..eb40f92b9cc0b 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -467,6 +467,7 @@ impl LocalityProviderExecutor { if let Some(mutation) = barrier.mutation.as_deref() { use crate::executor::Mutation; if let Mutation::StartFragmentBackfill { fragment_ids } = mutation { + tracing::info!("Start backfill of locality provider with fragment id: {:?}", &self.fragment_id); if fragment_ids.contains(&self.fragment_id) { start_backfill = true; } From f145ed1f25305b94f0f350ec55f8156ecb8e9539 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 19 Sep 2025 18:52:30 +0800 Subject: [PATCH 16/36] add session variable enable_locality_backfill --- e2e_test/batch/catalog/pg_settings.slt.part | 1 + src/common/src/session_config/mod.rs | 4 + .../src/optimizer/plan_node/logical_join.rs | 55 ++++++----- .../src/optimizer/plan_node/logical_scan.rs | 96 ++++++++++--------- .../src/barrier/backfill_order_control.rs | 9 +- src/meta/src/barrier/progress.rs | 2 +- src/meta/src/stream/stream_graph/fragment.rs | 15 ++- src/stream/src/executor/locality_provider.rs | 64 +++++++++---- 8 files changed, 150 insertions(+), 96 deletions(-) diff --git a/e2e_test/batch/catalog/pg_settings.slt.part b/e2e_test/batch/catalog/pg_settings.slt.part index 587f34ebdda91..b7c658a8622a8 100644 --- a/e2e_test/batch/catalog/pg_settings.slt.part +++ b/e2e_test/batch/catalog/pg_settings.slt.part @@ -39,6 +39,7 @@ user datestyle user disable_purify_definition user dml_rate_limit user enable_index_selection +user enable_locality_backfill user enable_join_ordering user enable_share_plan user enable_two_phase_agg diff --git a/src/common/src/session_config/mod.rs b/src/common/src/session_config/mod.rs index cc279e999c027..67f2f797bdedf 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -421,6 +421,10 @@ pub struct SessionConfig { /// Enable index selection for queries #[parameter(default = true)] enable_index_selection: bool, + + /// Enable locality backfill for streaming queries. Defaults to false. + #[parameter(default = false)] + enable_locality_backfill: bool, } fn check_iceberg_engine_connection(val: &str) -> Result<(), String> { diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index ea755fef94a70..942669e19fe0d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -1541,6 +1541,26 @@ impl LogicalJoin { .into()), } } + + fn try_better_locality_inner(&self, columns: &[usize]) -> Option { + let mut ctx = ToStreamContext::new(false); + // only pass through the locality information if it can be converted to dynamic filter + if let Ok(Some(_)) = self.to_stream_dynamic_filter(self.on().clone(), &mut ctx) { + // since dynamic filter only supports left input ref in the output indices, we can safely use o2i mapping to convert the required columns. + let o2i_mapping = self.core.o2i_col_mapping(); + let left_input_columns = columns + .iter() + .map(|&col| o2i_mapping.try_map(col)) + .collect::>>()?; + if let Some(better_left_plan) = self.left().try_better_locality(&left_input_columns) { + return Some( + self.clone_with_left_right(better_left_plan, self.right()) + .into(), + ); + } + } + None + } } impl ToBatch for LogicalJoin { @@ -1754,30 +1774,19 @@ impl ToStream for LogicalJoin { } fn try_better_locality(&self, columns: &[usize]) -> Option { - let mut ctx = ToStreamContext::new(false); - // only pass through the locality information if it can be converted to dynamic filter - if let Ok(Some(_)) = self.to_stream_dynamic_filter(self.on().clone(), &mut ctx) { - // since dynamic filter only supports left input ref in the output indices, we can safely use o2i mapping to convert the required columns. - let o2i_mapping = self.core.o2i_col_mapping(); - let left_input_columns = columns - .iter() - .map(|&col| o2i_mapping.try_map(col)) - .collect::>>()?; - if let Some(better_left_plan) = self.left().try_better_locality(&left_input_columns) { - return Some( - self.clone_with_left_right(better_left_plan, self.right()) - .into(), - ); - } - } - - Some( - LogicalLocalityProvider::new( - self.clone_with_left_right(self.left(), self.right()).into(), - columns.to_owned(), + if let Some(better_plan) = self.try_better_locality_inner(columns) { + return Some(better_plan); + } else if self.ctx().session_ctx().config().enable_locality_backfill() { + Some( + LogicalLocalityProvider::new( + self.clone_with_left_right(self.left(), self.right()).into(), + columns.to_owned(), + ) + .into(), ) - .into(), - ) + } else { + None + } } } diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 1041986a4c5e5..3d7b27b28f49b 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -26,8 +26,8 @@ use super::generic::{GenericPlanNode, GenericPlanRef}; use super::utils::{Distill, childless_record}; use super::{ BatchFilter, BatchPlanRef, BatchProject, ColPrunable, ExprRewritable, Logical, - LogicalPlanRef as PlanRef, PlanBase, PlanNodeId, PredicatePushdown, StreamTableScan, ToBatch, - ToStream, generic, + LogicalLocalityProvider, LogicalPlanRef as PlanRef, PlanBase, PlanNodeId, PlanTreeNode, + PredicatePushdown, StreamTableScan, ToBatch, ToStream, generic, }; use crate::TableCatalog; use crate::binder::BoundBaseTable; @@ -565,6 +565,52 @@ impl LogicalScan { None } + + fn try_better_locality_inner(&self, columns: &[usize]) -> Option { + if !self + .core + .ctx() + .session_ctx() + .config() + .enable_index_selection() + { + return None; + } + if columns.is_empty() { + return None; + } + if self.table_indexes().is_empty() { + return None; + } + let orders = if columns.len() <= 3 { + OrderType::all() + } else { + // Limit the number of order type combinations to avoid explosion. + // For more than 3 columns, we only consider ascending nulls last and descending. + // Since by default, indexes are created with ascending nulls last. + // This is a heuristic to reduce the search space. + vec![OrderType::ascending_nulls_last(), OrderType::descending()] + }; + for order_type_combo in columns + .iter() + .map(|&col| orders.iter().map(move |ot| ColumnOrder::new(col, *ot))) + .multi_cartesian_product() + .take(256) + // limit the number of combinations + { + let required_order = Order { + column_orders: order_type_combo, + }; + + let order_satisfied_index = self.indexes_satisfy_order(&required_order); + for index in order_satisfied_index { + if let Some(index_scan) = self.to_index_scan_if_index_covered(index) { + return Some(index_scan.into()); + } + } + } + None + } } impl ToBatch for LogicalScan { @@ -680,48 +726,12 @@ impl ToStream for LogicalScan { } fn try_better_locality(&self, columns: &[usize]) -> Option { - if !self - .core - .ctx() - .session_ctx() - .config() - .enable_index_selection() - { - return None; - } - if columns.is_empty() { - return None; - } - if self.table_indexes().is_empty() { - return None; - } - let orders = if columns.len() <= 3 { - OrderType::all() + if let Some(better_plan) = self.try_better_locality_inner(columns) { + Some(better_plan) + } else if self.ctx().session_ctx().config().enable_locality_backfill() { + Some(LogicalLocalityProvider::new(self.clone().into(), columns.to_owned()).into()) } else { - // Limit the number of order type combinations to avoid explosion. - // For more than 3 columns, we only consider ascending nulls last and descending. - // Since by default, indexes are created with ascending nulls last. - // This is a heuristic to reduce the search space. - vec![OrderType::ascending_nulls_last(), OrderType::descending()] - }; - for order_type_combo in columns - .iter() - .map(|&col| orders.iter().map(move |ot| ColumnOrder::new(col, *ot))) - .multi_cartesian_product() - .take(256) - // limit the number of combinations - { - let required_order = Order { - column_orders: order_type_combo, - }; - - let order_satisfied_index = self.indexes_satisfy_order(&required_order); - for index in order_satisfied_index { - if let Some(index_scan) = self.to_index_scan_if_index_covered(index) { - return Some(index_scan.into()); - } - } + None } - None } } diff --git a/src/meta/src/barrier/backfill_order_control.rs b/src/meta/src/barrier/backfill_order_control.rs index 0154ebdd3a289..09852b7babbbd 100644 --- a/src/meta/src/barrier/backfill_order_control.rs +++ b/src/meta/src/barrier/backfill_order_control.rs @@ -67,10 +67,11 @@ impl BackfillOrderState { let mut backfill_nodes: HashMap = HashMap::new(); for fragment in stream_job_fragments.fragments() { - if fragment - .fragment_type_mask - .contains_any([FragmentTypeFlag::StreamScan, FragmentTypeFlag::SourceScan, FragmentTypeFlag::LocalityProvider]) - { + if fragment.fragment_type_mask.contains_any([ + FragmentTypeFlag::StreamScan, + FragmentTypeFlag::SourceScan, + FragmentTypeFlag::LocalityProvider, + ]) { let fragment_id = fragment.fragment_id; backfill_nodes.insert( fragment_id, diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index 9ef0814932bf5..7a4ec26647b25 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -187,7 +187,7 @@ impl Progress { BackfillUpstreamType::MView => mv_count += 1, BackfillUpstreamType::Source => source_count += 1, BackfillUpstreamType::Values => (), - BackfillUpstreamType::LocalityProvider => mv_count += 1, // Count LocalityProvider as an MView for progress + BackfillUpstreamType::LocalityProvider => mv_count += 1, /* Count LocalityProvider as an MView for progress */ } } diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index ae079961b9405..cc2418be43697 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -1048,13 +1048,16 @@ impl StreamFragmentGraph { // 2. Add dependencies: all backfill fragments should run before LocalityProvider fragments let locality_provider_dependencies = self.find_locality_provider_dependencies(); - tracing::info!("LocalityProvider fragment dependencies: {locality_provider_dependencies:?}"); + tracing::info!( + "LocalityProvider fragment dependencies: {locality_provider_dependencies:?}" + ); let backfill_fragments: HashSet = mapping.values().flatten().copied().collect(); // Calculate LocalityProvider root fragments (zero indegree) // Root fragments are those that appear as keys but never appear as downstream dependencies - let all_locality_provider_fragments: HashSet = locality_provider_dependencies.keys().copied().collect(); + let all_locality_provider_fragments: HashSet = + locality_provider_dependencies.keys().copied().collect(); let downstream_locality_provider_fragments: HashSet = locality_provider_dependencies .values() .flatten() @@ -1064,7 +1067,9 @@ impl StreamFragmentGraph { .difference(&downstream_locality_provider_fragments) .copied() .collect(); - tracing::info!("LocalityProvider root fragments (zero indegree): {locality_provider_root_fragments:?}"); + tracing::info!( + "LocalityProvider root fragments (zero indegree): {locality_provider_root_fragments:?}" + ); // For each backfill fragment, add only the root LocalityProvider fragments as dependents // This ensures backfill completes before any LocalityProvider starts, while minimizing dependencies @@ -1082,7 +1087,9 @@ impl StreamFragmentGraph { .or_default() .extend(downstream_fragments); } - tracing::info!("Backfill fragments dependencies include scan backfill and locality provider: {fragment_ordering:?}"); + tracing::info!( + "Backfill fragments dependencies include scan backfill and locality provider: {fragment_ordering:?}" + ); fragment_ordering } diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index eb40f92b9cc0b..b04b3d2eb1674 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -81,7 +81,9 @@ impl LocalityBackfillState { } fn vnodes(&self) -> impl Iterator { - self.per_vnode.iter().map(|(&vnode, progress)| (vnode, progress)) + self.per_vnode + .iter() + .map(|(&vnode, progress)| (vnode, progress)) } fn has_progress(&self) -> bool { @@ -467,7 +469,10 @@ impl LocalityProviderExecutor { if let Some(mutation) = barrier.mutation.as_deref() { use crate::executor::Mutation; if let Mutation::StartFragmentBackfill { fragment_ids } = mutation { - tracing::info!("Start backfill of locality provider with fragment id: {:?}", &self.fragment_id); + tracing::info!( + "Start backfill of locality provider with fragment id: {:?}", + &self.fragment_id + ); if fragment_ids.contains(&self.fragment_id) { start_backfill = true; } @@ -623,13 +628,16 @@ impl LocalityProviderExecutor { .await?; // Update progress with current epoch and snapshot read count - let total_snapshot_processed_rows: u64 = backfill_state.vnodes().map(|(_, progress)| { - match progress { - &LocalityBackfillProgress::InProgress { processed_rows, .. } => processed_rows, + let total_snapshot_processed_rows: u64 = backfill_state + .vnodes() + .map(|(_, progress)| match progress { + &LocalityBackfillProgress::InProgress { processed_rows, .. } => { + processed_rows + } &LocalityBackfillProgress::Completed { total_rows, .. } => total_rows, &LocalityBackfillProgress::NotStarted => 0, - } - }).sum(); + }) + .sum(); self.progress.update( barrier.epoch, @@ -653,14 +661,20 @@ impl LocalityProviderExecutor { // Check if all vnodes are complete if backfill_state.is_completed() { // Backfill is complete, finish progress reporting - let total_snapshot_processed_rows: u64 = backfill_state.vnodes().map(|(_, progress)| { - match progress { - &LocalityBackfillProgress::Completed { total_rows, .. } => total_rows, - _ => 0, // Should all be completed at this point - } - }).sum(); + let total_snapshot_processed_rows: u64 = backfill_state + .vnodes() + .map(|(_, progress)| { + match progress { + &LocalityBackfillProgress::Completed { total_rows, .. } => { + total_rows + } + _ => 0, // Should all be completed at this point + } + }) + .sum(); - self.progress.finish(barrier_epoch, total_snapshot_processed_rows); + self.progress + .finish(barrier_epoch, total_snapshot_processed_rows); break 'backfill_loop; } } @@ -684,16 +698,22 @@ impl LocalityProviderExecutor { } // Calculate final total processed rows - let total_snapshot_processed_rows: u64 = backfill_state.vnodes().map(|(_, progress)| { - match progress { - &LocalityBackfillProgress::Completed { total_rows, .. } => total_rows, - &LocalityBackfillProgress::InProgress { processed_rows, .. } => processed_rows, + let total_snapshot_processed_rows: u64 = backfill_state + .vnodes() + .map(|(_, progress)| match progress { + &LocalityBackfillProgress::Completed { total_rows, .. } => { + total_rows + } + &LocalityBackfillProgress::InProgress { + processed_rows, .. + } => processed_rows, &LocalityBackfillProgress::NotStarted => 0, - } - }).sum(); + }) + .sum(); // Finish progress reporting - self.progress.finish(barrier.epoch, total_snapshot_processed_rows); + self.progress + .finish(barrier.epoch, total_snapshot_processed_rows); // Persist final state Self::persist_backfill_state( @@ -720,6 +740,8 @@ impl LocalityProviderExecutor { } } + // TODO: truncate the state table after backfill. + // After backfill completion, forward messages directly #[for_await] for msg in upstream { From 00f7f2b6ff7f32bdf5d4438834abe9d271088e29 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Mon, 22 Sep 2025 18:11:37 +0800 Subject: [PATCH 17/36] fix --- .../src/optimizer/plan_node/stream_locality_provider.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs index c8ceff25df3b7..33b2715e5dcbb 100644 --- a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs +++ b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs @@ -193,7 +193,7 @@ impl StreamLocalityProvider { catalog_builder.set_dist_key_in_pk(vec![0]); let num_of_columns = catalog_builder.columns().len(); - catalog_builder.set_value_indices((1..num_of_columns).collect_vec()); + catalog_builder.set_value_indices((0..num_of_columns).collect_vec()); catalog_builder .build(vec![0], 1) From 60bc6f7b1960e218e2e0bdf1bdf9996b3bf73902 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Thu, 25 Sep 2025 14:00:36 +0800 Subject: [PATCH 18/36] fix locality backfill metrics --- .../src/optimizer/plan_node/logical_scan.rs | 2 +- src/stream/src/executor/locality_provider.rs | 20 ++++++++++++++++--- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 3d7b27b28f49b..e1c7ea484634b 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -26,7 +26,7 @@ use super::generic::{GenericPlanNode, GenericPlanRef}; use super::utils::{Distill, childless_record}; use super::{ BatchFilter, BatchPlanRef, BatchProject, ColPrunable, ExprRewritable, Logical, - LogicalLocalityProvider, LogicalPlanRef as PlanRef, PlanBase, PlanNodeId, PlanTreeNode, + LogicalLocalityProvider, LogicalPlanRef as PlanRef, PlanBase, PlanNodeId, PredicatePushdown, StreamTableScan, ToBatch, ToStream, generic, }; use crate::TableCatalog; diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index b04b3d2eb1674..28cd854b14bd2 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -270,7 +270,6 @@ impl LocalityProviderExecutor { async fn persist_backfill_state( progress_table: &mut StateTable, backfill_state: &LocalityBackfillState, - _locality_columns: &[usize], ) -> StreamExecutorResult<()> { for (vnode, progress) in &backfill_state.per_vnode { let (is_finished, current_pos, row_count) = match progress { @@ -525,6 +524,10 @@ impl LocalityProviderExecutor { let mut upstream_chunk_buffer: Vec = vec![]; let mut pending_barrier: Option = None; + let metrics = self + .metrics + .new_backfill_metrics(state_table.table_id(), self.actor_id); + 'backfill_loop: loop { let mut cur_barrier_snapshot_processed_rows: u64 = 0; let mut cur_barrier_upstream_processed_rows: u64 = 0; @@ -577,6 +580,12 @@ impl LocalityProviderExecutor { chunk_cardinality; yield Message::Chunk(chunk); } + metrics + .backfill_snapshot_read_row_count + .inc_by(cur_barrier_snapshot_processed_rows); + metrics + .backfill_upstream_output_row_count + .inc_by(cur_barrier_upstream_processed_rows); break 'backfill_loop; } Some((vnode, row)) => { @@ -649,12 +658,18 @@ impl LocalityProviderExecutor { Self::persist_backfill_state( &mut progress_table, &backfill_state, - &self.locality_columns, ) .await?; let barrier_epoch = barrier.epoch; let post_commit = progress_table.commit(barrier_epoch).await?; + metrics + .backfill_snapshot_read_row_count + .inc_by(cur_barrier_snapshot_processed_rows); + metrics + .backfill_upstream_output_row_count + .inc_by(cur_barrier_upstream_processed_rows); + yield Message::Barrier(barrier); post_commit.post_yield_barrier(None).await?; @@ -719,7 +734,6 @@ impl LocalityProviderExecutor { Self::persist_backfill_state( &mut progress_table, &backfill_state, - &self.locality_columns, ) .await?; let post_commit = progress_table.commit(barrier.epoch).await?; From 7bae65ce68d69acb46197526ef3093f07057d0fc Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Thu, 25 Sep 2025 14:20:51 +0800 Subject: [PATCH 19/36] use chunk builder --- .../src/optimizer/plan_node/logical_scan.rs | 4 +- src/stream/src/executor/locality_provider.rs | 118 +++++++++++++----- 2 files changed, 91 insertions(+), 31 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index e1c7ea484634b..06f542b9a696b 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -26,8 +26,8 @@ use super::generic::{GenericPlanNode, GenericPlanRef}; use super::utils::{Distill, childless_record}; use super::{ BatchFilter, BatchPlanRef, BatchProject, ColPrunable, ExprRewritable, Logical, - LogicalLocalityProvider, LogicalPlanRef as PlanRef, PlanBase, PlanNodeId, - PredicatePushdown, StreamTableScan, ToBatch, ToStream, generic, + LogicalLocalityProvider, LogicalPlanRef as PlanRef, PlanBase, PlanNodeId, PredicatePushdown, + StreamTableScan, ToBatch, ToStream, generic, }; use crate::TableCatalog; use crate::binder::BoundBaseTable; diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index 28cd854b14bd2..7cc9c184c6d75 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -20,19 +20,24 @@ use futures::stream::select_with_strategy; use futures::{TryStreamExt, pin_mut, stream}; use futures_async_stream::try_stream; use itertools::Itertools; -use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::array::{DataChunk, Op, StreamChunk}; use risingwave_common::catalog::Schema; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::types::{Datum, ToOwnedDatum}; +use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::sort_util::cmp_datum_iter; +use risingwave_common_rate_limit::RateLimit; use risingwave_storage::StateStore; use risingwave_storage::store::PrefetchOptions; use crate::common::table::state_table::StateTable; +use crate::executor::backfill::utils::create_builder; use crate::executor::prelude::*; use crate::task::{CreateMviewProgressReporter, FragmentId}; +type Builders = HashMap; + /// Progress state for tracking backfill per vnode #[derive(Clone, Debug, PartialEq, Eq)] enum LocalityBackfillProgress { @@ -403,6 +408,28 @@ impl LocalityProviderExecutor { let chunk = StreamChunk::with_visibility(ops, columns, new_visibility.finish()); Ok(chunk) } + + fn handle_snapshot_chunk( + data_chunk: DataChunk, + vnode: VirtualNode, + pk_indices: &[usize], + backfill_state: &mut LocalityBackfillState, + cur_barrier_snapshot_processed_rows: &mut u64, + ) -> StreamExecutorResult { + let chunk = StreamChunk::from_parts(vec![Op::Insert; data_chunk.cardinality()], data_chunk); + let chunk_cardinality = chunk.cardinality() as u64; + + // Extract primary key from the last row to update progress + // As snapshot read streams are ordered by pk, we can use the last row to update current_pos + if let Some(last_row) = chunk.rows().last() { + let pk = last_row.1.project(pk_indices); + let pk_owned = pk.into_owned_row(); + backfill_state.update_progress(vnode, pk_owned, chunk_cardinality); + } + + *cur_barrier_snapshot_processed_rows += chunk_cardinality; + Ok(chunk) + } } impl Execute for LocalityProviderExecutor { @@ -528,6 +555,21 @@ impl LocalityProviderExecutor { .metrics .new_backfill_metrics(state_table.table_id(), self.actor_id); + // Create builders for snapshot data chunks + let snapshot_data_types = self.input_schema.data_types(); + let mut builders: Builders = state_table + .vnodes() + .iter_vnodes() + .map(|vnode| { + let builder = create_builder( + RateLimit::Disabled, + self.chunk_size, + snapshot_data_types.clone(), + ); + (vnode, builder) + }) + .collect(); + 'backfill_loop: loop { let mut cur_barrier_snapshot_processed_rows: u64 = 0; let mut cur_barrier_upstream_processed_rows: u64 = 0; @@ -573,7 +615,21 @@ impl LocalityProviderExecutor { match msg? { None => { // End of the snapshot read stream. - // Consume remaining rows in the buffer. + // Consume remaining rows in the builders. + for (vnode, builder) in &mut builders { + if let Some(data_chunk) = builder.consume_all() { + let chunk = Self::handle_snapshot_chunk( + data_chunk, + *vnode, + &pk_indices, + &mut backfill_state, + &mut cur_barrier_snapshot_processed_rows, + )?; + yield Message::Chunk(chunk); + } + } + + // Consume remaining rows in the upstream buffer. for chunk in upstream_chunk_buffer.drain(..) { let chunk_cardinality = chunk.cardinality() as u64; cur_barrier_upstream_processed_rows += @@ -589,23 +645,21 @@ impl LocalityProviderExecutor { break 'backfill_loop; } Some((vnode, row)) => { - // Extract primary key from row for progress tracking - let pk = row.clone().project(&pk_indices); - - // Convert projected row to OwnedRow for progress tracking - let pk_owned = pk.into_owned_row(); - - // Update progress for this vnode - backfill_state.update_progress(vnode, pk_owned, 1); - - cur_barrier_snapshot_processed_rows += 1; - - // Create chunk with single row - let chunk = StreamChunk::from_rows( - &[(Op::Insert, row)], - &self.input_schema.data_types(), - ); - yield Message::Chunk(chunk); + // Use builder to batch rows efficiently + let builder = builders.get_mut(&vnode).unwrap(); + if let Some(data_chunk) = builder.append_one_row(row) { + // Builder is full, handle the chunk + let chunk = Self::handle_snapshot_chunk( + data_chunk, + vnode, + &pk_indices, + &mut backfill_state, + &mut cur_barrier_snapshot_processed_rows, + )?; + yield Message::Chunk(chunk); + } + // If append_one_row returns None, row is buffered but no chunk is produced yet + // Progress will be updated when the builder is consumed later } } } @@ -619,6 +673,20 @@ impl LocalityProviderExecutor { None => break 'backfill_loop, // Reached end of backfill }; + // Consume remaining rows from builders at barrier + for (vnode, builder) in &mut builders { + if let Some(data_chunk) = builder.consume_all() { + let chunk = Self::handle_snapshot_chunk( + data_chunk, + *vnode, + &pk_indices, + &mut backfill_state, + &mut cur_barrier_snapshot_processed_rows, + )?; + yield Message::Chunk(chunk); + } + } + // Process upstream buffer chunks with marking for chunk in upstream_chunk_buffer.drain(..) { cur_barrier_upstream_processed_rows += chunk.cardinality() as u64; @@ -655,11 +723,7 @@ impl LocalityProviderExecutor { ); // Persist backfill progress - Self::persist_backfill_state( - &mut progress_table, - &backfill_state, - ) - .await?; + Self::persist_backfill_state(&mut progress_table, &backfill_state).await?; let barrier_epoch = barrier.epoch; let post_commit = progress_table.commit(barrier_epoch).await?; @@ -731,11 +795,7 @@ impl LocalityProviderExecutor { .finish(barrier.epoch, total_snapshot_processed_rows); // Persist final state - Self::persist_backfill_state( - &mut progress_table, - &backfill_state, - ) - .await?; + Self::persist_backfill_state(&mut progress_table, &backfill_state).await?; let post_commit = progress_table.commit(barrier.epoch).await?; yield Message::Barrier(barrier); From 171e1247e57e29d530c8de86a44671bd9faae17f Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Thu, 25 Sep 2025 14:45:35 +0800 Subject: [PATCH 20/36] fmt --- .../src/optimizer/plan_node/logical_join.rs | 2 +- .../plan_node/logical_locality_provider.rs | 12 ++--- .../plan_node/stream_locality_provider.rs | 6 +-- src/meta/src/stream/stream_graph/fragment.rs | 15 +++--- src/stream/src/executor/locality_provider.rs | 54 ++++++------------- 5 files changed, 31 insertions(+), 58 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index 942669e19fe0d..ad1b4c68d860d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -1775,7 +1775,7 @@ impl ToStream for LogicalJoin { fn try_better_locality(&self, columns: &[usize]) -> Option { if let Some(better_plan) = self.try_better_locality_inner(columns) { - return Some(better_plan); + Some(better_plan) } else if self.ctx().session_ctx().config().enable_locality_backfill() { Some( LogicalLocalityProvider::new( diff --git a/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs b/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs index 46859b2d3e35a..7a3a4239dafd0 100644 --- a/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs +++ b/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs @@ -112,8 +112,8 @@ impl ToBatch for LogicalLocalityProvider { fn to_batch(&self) -> Result { // LocalityProvider is a streaming-only operator Err(crate::error::ErrorCode::NotSupported( - "LocalityProvider in batch mode".to_string(), - "LocalityProvider is only supported in streaming mode for backfilling".to_string(), + "LocalityProvider in batch mode".to_owned(), + "LocalityProvider is only supported in streaming mode for backfilling".to_owned(), ) .into()) } @@ -168,12 +168,10 @@ impl LogicalLocalityProvider { pub fn try_better_locality(&self, columns: &[usize]) -> Option { if columns == self.locality_columns() { Some(self.clone().into()) + } else if let Some(better_input) = self.input().try_better_locality(columns) { + Some(better_input) } else { - if let Some(better_input) = self.input().try_better_locality(columns) { - Some(better_input) - } else { - Some(Self::new(self.input(), columns.to_owned()).into()) - } + Some(Self::new(self.input(), columns.to_owned()).into()) } } } diff --git a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs index 33b2715e5dcbb..f5e82cc2343b8 100644 --- a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs +++ b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs @@ -124,7 +124,7 @@ impl ExprVisitable for StreamLocalityProvider { impl StreamLocalityProvider { /// Build the state table catalog for buffering input data /// Schema: same as input schema (locality handled by primary key ordering) - /// Key: locality_columns (vnode handled internally by StateTable) + /// Key: `locality_columns` (vnode handled internally by `StateTable`) fn build_state_catalog(&self, state: &mut BuildFragmentGraphState) -> TableCatalog { let mut catalog_builder = TableCatalogBuilder::default(); let input = self.input(); @@ -146,7 +146,7 @@ impl StreamLocalityProvider { } } - catalog_builder.set_value_indices((0..input_schema.len()).into_iter().collect()); + catalog_builder.set_value_indices((0..input_schema.len()).collect()); catalog_builder .build( @@ -157,7 +157,7 @@ impl StreamLocalityProvider { } /// Build the progress table catalog for tracking backfill progress - /// Schema: | vnode | pk(locality columns + input stream keys) | backfill_finished | row_count | + /// Schema: | vnode | pk(locality columns + input stream keys) | `backfill_finished` | `row_count` | /// Key: | vnode | pk(locality columns + input stream keys) | fn build_progress_catalog(&self, state: &mut BuildFragmentGraphState) -> TableCatalog { let mut catalog_builder = TableCatalogBuilder::default(); diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index cc2418be43697..60f52b95d0601 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -1094,13 +1094,13 @@ impl StreamFragmentGraph { fragment_ordering } - /// Find dependency relationships among fragments containing LocalityProvider nodes. + /// Find dependency relationships among fragments containing `LocalityProvider` nodes. /// Returns a mapping where each fragment ID maps to a list of fragment IDs that should be processed after it. - /// Following the same semantics as FragmentBackfillOrder: - /// `G[10] -> [1, 2, 11]` means LocalityProvider in fragment 10 should be processed - /// before LocalityProviders in fragments 1, 2, and 11. + /// Following the same semantics as `FragmentBackfillOrder`: + /// `G[10] -> [1, 2, 11]` means `LocalityProvider` in fragment 10 should be processed + /// before `LocalityProviders` in fragments 1, 2, and 11. /// - /// This method assumes each fragment contains at most one LocalityProvider node. + /// This method assumes each fragment contains at most one `LocalityProvider` node. pub fn find_locality_provider_dependencies(&self) -> HashMap> { let mut locality_provider_fragments = HashSet::new(); let mut dependencies: HashMap> = HashMap::new(); @@ -1143,7 +1143,6 @@ impl StreamFragmentGraph { dependencies } - /// Check if a fragment contains a LocalityProvider node fn fragment_has_locality_provider(&self, fragment: &BuildingFragment) -> bool { let mut has_locality_provider = false; @@ -1161,7 +1160,7 @@ impl StreamFragmentGraph { has_locality_provider } - /// Recursively collect downstream LocalityProvider fragments + /// Recursively collect downstream `LocalityProvider` fragments fn collect_downstream_locality_providers( &self, current_fragment_id: GlobalFragmentId, @@ -1175,7 +1174,7 @@ impl StreamFragmentGraph { visited.insert(current_fragment_id); // Check all downstream fragments - for (&downstream_id, _edge) in self.get_downstreams(current_fragment_id) { + for &downstream_id in self.get_downstreams(current_fragment_id).keys() { let downstream_fragment_id = downstream_id.as_global_id(); // If the downstream fragment is a LocalityProvider, add it to results diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index 7cc9c184c6d75..05a53b73f991d 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -166,6 +166,7 @@ pub struct LocalityProviderExecutor { upstream: Executor, /// Locality columns (indices in input schema) + #[allow(dead_code)] locality_columns: Vec, /// State table for buffering input data @@ -174,13 +175,13 @@ pub struct LocalityProviderExecutor { /// Progress table for tracking backfill progress per vnode progress_table: StateTable, - /// Schema of the input input_schema: Schema, /// Progress reporter for materialized view creation progress: CreateMviewProgressReporter, - /// Actor ID for this executor + fragment_id: FragmentId, + actor_id: ActorId, /// Metrics @@ -188,9 +189,6 @@ pub struct LocalityProviderExecutor { /// Chunk size for output chunk_size: usize, - - /// Fragment ID of the fragment this LocalityProvider belongs to - fragment_id: FragmentId, } impl LocalityProviderExecutor { @@ -320,7 +318,6 @@ impl LocalityProviderExecutor { /// Load backfill state from progress table async fn load_backfill_state( progress_table: &StateTable, - _locality_columns: &[usize], ) -> StreamExecutorResult { let mut backfill_state = LocalityBackfillState::new(progress_table.vnodes().iter_vnodes()); let mut total_snapshot_rows = 0; @@ -387,7 +384,7 @@ impl LocalityProviderExecutor { let pk_indices = state_table.pk_indices(); let pk_order = state_table.pk_serde().get_order_types(); - for (_i, row) in data.rows().enumerate() { + for row in data.rows() { // Project to primary key columns for comparison let pk = row.project(pk_indices); let vnode = state_table.compute_vnode_by_pk(pk); @@ -458,8 +455,7 @@ impl LocalityProviderExecutor { progress_table.init_epoch(first_epoch).await?; // Load backfill state from progress table - let mut backfill_state = - Self::load_backfill_state(&progress_table, &self.locality_columns).await?; + let mut backfill_state = Self::load_backfill_state(&progress_table).await?; // Get pk info from state table let pk_indices = state_table.pk_indices().iter().cloned().collect_vec(); @@ -707,12 +703,12 @@ impl LocalityProviderExecutor { // Update progress with current epoch and snapshot read count let total_snapshot_processed_rows: u64 = backfill_state .vnodes() - .map(|(_, progress)| match progress { - &LocalityBackfillProgress::InProgress { processed_rows, .. } => { + .map(|(_, progress)| match *progress { + LocalityBackfillProgress::InProgress { processed_rows, .. } => { processed_rows } - &LocalityBackfillProgress::Completed { total_rows, .. } => total_rows, - &LocalityBackfillProgress::NotStarted => 0, + LocalityBackfillProgress::Completed { total_rows, .. } => total_rows, + LocalityBackfillProgress::NotStarted => 0, }) .sum(); @@ -736,26 +732,6 @@ impl LocalityProviderExecutor { yield Message::Barrier(barrier); post_commit.post_yield_barrier(None).await?; - - // Check if all vnodes are complete - if backfill_state.is_completed() { - // Backfill is complete, finish progress reporting - let total_snapshot_processed_rows: u64 = backfill_state - .vnodes() - .map(|(_, progress)| { - match progress { - &LocalityBackfillProgress::Completed { total_rows, .. } => { - total_rows - } - _ => 0, // Should all be completed at this point - } - }) - .sum(); - - self.progress - .finish(barrier_epoch, total_snapshot_processed_rows); - break 'backfill_loop; - } } } @@ -779,14 +755,14 @@ impl LocalityProviderExecutor { // Calculate final total processed rows let total_snapshot_processed_rows: u64 = backfill_state .vnodes() - .map(|(_, progress)| match progress { - &LocalityBackfillProgress::Completed { total_rows, .. } => { + .map(|(_, progress)| match *progress { + LocalityBackfillProgress::Completed { total_rows, .. } => { total_rows } - &LocalityBackfillProgress::InProgress { - processed_rows, .. - } => processed_rows, - &LocalityBackfillProgress::NotStarted => 0, + LocalityBackfillProgress::InProgress { processed_rows, .. } => { + processed_rows + } + LocalityBackfillProgress::NotStarted => 0, }) .sum(); From 4a86f92923a88736792da9918ebe0f05f7d037e2 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Thu, 25 Sep 2025 16:07:20 +0800 Subject: [PATCH 21/36] fix --- .../src/optimizer/plan_node/stream_locality_provider.rs | 2 +- src/prost/build.rs | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs index f5e82cc2343b8..7c941e6a74ca9 100644 --- a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs +++ b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs @@ -101,7 +101,7 @@ impl StreamNode for StreamLocalityProvider { progress_table: Some(progress_table.to_prost()), }; - PbNodeBody::LocalityProvider(locality_provider_node) + PbNodeBody::LocalityProvider(Box::new(locality_provider_node)) } } diff --git a/src/prost/build.rs b/src/prost/build.rs index babc61295e84f..0d818976d015b 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -170,6 +170,7 @@ fn main() -> Result<(), Box> { .boxed(".stream_plan.StreamNode.node_body.sync_log_store") .boxed(".stream_plan.StreamNode.node_body.materialized_exprs") .boxed(".stream_plan.StreamNode.node_body.vector_index_write") + .boxed(".stream_plan.StreamNode.node_body.locality_provider") // `Udf` is 248 bytes, while 2nd largest field is 32 bytes. .boxed(".expr.ExprNode.rex_node.udf") // Eq + Hash are for plan nodes to do common sub-plan detection. From 6c6e2f00f10c779911e18d1eb85e8fc6c6338013 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Thu, 25 Sep 2025 17:26:27 +0800 Subject: [PATCH 22/36] fix --- src/common/src/catalog/mod.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index 3190d1ab8470d..974e3697caa9c 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -893,6 +893,11 @@ mod tests { 65536, "UPSTREAM_SINK_UNION", ), + ( + LocalityProvider, + 131072, + "LOCALITY_PROVIDER", + ), ] "#]] .assert_debug_eq( From 1151be146af595e143433309979b76d499b7e34b Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 26 Sep 2025 00:04:40 +0800 Subject: [PATCH 23/36] fix --- e2e_test/batch/catalog/pg_settings.slt.part | 1 + 1 file changed, 1 insertion(+) diff --git a/e2e_test/batch/catalog/pg_settings.slt.part b/e2e_test/batch/catalog/pg_settings.slt.part index b7c658a8622a8..298ce3a7c91e2 100644 --- a/e2e_test/batch/catalog/pg_settings.slt.part +++ b/e2e_test/batch/catalog/pg_settings.slt.part @@ -41,6 +41,7 @@ user dml_rate_limit user enable_index_selection user enable_locality_backfill user enable_join_ordering +user enable_locality_backfill user enable_share_plan user enable_two_phase_agg user extra_float_digits From 714defce75b3356e9b055065c70357cf1600bb3c Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 26 Sep 2025 10:58:10 +0800 Subject: [PATCH 24/36] fix --- e2e_test/batch/catalog/pg_settings.slt.part | 1 - 1 file changed, 1 deletion(-) diff --git a/e2e_test/batch/catalog/pg_settings.slt.part b/e2e_test/batch/catalog/pg_settings.slt.part index 298ce3a7c91e2..e32b89e8aa344 100644 --- a/e2e_test/batch/catalog/pg_settings.slt.part +++ b/e2e_test/batch/catalog/pg_settings.slt.part @@ -39,7 +39,6 @@ user datestyle user disable_purify_definition user dml_rate_limit user enable_index_selection -user enable_locality_backfill user enable_join_ordering user enable_locality_backfill user enable_share_plan From c348fe2ee433288476d36cd82d65f4f98646402d Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Fri, 26 Sep 2025 13:28:31 +0800 Subject: [PATCH 25/36] remove deadcode --- src/stream/src/executor/locality_provider.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/stream/src/executor/locality_provider.rs b/src/stream/src/executor/locality_provider.rs index 05a53b73f991d..519c488832eb8 100644 --- a/src/stream/src/executor/locality_provider.rs +++ b/src/stream/src/executor/locality_provider.rs @@ -569,7 +569,6 @@ impl LocalityProviderExecutor { 'backfill_loop: loop { let mut cur_barrier_snapshot_processed_rows: u64 = 0; let mut cur_barrier_upstream_processed_rows: u64 = 0; - let _snapshot_read_complete = false; // Create the backfill stream with upstream and snapshot { From 8b01b5a323b223c906f71091db6cd23d7fd8d568 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Mon, 29 Sep 2025 13:26:43 +0800 Subject: [PATCH 26/36] fmt --- .../optimizer/plan_node/logical_locality_provider.rs | 11 +++++++---- .../optimizer/plan_node/stream_locality_provider.rs | 6 ++++-- src/meta/src/stream/stream_graph/fragment.rs | 10 ---------- src/stream/src/from_proto/locality_provider.rs | 6 +++--- 4 files changed, 14 insertions(+), 19 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs b/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs index 7a3a4239dafd0..2670c696a5763 100644 --- a/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs +++ b/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs @@ -32,6 +32,10 @@ use crate::utils::{ColIndexMapping, Condition}; /// `LogicalLocalityProvider` provides locality for operators during backfilling. /// It buffers input data into a state table using locality columns as primary key prefix. +/// +/// The `LocalityProvider` has 2 states: +/// - One is used to buffer data during backfilling and provide data locality. +/// - The other one is a progress table like normal backfill operator to track the backfilling progress of itself. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalLocalityProvider { pub base: PlanBase, @@ -45,12 +49,10 @@ impl LogicalLocalityProvider { LogicalLocalityProvider { base, core } } - /// Create a `LogicalLocalityProvider` with the given input and locality columns pub fn create(input: PlanRef, locality_columns: Vec) -> PlanRef { LogicalLocalityProvider::new(input, locality_columns).into() } - /// Get the locality columns of the locality provider. pub fn locality_columns(&self) -> &[usize] { &self.core.locality_columns } @@ -128,7 +130,9 @@ impl ToStream for LogicalLocalityProvider { RequiredDist::shard_by_key(self.input().schema().len(), self.locality_columns()); let input = required_dist.streaming_enforce_if_not_satisfies(input)?; let input = if input.as_stream_exchange().is_none() { - // force a no shuffle exchange to ensure locality provider is in its own fragment + // Force a no shuffle exchange to ensure locality provider is in its own fragment. + // This is important to ensure the backfill ordering can recognize and build + // the dependency graph among different backfill-needed fragments. StreamExchange::new_no_shuffle(input).into() } else { input @@ -164,7 +168,6 @@ impl ExprVisitable for LogicalLocalityProvider { } impl LogicalLocalityProvider { - /// Try to provide better locality by transforming input pub fn try_better_locality(&self, columns: &[usize]) -> Option { if columns == self.locality_columns() { Some(self.clone().into()) diff --git a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs index 7c941e6a74ca9..ec46c83ab846c 100644 --- a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs +++ b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs @@ -45,7 +45,9 @@ impl StreamLocalityProvider { let dist = match input.distribution() { Distribution::HashShard(keys) => { // If the input is hash-distributed, we make it a UpstreamHashShard distribution - // just like a normal table scan. It is used to ensure locality provider is in its own fragment + // just like a normal table scan. It is used to ensure locality provider is in its own fragment. + // This is important to ensure the backfill ordering can recognize and build + // the dependency graph among different backfill-needed fragments. Distribution::UpstreamHashShard(keys.clone(), TableId::placeholder()) } Distribution::UpstreamHashShard(keys, table_id) => { @@ -135,7 +137,7 @@ impl StreamLocalityProvider { catalog_builder.add_column(field); } - // Set locality columns as primary key (vnode will be handled internally) + // Set locality columns as primary key. for locality_col_idx in self.locality_columns() { catalog_builder.add_order_column(*locality_col_idx, OrderType::ascending()); } diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index 60f52b95d0601..3ba9103d1efd9 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -1044,13 +1044,9 @@ impl StreamFragmentGraph { } } } - tracing::info!("Backfill fragment ordering from frontend: {fragment_ordering:?}"); // 2. Add dependencies: all backfill fragments should run before LocalityProvider fragments let locality_provider_dependencies = self.find_locality_provider_dependencies(); - tracing::info!( - "LocalityProvider fragment dependencies: {locality_provider_dependencies:?}" - ); let backfill_fragments: HashSet = mapping.values().flatten().copied().collect(); @@ -1067,9 +1063,6 @@ impl StreamFragmentGraph { .difference(&downstream_locality_provider_fragments) .copied() .collect(); - tracing::info!( - "LocalityProvider root fragments (zero indegree): {locality_provider_root_fragments:?}" - ); // For each backfill fragment, add only the root LocalityProvider fragments as dependents // This ensures backfill completes before any LocalityProvider starts, while minimizing dependencies @@ -1087,9 +1080,6 @@ impl StreamFragmentGraph { .or_default() .extend(downstream_fragments); } - tracing::info!( - "Backfill fragments dependencies include scan backfill and locality provider: {fragment_ordering:?}" - ); fragment_ordering } diff --git a/src/stream/src/from_proto/locality_provider.rs b/src/stream/src/from_proto/locality_provider.rs index e9a30a4a86255..7472705f8d34e 100644 --- a/src/stream/src/from_proto/locality_provider.rs +++ b/src/stream/src/from_proto/locality_provider.rs @@ -22,8 +22,6 @@ use crate::common::table::state_table::StateTableBuilder; use crate::executor::Executor; use crate::executor::locality_provider::LocalityProviderExecutor; -pub struct LocalityProviderBuilder; - impl ExecutorBuilder for LocalityProviderBuilder { type Node = LocalityProviderNode; @@ -77,10 +75,12 @@ impl ExecutorBuilder for LocalityProviderBuilder { input_schema, progress, params.executor_stats.clone(), - 1024, // default chunk size + params.env.config().developer.chunk_size, params.actor_context.fragment_id, ); Ok((params.info, exec).into()) } } + +pub struct LocalityProviderBuilder; From deb2ccb1ab38c0319263d56e57e5f4b692d36f33 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Mon, 29 Sep 2025 13:28:12 +0800 Subject: [PATCH 27/36] set enable_locality_backfill = true to test e2e backfill test --- src/common/src/session_config/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/common/src/session_config/mod.rs b/src/common/src/session_config/mod.rs index 67f2f797bdedf..85f1a41596efc 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -423,7 +423,7 @@ pub struct SessionConfig { enable_index_selection: bool, /// Enable locality backfill for streaming queries. Defaults to false. - #[parameter(default = false)] + #[parameter(default = true)] enable_locality_backfill: bool, } From 1d7564fcb946a942705a98859dbc4f5c99492796 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Mon, 29 Sep 2025 13:58:17 +0800 Subject: [PATCH 28/36] fix --- src/frontend/src/optimizer/plan_node/logical_agg.rs | 13 +++++++++---- .../plan_node/logical_locality_provider.rs | 1 + 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 299425c8a446e..8ca844d3e1977 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -1416,10 +1416,15 @@ impl ToStream for LogicalAgg { use super::stream::prelude::*; let eowc = ctx.emit_on_window_close(); - let input = self - .input() - .try_better_locality(&self.group_key().to_vec()) - .unwrap_or_else(|| self.input()); + let input = if self.group_key().is_empty() { + self.input() + } else { + self + .input() + .try_better_locality(&self.group_key().to_vec()) + .unwrap_or_else(|| self.input()) + }; + let stream_input = input.to_stream(ctx)?; // Use Dedup operator, if possible. diff --git a/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs b/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs index 2670c696a5763..b487e0917fd4f 100644 --- a/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs +++ b/src/frontend/src/optimizer/plan_node/logical_locality_provider.rs @@ -44,6 +44,7 @@ pub struct LogicalLocalityProvider { impl LogicalLocalityProvider { pub fn new(input: PlanRef, locality_columns: Vec) -> Self { + assert!(!locality_columns.is_empty()); let core = generic::LocalityProvider::new(input, locality_columns); let base = PlanBase::new_logical_with_core(&core); LogicalLocalityProvider { base, core } From 22bc8b8be9fe944a6585e083e0c970eb21db614e Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Mon, 29 Sep 2025 13:58:32 +0800 Subject: [PATCH 29/36] Revert "set enable_locality_backfill = true to test e2e backfill test" This reverts commit deb2ccb1ab38c0319263d56e57e5f4b692d36f33. --- src/common/src/session_config/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/common/src/session_config/mod.rs b/src/common/src/session_config/mod.rs index 85f1a41596efc..67f2f797bdedf 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -423,7 +423,7 @@ pub struct SessionConfig { enable_index_selection: bool, /// Enable locality backfill for streaming queries. Defaults to false. - #[parameter(default = true)] + #[parameter(default = false)] enable_locality_backfill: bool, } From 16f6b9ce1ee6caaea9e6a70d4513d5885102d91d Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Mon, 29 Sep 2025 14:17:11 +0800 Subject: [PATCH 30/36] fmt --- src/frontend/src/optimizer/plan_node/logical_agg.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 8ca844d3e1977..3238f71936577 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -1419,8 +1419,7 @@ impl ToStream for LogicalAgg { let input = if self.group_key().is_empty() { self.input() } else { - self - .input() + self.input() .try_better_locality(&self.group_key().to_vec()) .unwrap_or_else(|| self.input()) }; From e44a12f09709c08e7ba79288d4459170b9451d87 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Mon, 29 Sep 2025 15:25:28 +0800 Subject: [PATCH 31/36] add test --- .../testdata/input/locality_backfill.yaml | 13 ++++++++ .../testdata/output/locality_backfill.yaml | 32 +++++++++++++++++++ 2 files changed, 45 insertions(+) create mode 100644 src/frontend/planner_test/tests/testdata/input/locality_backfill.yaml create mode 100644 src/frontend/planner_test/tests/testdata/output/locality_backfill.yaml diff --git a/src/frontend/planner_test/tests/testdata/input/locality_backfill.yaml b/src/frontend/planner_test/tests/testdata/input/locality_backfill.yaml new file mode 100644 index 0000000000000..4d63c72fea2fb --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/input/locality_backfill.yaml @@ -0,0 +1,13 @@ +- sql: | + set enable_locality_backfill = true; + create table t (a int, b int, c int); + select count(*) from t group by b; + expected_outputs: + - stream_plan +- sql: | + set enable_locality_backfill = true; + create table t1 (a int, b int, c int); + create table t2 (a int, b int, c int); + select count(*) from t1 join t2 on t1.a = t2.a group by t1.b; + expected_outputs: + - stream_plan \ No newline at end of file diff --git a/src/frontend/planner_test/tests/testdata/output/locality_backfill.yaml b/src/frontend/planner_test/tests/testdata/output/locality_backfill.yaml new file mode 100644 index 0000000000000..9070cb62dfb10 --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/output/locality_backfill.yaml @@ -0,0 +1,32 @@ +# This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. +- sql: | + set enable_locality_backfill = true; + create table t (a int, b int, c int); + select count(*) from t group by b; + stream_plan: |- + StreamMaterialize { columns: [count, t.b(hidden)], stream_key: [t.b], pk_columns: [t.b], pk_conflict: NoCheck } + └─StreamProject { exprs: [count, t.b] } + └─StreamHashAgg { group_key: [t.b], aggs: [count] } + └─StreamLocalityProvider { locality_columns: [0] } + └─StreamExchange { dist: HashShard(t.b) } + └─StreamTableScan { table: t, columns: [t.b, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } +- sql: | + set enable_locality_backfill = true; + create table t1 (a int, b int, c int); + create table t2 (a int, b int, c int); + select count(*) from t1 join t2 on t1.a = t2.a group by t1.b; + stream_plan: |- + StreamMaterialize { columns: [count, t1.b(hidden)], stream_key: [t1.b], pk_columns: [t1.b], pk_conflict: NoCheck } + └─StreamProject { exprs: [count, t1.b] } + └─StreamHashAgg { group_key: [t1.b], aggs: [count] } + └─StreamLocalityProvider { locality_columns: [0] } + └─StreamExchange { dist: HashShard(t1.b) } + └─StreamHashJoin { type: Inner, predicate: t1.a = t2.a, output: [t1.b, t1._row_id, t1.a, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.a) } + │ └─StreamLocalityProvider { locality_columns: [0] } + │ └─StreamExchange { dist: HashShard(t1.a) } + │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.a) } + └─StreamLocalityProvider { locality_columns: [0] } + └─StreamExchange { dist: HashShard(t2.a) } + └─StreamTableScan { table: t2, columns: [t2.a, t2._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t2._row_id], pk: [_row_id], dist: UpstreamHashShard(t2._row_id) } From 9dabe6a3cff237ae8d2e422bb226da94be6578b7 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Mon, 29 Sep 2025 17:32:00 +0800 Subject: [PATCH 32/36] add tests --- e2e_test/backfill/locality_backfill/basic.slt | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) create mode 100644 e2e_test/backfill/locality_backfill/basic.slt diff --git a/e2e_test/backfill/locality_backfill/basic.slt b/e2e_test/backfill/locality_backfill/basic.slt new file mode 100644 index 0000000000000..285e59b213a1a --- /dev/null +++ b/e2e_test/backfill/locality_backfill/basic.slt @@ -0,0 +1,35 @@ +statement ok +set enable_locality_backfill=true; + +statement ok +create table t1(a int, b int); + +statement ok +create table t2(a int, b int); + +statement ok +insert into t1 select i, 123 from generate_series(1, 1000, 1) i; + +statement ok +insert into t2 select i, 123 from generate_series(1, 1000, 1) i ; + +statement ok +flush; + +statement ok +create materialized view mv as select count(*) from t1 join t2 on t1.a = t2.a group by t1.b; + + +query ? +select * from mv; +---- +1000 + +statement ok +drop materialized view mv; + +statement ok +drop table t1; + +statement ok +drop table t2; From 3dd2398f7f184a32d84086c3c0471e9ac844f193 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Mon, 29 Sep 2025 17:48:43 +0800 Subject: [PATCH 33/36] add locality backfill test to run backfill tests --- ci/scripts/run-backfill-tests.sh | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/ci/scripts/run-backfill-tests.sh b/ci/scripts/run-backfill-tests.sh index ea7a5f0b6ef94..8c719dafa85d0 100755 --- a/ci/scripts/run-backfill-tests.sh +++ b/ci/scripts/run-backfill-tests.sh @@ -422,6 +422,16 @@ test_cross_db_snapshot_backfill() { kill_cluster } +test_locality_backfill() { + echo "--- e2e, locality backfill test, $RUNTIME_CLUSTER_PROFILE" + + risedev ci-start $RUNTIME_CLUSTER_PROFILE + + sqllogictest -p 4566 -d dev 'e2e_test/backfill/locality_backfill/basic.slt' + + kill_cluster +} + main() { set -euo pipefail test_snapshot_and_upstream_read @@ -433,6 +443,7 @@ main() { test_scale_in test_cross_db_snapshot_backfill + test_locality_backfill # Only if profile is "ci-release", run it. if [[ ${profile:-} == "ci-release" ]]; then From 679e2b1e8204d2bdf7c17b6e199345884d7fdd4c Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Thu, 9 Oct 2025 11:06:11 +0800 Subject: [PATCH 34/36] fix fields_pretty --- .../testdata/output/locality_backfill.yaml | 8 +++--- .../plan_node/generic/locality_provider.rs | 25 +++++++------------ .../plan_node/stream_locality_provider.rs | 11 ++++++-- 3 files changed, 22 insertions(+), 22 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/locality_backfill.yaml b/src/frontend/planner_test/tests/testdata/output/locality_backfill.yaml index 9070cb62dfb10..98126aa4336d7 100644 --- a/src/frontend/planner_test/tests/testdata/output/locality_backfill.yaml +++ b/src/frontend/planner_test/tests/testdata/output/locality_backfill.yaml @@ -7,7 +7,7 @@ StreamMaterialize { columns: [count, t.b(hidden)], stream_key: [t.b], pk_columns: [t.b], pk_conflict: NoCheck } └─StreamProject { exprs: [count, t.b] } └─StreamHashAgg { group_key: [t.b], aggs: [count] } - └─StreamLocalityProvider { locality_columns: [0] } + └─StreamLocalityProvider { locality_columns: [t.b] } └─StreamExchange { dist: HashShard(t.b) } └─StreamTableScan { table: t, columns: [t.b, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - sql: | @@ -19,14 +19,14 @@ StreamMaterialize { columns: [count, t1.b(hidden)], stream_key: [t1.b], pk_columns: [t1.b], pk_conflict: NoCheck } └─StreamProject { exprs: [count, t1.b] } └─StreamHashAgg { group_key: [t1.b], aggs: [count] } - └─StreamLocalityProvider { locality_columns: [0] } + └─StreamLocalityProvider { locality_columns: [t1.b] } └─StreamExchange { dist: HashShard(t1.b) } └─StreamHashJoin { type: Inner, predicate: t1.a = t2.a, output: [t1.b, t1._row_id, t1.a, t2._row_id] } ├─StreamExchange { dist: HashShard(t1.a) } - │ └─StreamLocalityProvider { locality_columns: [0] } + │ └─StreamLocalityProvider { locality_columns: [t1.a] } │ └─StreamExchange { dist: HashShard(t1.a) } │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard(t2.a) } - └─StreamLocalityProvider { locality_columns: [0] } + └─StreamLocalityProvider { locality_columns: [t2.a] } └─StreamExchange { dist: HashShard(t2.a) } └─StreamTableScan { table: t2, columns: [t2.a, t2._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t2._row_id], pk: [_row_id], dist: UpstreamHashShard(t2._row_id) } diff --git a/src/frontend/src/optimizer/plan_node/generic/locality_provider.rs b/src/frontend/src/optimizer/plan_node/generic/locality_provider.rs index 017c52a7318cc..f1268ede8d886 100644 --- a/src/frontend/src/optimizer/plan_node/generic/locality_provider.rs +++ b/src/frontend/src/optimizer/plan_node/generic/locality_provider.rs @@ -12,9 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::fmt; - -use risingwave_common::catalog::Schema; +use pretty_xmlish::Pretty; +use risingwave_common::catalog::{FieldDisplay, Schema}; use super::{GenericPlanNode, GenericPlanRef, impl_distill_unit_from_fields}; use crate::expr::ExprRewriter; @@ -30,16 +29,6 @@ pub struct LocalityProvider { pub locality_columns: Vec, } -impl fmt::Display for LocalityProvider { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!( - f, - "LocalityProvider {{ locality_columns: {:?} }}", - self.locality_columns - ) - } -} - impl LocalityProvider { pub fn new(input: PlanRef, locality_columns: Vec) -> Self { Self { @@ -48,9 +37,13 @@ impl LocalityProvider { } } - pub fn fields_pretty<'a>(&self) -> Vec<(&'a str, pretty_xmlish::Pretty<'a>)> { - let locality_columns_str = format!("{:?}", self.locality_columns); - vec![("locality_columns", locality_columns_str.into())] + pub fn fields_pretty<'a>(&self) -> Vec<(&'a str, Pretty<'a>)> { + let locality_columns_display = self + .locality_columns + .iter() + .map(|&i| Pretty::display(&FieldDisplay(self.input.schema().fields.get(i).unwrap()))) + .collect(); + vec![("locality_columns", Pretty::Array(locality_columns_display))] } } diff --git a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs index ec46c83ab846c..f8a7779a9dfab 100644 --- a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs +++ b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs @@ -13,6 +13,7 @@ // limitations under the License. use itertools::Itertools; +use pretty_xmlish::XmlNode; use risingwave_common::catalog::Field; use risingwave_common::hash::VirtualNode; use risingwave_common::types::DataType; @@ -21,7 +22,7 @@ use risingwave_pb::stream_plan::LocalityProviderNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::stream::prelude::*; -use super::utils::{TableCatalogBuilder, impl_distill_by_unit}; +use super::utils::{Distill, TableCatalogBuilder, childless_record}; use super::{ExprRewritable, PlanTreeNodeUnary, StreamNode, StreamPlanRef as PlanRef, generic}; use crate::TableCatalog; use crate::catalog::TableId; @@ -88,7 +89,13 @@ impl PlanTreeNodeUnary for StreamLocalityProvider { } impl_plan_tree_node_for_unary! { Stream, StreamLocalityProvider } -impl_distill_by_unit!(StreamLocalityProvider, core, "StreamLocalityProvider"); + +impl Distill for StreamLocalityProvider { + fn distill<'a>(&self) -> XmlNode<'a> { + let vec = self.core.fields_pretty(); + childless_record("StreamLocalityProvider", vec) + } +} impl StreamNode for StreamLocalityProvider { fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> PbNodeBody { From a7176c044e6b07292213e84d495ff727a73c5e74 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Thu, 9 Oct 2025 11:26:51 +0800 Subject: [PATCH 35/36] expect_stream_key --- .../plan_node/stream_locality_provider.rs | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs index f8a7779a9dfab..15db8ef9bc152 100644 --- a/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs +++ b/src/frontend/src/optimizer/plan_node/stream_locality_provider.rs @@ -149,10 +149,8 @@ impl StreamLocalityProvider { catalog_builder.add_order_column(*locality_col_idx, OrderType::ascending()); } // add streaming key of the input as the rest of the primary key - if let Some(stream_key) = input.stream_key() { - for &key_col_idx in stream_key { - catalog_builder.add_order_column(key_col_idx, OrderType::ascending()); - } + for &key_col_idx in input.expect_stream_key() { + catalog_builder.add_order_column(key_col_idx, OrderType::ascending()); } catalog_builder.set_value_indices((0..input_schema.len()).collect()); @@ -184,11 +182,9 @@ impl StreamLocalityProvider { } // Add stream key columns as part of primary key (excluding those already added as locality columns) - if let Some(stream_key) = input.stream_key() { - for &key_col_idx in stream_key { - let field = &input_schema.fields[key_col_idx]; - catalog_builder.add_column(field); - } + for &key_col_idx in input.expect_stream_key() { + let field = &input_schema.fields[key_col_idx]; + catalog_builder.add_column(field); } // Add backfill_finished column From a49a912ffbc1c87e062ba7d4e85ae0666b132e63 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Thu, 9 Oct 2025 17:01:58 +0800 Subject: [PATCH 36/36] add no shuffle test --- .../tests/testdata/input/locality_backfill.yaml | 6 ++++++ .../tests/testdata/output/locality_backfill.yaml | 11 +++++++++++ 2 files changed, 17 insertions(+) diff --git a/src/frontend/planner_test/tests/testdata/input/locality_backfill.yaml b/src/frontend/planner_test/tests/testdata/input/locality_backfill.yaml index 4d63c72fea2fb..6b197d70b7e9e 100644 --- a/src/frontend/planner_test/tests/testdata/input/locality_backfill.yaml +++ b/src/frontend/planner_test/tests/testdata/input/locality_backfill.yaml @@ -9,5 +9,11 @@ create table t1 (a int, b int, c int); create table t2 (a int, b int, c int); select count(*) from t1 join t2 on t1.a = t2.a group by t1.b; + expected_outputs: + - stream_plan +- sql: | + set enable_locality_backfill = true; + create table t (a int, b int, c int, primary key (b, a)); + select count(*) from t group by a, b; expected_outputs: - stream_plan \ No newline at end of file diff --git a/src/frontend/planner_test/tests/testdata/output/locality_backfill.yaml b/src/frontend/planner_test/tests/testdata/output/locality_backfill.yaml index 98126aa4336d7..2c72134309bec 100644 --- a/src/frontend/planner_test/tests/testdata/output/locality_backfill.yaml +++ b/src/frontend/planner_test/tests/testdata/output/locality_backfill.yaml @@ -30,3 +30,14 @@ └─StreamLocalityProvider { locality_columns: [t2.a] } └─StreamExchange { dist: HashShard(t2.a) } └─StreamTableScan { table: t2, columns: [t2.a, t2._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t2._row_id], pk: [_row_id], dist: UpstreamHashShard(t2._row_id) } +- sql: | + set enable_locality_backfill = true; + create table t (a int, b int, c int, primary key (b, a)); + select count(*) from t group by a, b; + stream_plan: |- + StreamMaterialize { columns: [count, t.a(hidden), t.b(hidden)], stream_key: [t.a, t.b], pk_columns: [t.a, t.b], pk_conflict: NoCheck } + └─StreamProject { exprs: [count, t.a, t.b] } + └─StreamHashAgg { group_key: [t.a, t.b], aggs: [count] } + └─StreamLocalityProvider { locality_columns: [t.a, t.b] } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(t.b, t.a) } + └─StreamTableScan { table: t, columns: [t.a, t.b], stream_scan_type: ArrangementBackfill, stream_key: [t.b, t.a], pk: [b, a], dist: UpstreamHashShard(t.b, t.a) }