diff --git a/e2e_test/ddl/search_path.slt b/e2e_test/ddl/search_path.slt index 06db7f3f45c90..e01ed5d602936 100644 --- a/e2e_test/ddl/search_path.slt +++ b/e2e_test/ddl/search_path.slt @@ -76,6 +76,11 @@ select a from test order by a; 1 2 +# Issue #15195 +# index shall be created in `search_path_test2` (same as table) rather than `search_path_test1` (first in path) +statement ok +create index if not exists index1_test_a on test(a); + statement ok drop table test; diff --git a/src/compute/tests/cdc_tests.rs b/src/compute/tests/cdc_tests.rs index 7fc3641770a9e..30849c40d4f84 100644 --- a/src/compute/tests/cdc_tests.rs +++ b/src/compute/tests/cdc_tests.rs @@ -44,31 +44,22 @@ use risingwave_stream::error::StreamResult; use risingwave_stream::executor::monitor::StreamingMetrics; use risingwave_stream::executor::test_utils::MockSource; use risingwave_stream::executor::{ - expect_first_barrier, ActorContext, AddMutation, Barrier, BoxedExecutor as StreamBoxedExecutor, - BoxedMessageStream, CdcBackfillExecutor, Executor, ExecutorInfo, ExternalStorageTable, - MaterializeExecutor, Message, Mutation, PkIndices, PkIndicesRef, StreamExecutorError, + expect_first_barrier, ActorContext, AddMutation, Barrier, BoxedMessageStream, + CdcBackfillExecutor, Execute, Executor as StreamExecutor, ExecutorInfo, ExternalStorageTable, + MaterializeExecutor, Message, Mutation, StreamExecutorError, }; // mock upstream binlog offset starting from "1.binlog, pos=0" pub struct MockOffsetGenExecutor { - upstream: Option<StreamBoxedExecutor>, - - schema: Schema, - - pk_indices: PkIndices, - - identity: String, + upstream: Option<StreamExecutor>, start_offset: u32, } impl MockOffsetGenExecutor { - pub fn new(upstream: StreamBoxedExecutor, schema: Schema, pk_indices: PkIndices) -> Self { + pub fn new(upstream: StreamExecutor) -> Self { Self { upstream: Some(upstream), - schema, - pk_indices, - identity: "MockOffsetGenExecutor".to_string(), start_offset: 0, } } @@ -131,22 +122,10 @@ impl MockOffsetGenExecutor { } } -impl Executor for MockOffsetGenExecutor { +impl Execute for MockOffsetGenExecutor { fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.pk_indices - } - - fn identity(&self) -> &str { - &self.identity - } } #[tokio::test] @@ -154,21 +133,26 @@ async fn test_cdc_backfill() -> StreamResult<()> { use risingwave_common::types::DataType; let memory_state_store = MemoryStateStore::new(); - let table_id = TableId::new(1002); - let schema = Schema::new(vec![ - Field::unnamed(DataType::Jsonb), // payload - Field::unnamed(DataType::Varchar), // _rw_offset - ]); - let column_ids = vec![0.into(), 1.into()]; - - let pk_indices = vec![0]; - - let (mut tx, source) = MockSource::channel(schema.clone(), pk_indices.clone()); - let _actor_ctx = ActorContext::for_test(0x3a3a3a); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor( + Schema::new(vec![ + Field::unnamed(DataType::Jsonb), // payload + ]), + vec![0], + ); // mock upstream offset (start from "1.binlog, pos=0") for ingested chunks - let mock_offset_executor = - MockOffsetGenExecutor::new(Box::new(source), schema.clone(), pk_indices.clone()); + let mock_offset_executor = StreamExecutor::new( + ExecutorInfo { + schema: Schema::new(vec![ + Field::unnamed(DataType::Jsonb), // payload + Field::unnamed(DataType::Varchar), // _rw_offset + ]), + pk_indices: vec![0], + identity: "MockOffsetGenExecutor".to_string(), + }, + MockOffsetGenExecutor::new(source).boxed(), + ); let binlog_file = String::from("1.binlog"); // mock binlog watermarks for backfill @@ -188,13 +172,15 @@ async fn test_cdc_backfill() -> StreamResult<()> { Field::with_name(DataType::Int64, "id"), // primary key Field::with_name(DataType::Float64, "price"), ]); + let table_pk_indices = vec![0]; + let table_pk_order_types = vec![OrderType::ascending()]; let external_table = ExternalStorageTable::new( - table_id, + TableId::new(1234), table_name, ExternalTableReaderImpl::Mock(MockExternalTableReader::new(binlog_watermarks)), table_schema.clone(), - vec![OrderType::ascending()], - pk_indices, + table_pk_order_types, + table_pk_indices.clone(), vec![0, 1], ); @@ -224,32 +210,35 @@ async fn test_cdc_backfill() -> StreamResult<()> { vec![0_usize], ) .await; - let info = ExecutorInfo { - schema: table_schema.clone(), - pk_indices: vec![0], - identity: "CdcBackfillExecutor".to_string(), - }; - let cdc_backfill = CdcBackfillExecutor::new( - ActorContext::for_test(actor_id), - info, - external_table, - Box::new(mock_offset_executor), - vec![0, 1], - None, - Arc::new(StreamingMetrics::unused()), - state_table, - 4, // 4 rows in a snapshot chunk - false, + + let cdc_backfill = StreamExecutor::new( + ExecutorInfo { + schema: table_schema.clone(), + pk_indices: table_pk_indices, + identity: "CdcBackfillExecutor".to_string(), + }, + CdcBackfillExecutor::new( + ActorContext::for_test(actor_id), + external_table, + mock_offset_executor, + vec![0, 1], + None, + Arc::new(StreamingMetrics::unused()), + state_table, + 4, // 4 rows in a snapshot chunk + false, + ) + .boxed(), ); // Create a `MaterializeExecutor` to write the changes to storage. + let materialize_table_id = TableId::new(5678); let mut materialize = MaterializeExecutor::for_test( - Box::new(cdc_backfill), + cdc_backfill, memory_state_store.clone(), - table_id, + materialize_table_id, vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids.clone(), - 4, + vec![0.into(), 1.into()], Arc::new(AtomicU64::new(0)), ConflictBehavior::Overwrite, ) @@ -354,7 +343,7 @@ async fn test_cdc_backfill() -> StreamResult<()> { // Since we have not polled `Materialize`, we cannot scan anything from this table let table = StorageTable::for_test( memory_state_store.clone(), - table_id, + materialize_table_id, column_descs.clone(), vec![OrderType::ascending()], vec![0], diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index 21fbe90b514e0..5e9fb3dfea86a 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -56,7 +56,7 @@ use risingwave_stream::executor::monitor::StreamingMetrics; use risingwave_stream::executor::row_id_gen::RowIdGenExecutor; use risingwave_stream::executor::source_executor::SourceExecutor; use risingwave_stream::executor::{ - ActorContext, Barrier, Executor, ExecutorInfo, MaterializeExecutor, Message, PkIndices, + ActorContext, Barrier, Execute, Executor, ExecutorInfo, MaterializeExecutor, Message, PkIndices, }; use tokio::sync::mpsc::unbounded_channel; @@ -162,56 +162,58 @@ async fn test_table_materialize() -> StreamResult<()> { let system_params_manager = LocalSystemParamsManager::for_test(); // Create a `SourceExecutor` to read the changes. - let source_executor = SourceExecutor::<PanicStateStore>::new( - actor_ctx.clone(), + let source_executor = Executor::new( ExecutorInfo { schema: all_schema.clone(), pk_indices: pk_indices.clone(), identity: format!("SourceExecutor {:X}", 1), }, - None, // There is no external stream source. - Arc::new(StreamingMetrics::unused()), - barrier_rx, - system_params_manager.get_params(), - SourceCtrlOpts::default(), - ConnectorParams::default(), + SourceExecutor::<PanicStateStore>::new( + actor_ctx.clone(), + None, // There is no external stream source. + Arc::new(StreamingMetrics::unused()), + barrier_rx, + system_params_manager.get_params(), + SourceCtrlOpts::default(), + ConnectorParams::default(), + ) + .boxed(), ); // Create a `DmlExecutor` to accept data change from users. - let dml_executor = DmlExecutor::new( + let dml_executor = Executor::new( ExecutorInfo { schema: all_schema.clone(), pk_indices: pk_indices.clone(), identity: format!("DmlExecutor {:X}", 2), }, - Box::new(source_executor), - dml_manager.clone(), - table_id, - INITIAL_TABLE_VERSION_ID, - column_descs.clone(), - 1024, + DmlExecutor::new( + source_executor, + dml_manager.clone(), + table_id, + INITIAL_TABLE_VERSION_ID, + column_descs.clone(), + 1024, + ) + .boxed(), ); - let row_id_gen_executor = RowIdGenExecutor::new( - actor_ctx, + let row_id_gen_executor = Executor::new( ExecutorInfo { schema: all_schema.clone(), pk_indices: pk_indices.clone(), identity: format!("RowIdGenExecutor {:X}", 3), }, - Box::new(dml_executor), - row_id_index, - vnodes, + RowIdGenExecutor::new(actor_ctx, dml_executor, row_id_index, vnodes).boxed(), ); // Create a `MaterializeExecutor` to write the changes to storage. let mut materialize = MaterializeExecutor::for_test( - Box::new(row_id_gen_executor), + row_id_gen_executor, memory_state_store.clone(), table_id, vec![ColumnOrder::new(0, OrderType::ascending())], all_column_ids.clone(), - 4, Arc::new(AtomicU64::new(0)), ConflictBehavior::NoCheck, ) diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index af9d9f52b1752..c687f56fad8e8 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -45,15 +45,11 @@ use crate::session::SessionImpl; use crate::stream_fragmenter::build_graph; use crate::TableCatalog; -pub(crate) fn gen_create_index_plan( +pub(crate) fn resolve_index_schema( session: &SessionImpl, - context: OptimizerContextRef, index_name: ObjectName, table_name: ObjectName, - columns: Vec<OrderByExpr>, - include: Vec<Ident>, - distributed_by: Vec<ast::Expr>, -) -> Result<(PlanRef, PbTable, PbIndex)> { +) -> Result<(String, Arc<TableCatalog>, String)> { let db_name = session.database(); let (schema_name, table_name) = Binder::resolve_schema_qualified_name(db_name, table_name)?; let search_path = session.config().search_path(); @@ -63,12 +59,22 @@ pub(crate) fn gen_create_index_plan( let index_table_name = Binder::resolve_index_name(index_name)?; let catalog_reader = session.env().catalog_reader(); - let (table, schema_name) = { - let read_guard = catalog_reader.read_guard(); - let (table, schema_name) = - read_guard.get_table_by_name(db_name, schema_path, &table_name)?; - (table.clone(), schema_name.to_string()) - }; + let read_guard = catalog_reader.read_guard(); + let (table, schema_name) = read_guard.get_table_by_name(db_name, schema_path, &table_name)?; + Ok((schema_name.to_string(), table.clone(), index_table_name)) +} + +pub(crate) fn gen_create_index_plan( + session: &SessionImpl, + context: OptimizerContextRef, + schema_name: String, + table: Arc<TableCatalog>, + index_table_name: String, + columns: Vec<OrderByExpr>, + include: Vec<Ident>, + distributed_by: Vec<ast::Expr>, +) -> Result<(PlanRef, PbTable, PbIndex)> { + let table_name = table.name.clone(); if table.is_index() { return Err( @@ -404,22 +410,27 @@ pub async fn handle_create_index( let session = handler_args.session.clone(); let (graph, index_table, index) = { - { - if let Either::Right(resp) = session.check_relation_name_duplicated( - index_name.clone(), - StatementType::CREATE_INDEX, - if_not_exists, - )? { - return Ok(resp); - } + let (schema_name, table, index_table_name) = + resolve_index_schema(&session, index_name, table_name)?; + let qualified_index_name = ObjectName(vec![ + Ident::with_quote_unchecked('"', &schema_name), + Ident::with_quote_unchecked('"', &index_table_name), + ]); + if let Either::Right(resp) = session.check_relation_name_duplicated( + qualified_index_name, + StatementType::CREATE_INDEX, + if_not_exists, + )? { + return Ok(resp); } let context = OptimizerContext::from_handler_args(handler_args); let (plan, index_table, index) = gen_create_index_plan( &session, context.into(), - index_name.clone(), - table_name, + schema_name, + table, + index_table_name, columns, include, distributed_by, @@ -437,7 +448,7 @@ pub async fn handle_create_index( tracing::trace!( "name={}, graph=\n{}", - index_name, + index.name, serde_json::to_string_pretty(&graph).unwrap() ); diff --git a/src/frontend/src/handler/explain.rs b/src/frontend/src/handler/explain.rs index b966cca8f50cf..b46882156a24c 100644 --- a/src/frontend/src/handler/explain.rs +++ b/src/frontend/src/handler/explain.rs @@ -18,7 +18,7 @@ use risingwave_common::types::Fields; use risingwave_sqlparser::ast::{ExplainOptions, ExplainType, Statement}; use thiserror_ext::AsReport; -use super::create_index::gen_create_index_plan; +use super::create_index::{gen_create_index_plan, resolve_index_schema}; use super::create_mv::gen_create_mv_plan; use super::create_sink::{gen_sink_plan, get_partition_compute_info}; use super::create_table::ColumnIdGenerator; @@ -133,15 +133,20 @@ async fn do_handle_explain( include, distributed_by, .. - } => gen_create_index_plan( - &session, - context.clone(), - name, - table_name, - columns, - include, - distributed_by, - ) + } => { + let (schema_name, table, index_table_name) = + resolve_index_schema(&session, name, table_name)?; + gen_create_index_plan( + &session, + context.clone(), + schema_name, + table, + index_table_name, + columns, + include, + distributed_by, + ) + } .map(|x| x.0), // -- Batch Queries -- diff --git a/src/meta/model_v2/src/table.rs b/src/meta/model_v2/src/table.rs index 06710d42d9d25..0039b9cccc2e0 100644 --- a/src/meta/model_v2/src/table.rs +++ b/src/meta/model_v2/src/table.rs @@ -24,7 +24,7 @@ use crate::{ TableId, TableVersion, }; -#[derive(Clone, Debug, PartialEq, Copy, Eq, EnumIter, DeriveActiveEnum)] +#[derive(Clone, Debug, PartialEq, Hash, Copy, Eq, EnumIter, DeriveActiveEnum)] #[sea_orm(rs_type = "String", db_type = "String(None)")] pub enum TableType { #[sea_orm(string_value = "TABLE")] diff --git a/src/meta/node/src/server.rs b/src/meta/node/src/server.rs index 4792293863237..3397317fe1939 100644 --- a/src/meta/node/src/server.rs +++ b/src/meta/node/src/server.rs @@ -478,20 +478,13 @@ pub async fn start_service_as_election_leader( prometheus_http_query::Client::from_str(x).unwrap() }); let prometheus_selector = opts.prometheus_selector.unwrap_or_default(); - let diagnose_command = match &metadata_manager { - MetadataManager::V1(mgr) => Some(Arc::new( - risingwave_meta::manager::diagnose::DiagnoseCommand::new( - mgr.cluster_manager.clone(), - mgr.catalog_manager.clone(), - mgr.fragment_manager.clone(), - hummock_manager.clone(), - env.event_log_manager_ref(), - prometheus_client.clone(), - prometheus_selector.clone(), - ), - )), - MetadataManager::V2(_) => None, - }; + let diagnose_command = Arc::new(risingwave_meta::manager::diagnose::DiagnoseCommand::new( + metadata_manager.clone(), + hummock_manager.clone(), + env.event_log_manager_ref(), + prometheus_client.clone(), + prometheus_selector.clone(), + )); let trace_state = otlp_embedded::State::new(otlp_embedded::Config { max_length: opts.cached_traces_num, diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index a1efaa756bb44..fae19874ab407 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -2446,6 +2446,18 @@ impl CatalogController { } } +/// `CatalogStats` is a struct to store the statistics of all catalogs. +pub struct CatalogStats { + pub table_num: u64, + pub mview_num: u64, + pub index_num: u64, + pub source_num: u64, + pub sink_num: u64, + pub function_num: u64, + pub streaming_job_num: u64, + pub actor_num: u64, +} + impl CatalogControllerInner { pub async fn snapshot(&self) -> MetaResult<(Catalog, Vec<PbUserInfo>)> { let databases = self.list_databases().await?; @@ -2476,6 +2488,40 @@ impl CatalogControllerInner { )) } + pub async fn stats(&self) -> MetaResult<CatalogStats> { + let mut table_num_map: HashMap<_, _> = Table::find() + .select_only() + .column(table::Column::TableType) + .column_as(table::Column::TableId.count(), "num") + .group_by(table::Column::TableType) + .having(table::Column::TableType.ne(TableType::Internal)) + .into_tuple::<(TableType, i64)>() + .all(&self.db) + .await? + .into_iter() + .map(|(table_type, num)| (table_type, num as u64)) + .collect(); + + let source_num = Source::find().count(&self.db).await?; + let sink_num = Sink::find().count(&self.db).await?; + let function_num = Function::find().count(&self.db).await?; + let streaming_job_num = StreamingJob::find().count(&self.db).await?; + let actor_num = Actor::find().count(&self.db).await?; + + Ok(CatalogStats { + table_num: table_num_map.remove(&TableType::Table).unwrap_or(0), + mview_num: table_num_map + .remove(&TableType::MaterializedView) + .unwrap_or(0), + index_num: table_num_map.remove(&TableType::Index).unwrap_or(0), + source_num, + sink_num, + function_num, + streaming_job_num, + actor_num, + }) + } + async fn list_databases(&self) -> MetaResult<Vec<PbDatabase>> { let db_objs = Database::find() .find_also_related(Object) diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index 833e642a83e74..c9beada284d5d 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -702,6 +702,23 @@ impl CatalogController { Ok(count > 0) } + pub async fn worker_actor_count(&self) -> MetaResult<HashMap<WorkerId, usize>> { + let inner = self.inner.read().await; + let actor_cnt: Vec<(WorkerId, i64)> = Actor::find() + .select_only() + .column(actor::Column::WorkerId) + .column_as(actor::Column::ActorId.count(), "count") + .group_by(actor::Column::WorkerId) + .into_tuple() + .all(&inner.db) + .await?; + + Ok(actor_cnt + .into_iter() + .map(|(worker_id, count)| (worker_id, count as usize)) + .collect()) + } + // TODO: This function is too heavy, we should avoid using it and implement others on demand. pub async fn table_fragments(&self) -> MetaResult<BTreeMap<ObjectId, PbTableFragments>> { let inner = self.inner.read().await; diff --git a/src/meta/src/dashboard/mod.rs b/src/meta/src/dashboard/mod.rs index b5104e557a1b2..b4ae8e2c04b36 100644 --- a/src/meta/src/dashboard/mod.rs +++ b/src/meta/src/dashboard/mod.rs @@ -47,7 +47,7 @@ pub struct DashboardService { pub metadata_manager: MetadataManager, pub compute_clients: ComputeClientPool, pub ui_path: Option<String>, - pub diagnose_command: Option<DiagnoseCommandRef>, + pub diagnose_command: DiagnoseCommandRef, pub trace_state: otlp_embedded::StateRef, } @@ -355,13 +355,7 @@ pub(super) mod handlers { } pub async fn diagnose(Extension(srv): Extension<Service>) -> Result<String> { - let report = if let Some(cmd) = &srv.diagnose_command { - cmd.report().await - } else { - "Not supported in sql-backend".to_string() - }; - - Ok(report) + Ok(srv.diagnose_command.report().await) } pub async fn get_back_pressure( diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index fdae16efe5a7b..ecfac68833d0e 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -858,6 +858,22 @@ impl FragmentManager { actor_maps } + pub async fn node_actor_count(&self) -> HashMap<WorkerId, usize> { + let mut actor_count = HashMap::new(); + + let map = &self.core.read().await.table_fragments; + for fragments in map.values() { + for actor_status in fragments.actor_status.values() { + if let Some(pu) = &actor_status.parallel_unit { + let e = actor_count.entry(pu.worker_node_id).or_insert(0); + *e += 1; + } + } + } + + actor_count + } + // edit the `rate_limit` of the `Source` node in given `source_id`'s fragments // return the actor_ids to be applied pub async fn update_source_rate_limit_by_source_id( diff --git a/src/meta/src/manager/diagnose.rs b/src/meta/src/manager/diagnose.rs index a8c1d050ab748..76184de9eb7ce 100644 --- a/src/meta/src/manager/diagnose.rs +++ b/src/meta/src/manager/diagnose.rs @@ -13,7 +13,7 @@ // limitations under the License. use std::cmp::{Ordering, Reverse}; -use std::collections::{BinaryHeap, HashMap}; +use std::collections::BinaryHeap; use std::fmt::Write; use std::sync::Arc; @@ -27,17 +27,16 @@ use risingwave_pb::meta::EventLog; use risingwave_pb::monitor_service::StackTraceResponse; use risingwave_rpc_client::ComputeClientPool; use serde_json::json; +use thiserror_ext::AsReport; use crate::hummock::HummockManagerRef; use crate::manager::event_log::EventLogMangerRef; -use crate::manager::{CatalogManagerRef, ClusterManagerRef, FragmentManagerRef}; +use crate::manager::MetadataManager; pub type DiagnoseCommandRef = Arc<DiagnoseCommand>; pub struct DiagnoseCommand { - cluster_manager: ClusterManagerRef, - catalog_manager: CatalogManagerRef, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, hummock_manger: HummockManagerRef, event_log_manager: EventLogMangerRef, prometheus_client: Option<prometheus_http_query::Client>, @@ -46,18 +45,14 @@ pub struct DiagnoseCommand { impl DiagnoseCommand { pub fn new( - cluster_manager: ClusterManagerRef, - catalog_manager: CatalogManagerRef, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, hummock_manger: HummockManagerRef, event_log_manager: EventLogMangerRef, prometheus_client: Option<prometheus_http_query::Client>, prometheus_selector: String, ) -> Self { Self { - cluster_manager, - catalog_manager, - fragment_manager, + metadata_manager, hummock_manger, event_log_manager, prometheus_client, @@ -90,49 +85,60 @@ impl DiagnoseCommand { #[cfg_attr(coverage, coverage(off))] async fn write_catalog(&self, s: &mut String) { + match &self.metadata_manager { + MetadataManager::V1(_) => self.write_catalog_v1(s).await, + MetadataManager::V2(_) => self.write_catalog_v2(s).await, + } + } + + #[cfg_attr(coverage, coverage(off))] + async fn write_catalog_v1(&self, s: &mut String) { + let mgr = self.metadata_manager.as_v1_ref(); let _ = writeln!(s, "number of fragment: {}", self.fragment_num().await); let _ = writeln!(s, "number of actor: {}", self.actor_num().await); let _ = writeln!( s, "number of source: {}", - self.catalog_manager.source_count().await + mgr.catalog_manager.source_count().await ); let _ = writeln!( s, "number of table: {}", - self.catalog_manager.table_count().await + mgr.catalog_manager.table_count().await ); let _ = writeln!( s, "number of materialized view: {}", - self.catalog_manager.materialized_view_count().await + mgr.catalog_manager.materialized_view_count().await ); let _ = writeln!( s, "number of sink: {}", - self.catalog_manager.sink_count().await + mgr.catalog_manager.sink_count().await ); let _ = writeln!( s, "number of index: {}", - self.catalog_manager.index_count().await + mgr.catalog_manager.index_count().await ); let _ = writeln!( s, "number of function: {}", - self.catalog_manager.function_count().await + mgr.catalog_manager.function_count().await ); } #[cfg_attr(coverage, coverage(off))] async fn fragment_num(&self) -> usize { - let core = self.fragment_manager.get_fragment_read_guard().await; + let mgr = self.metadata_manager.as_v1_ref(); + let core = mgr.fragment_manager.get_fragment_read_guard().await; core.table_fragments().len() } #[cfg_attr(coverage, coverage(off))] async fn actor_num(&self) -> usize { - let core = self.fragment_manager.get_fragment_read_guard().await; + let mgr = self.metadata_manager.as_v1_ref(); + let core = mgr.fragment_manager.get_fragment_read_guard().await; core.table_fragments() .values() .map(|t| t.actor_status.len()) @@ -140,25 +146,38 @@ impl DiagnoseCommand { } #[cfg_attr(coverage, coverage(off))] - async fn write_worker_nodes(&self, s: &mut String) { - let mut worker_actor_count: HashMap<u32, usize> = HashMap::new(); - for f in self - .fragment_manager - .get_fragment_read_guard() - .await - .table_fragments() - .values() - { - for a in f.actor_status.values() { - if let Some(pu) = &a.parallel_unit { - let e = worker_actor_count.entry(pu.worker_node_id).or_insert(0); - *e += 1; - } + async fn write_catalog_v2(&self, s: &mut String) { + let mgr = self.metadata_manager.as_v2_ref(); + let guard = mgr.catalog_controller.get_inner_read_guard().await; + let stat = match guard.stats().await { + Ok(stat) => stat, + Err(err) => { + tracing::warn!(error=?err.as_report(), "failed to get catalog stats"); + return; } - } + }; + let _ = writeln!(s, "number of fragment: {}", stat.streaming_job_num); + let _ = writeln!(s, "number of actor: {}", stat.actor_num); + let _ = writeln!(s, "number of source: {}", stat.source_num); + let _ = writeln!(s, "number of table: {}", stat.table_num); + let _ = writeln!(s, "number of materialized view: {}", stat.mview_num); + let _ = writeln!(s, "number of sink: {}", stat.sink_num); + let _ = writeln!(s, "number of index: {}", stat.index_num); + let _ = writeln!(s, "number of function: {}", stat.function_num); + } + + #[cfg_attr(coverage, coverage(off))] + async fn write_worker_nodes(&self, s: &mut String) { + let Ok(worker_actor_count) = self.metadata_manager.worker_actor_count().await else { + tracing::warn!("failed to get worker actor count"); + return; + }; use comfy_table::{Row, Table}; - let worker_nodes = self.cluster_manager.list_worker_node(None, None).await; + let Ok(worker_nodes) = self.metadata_manager.list_worker_node(None, None).await else { + tracing::warn!("failed to get worker nodes"); + return; + }; let mut table = Table::new(); table.set_header({ let mut row = Row::new(); @@ -636,10 +655,14 @@ impl DiagnoseCommand { #[cfg_attr(coverage, coverage(off))] async fn write_await_tree(&self, s: &mut String) { // Most lines of code are copied from dashboard::handlers::dump_await_tree_all, because the latter cannot be called directly from here. - let worker_nodes = self - .cluster_manager + let Ok(worker_nodes) = self + .metadata_manager .list_worker_node(Some(WorkerType::ComputeNode), None) - .await; + .await + else { + tracing::warn!("failed to get worker nodes"); + return; + }; let mut all = Default::default(); diff --git a/src/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs index eef1c0c101256..fd9b51fa8ba09 100644 --- a/src/meta/src/manager/metadata.rs +++ b/src/meta/src/manager/metadata.rs @@ -637,6 +637,19 @@ impl MetadataManager { } } + pub async fn worker_actor_count(&self) -> MetaResult<HashMap<WorkerId, usize>> { + match &self { + MetadataManager::V1(mgr) => Ok(mgr.fragment_manager.node_actor_count().await), + MetadataManager::V2(mgr) => { + let actor_cnt = mgr.catalog_controller.worker_actor_count().await?; + Ok(actor_cnt + .into_iter() + .map(|(id, cnt)| (id as WorkerId, cnt)) + .collect()) + } + } + } + pub async fn count_streaming_job(&self) -> MetaResult<usize> { match self { MetadataManager::V1(mgr) => Ok(mgr.fragment_manager.count_streaming_job().await), diff --git a/src/stream/benches/stream_hash_agg.rs b/src/stream/benches/stream_hash_agg.rs index 90ac7ef7725e7..2a0d1a93cc6b7 100644 --- a/src/stream/benches/stream_hash_agg.rs +++ b/src/stream/benches/stream_hash_agg.rs @@ -26,7 +26,7 @@ use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::StateStore; use risingwave_stream::executor::test_utils::agg_executor::new_boxed_hash_agg_executor; use risingwave_stream::executor::test_utils::*; -use risingwave_stream::executor::{BoxedExecutor, PkIndices}; +use risingwave_stream::executor::{Executor, PkIndices}; use tokio::runtime::Runtime; risingwave_expr_impl::enable!(); @@ -47,7 +47,7 @@ fn bench_hash_agg(c: &mut Criterion) { /// This aims to mirror `q17`'s aggregator. /// We can include more executor patterns as needed. -fn setup_bench_hash_agg<S: StateStore>(store: S) -> BoxedExecutor { +fn setup_bench_hash_agg<S: StateStore>(store: S) -> Executor { // ---- Define hash agg executor parameters ---- let input_data_types = vec![ // to_char(date_time) @@ -119,7 +119,8 @@ fn setup_bench_hash_agg<S: StateStore>(store: S) -> BoxedExecutor { ); // ---- Create MockSourceExecutor ---- - let (mut tx, source) = MockSource::channel(schema, PkIndices::new()); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, PkIndices::new()); tx.push_barrier(1, false); for chunk in chunks { tx.push_chunk(chunk); @@ -134,7 +135,7 @@ fn setup_bench_hash_agg<S: StateStore>(store: S) -> BoxedExecutor { block_on(new_boxed_hash_agg_executor( store, - Box::new(source), + source, false, agg_calls, row_count_index, @@ -146,7 +147,7 @@ fn setup_bench_hash_agg<S: StateStore>(store: S) -> BoxedExecutor { )) } -pub async fn execute_executor(executor: BoxedExecutor) { +pub async fn execute_executor(executor: Executor) { let mut stream = executor.execute(); while let Some(ret) = stream.next().await { _ = black_box(ret.unwrap()); diff --git a/src/stream/src/executor/agg_common.rs b/src/stream/src/executor/agg_common.rs index 603cb7f244f87..91c414877a76b 100644 --- a/src/stream/src/executor/agg_common.rs +++ b/src/stream/src/executor/agg_common.rs @@ -29,7 +29,7 @@ pub struct AggExecutorArgs<S: StateStore, E: AggExecutorExtraArgs> { pub version: PbAggNodeVersion, // basic - pub input: Box<dyn Executor>, + pub input: Executor, pub actor_ctx: ActorContextRef, pub info: ExecutorInfo, diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 0afea6720cad2..2dd277449be13 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -22,7 +22,6 @@ use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::{DataChunk, Op, StreamChunk}; use risingwave_common::bail; -use risingwave_common::catalog::Schema; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::OwnedRow; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; @@ -40,8 +39,8 @@ use crate::executor::backfill::utils::{ }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ - expect_first_barrier, Barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, - HashMap, Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult, + expect_first_barrier, Barrier, BoxedMessageStream, Execute, Executor, HashMap, Message, + StreamExecutorError, StreamExecutorResult, }; use crate::task::{ActorId, CreateMviewProgress}; @@ -57,7 +56,7 @@ pub struct ArrangementBackfillExecutor<S: StateStore, SD: ValueRowSerde> { upstream_table: ReplicatedStateTable<S, SD>, /// Upstream with the same schema with the upstream table. - upstream: BoxedExecutor, + upstream: Executor, /// Internal state table for persisting state of backfill state. state_table: StateTable<S>, @@ -69,8 +68,6 @@ pub struct ArrangementBackfillExecutor<S: StateStore, SD: ValueRowSerde> { actor_id: ActorId, - info: ExecutorInfo, - metrics: Arc<StreamingMetrics>, chunk_size: usize, @@ -86,9 +83,8 @@ where #[allow(clippy::too_many_arguments)] #[allow(dead_code)] pub fn new( - info: ExecutorInfo, upstream_table: ReplicatedStateTable<S, SD>, - upstream: BoxedExecutor, + upstream: Executor, state_table: StateTable<S>, output_indices: Vec<usize>, progress: CreateMviewProgress, @@ -97,7 +93,6 @@ where rate_limit: Option<usize>, ) -> Self { Self { - info, upstream_table, upstream, state_table, @@ -710,7 +705,7 @@ where } } -impl<S, SD> Executor for ArrangementBackfillExecutor<S, SD> +impl<S, SD> Execute for ArrangementBackfillExecutor<S, SD> where S: StateStore, SD: ValueRowSerde, @@ -718,16 +713,4 @@ where fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs index bb99e2ecab369..e4b83622951f6 100644 --- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -45,8 +45,8 @@ use crate::executor::backfill::utils::{ }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult, + expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, + StreamExecutorError, StreamExecutorResult, }; use crate::task::CreateMviewProgress; @@ -55,13 +55,12 @@ const METADATA_STATE_LEN: usize = 4; pub struct CdcBackfillExecutor<S: StateStore> { actor_ctx: ActorContextRef, - info: ExecutorInfo, /// The external table to be backfilled external_table: ExternalStorageTable, /// Upstream changelog stream which may contain metadata columns, e.g. `_rw_offset` - upstream: BoxedExecutor, + upstream: Executor, /// The column indices need to be forwarded to the downstream from the upstream and table scan. /// User may select a subset of columns from the upstream table. @@ -83,9 +82,8 @@ impl<S: StateStore> CdcBackfillExecutor<S> { #[allow(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, - info: ExecutorInfo, external_table: ExternalStorageTable, - upstream: BoxedExecutor, + upstream: Executor, output_indices: Vec<usize>, progress: Option<CreateMviewProgress>, metrics: Arc<StreamingMetrics>, @@ -95,7 +93,6 @@ impl<S: StateStore> CdcBackfillExecutor<S> { ) -> Self { Self { actor_ctx, - info, external_table, upstream, output_indices, @@ -609,22 +606,10 @@ fn get_rw_columns(schema: &Schema) -> Vec<SourceColumnDesc> { .collect_vec() } -impl<S: StateStore> Executor for CdcBackfillExecutor<S> { +impl<S: StateStore> Execute for CdcBackfillExecutor<S> { fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } #[cfg(test)] @@ -639,7 +624,6 @@ mod tests { use crate::executor::backfill::cdc::cdc_backfill::transform_upstream; use crate::executor::test_utils::MockSource; - use crate::executor::Executor; #[tokio::test] async fn test_transform_upstream_chunk() { @@ -649,7 +633,8 @@ mod tests { Field::unnamed(DataType::Varchar), // _rw_table_name ]); let pk_indices = vec![1]; - let (mut tx, source) = MockSource::channel(schema.clone(), pk_indices.clone()); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema.clone(), pk_indices.clone()); // let payload = r#"{"before": null,"after":{"O_ORDERKEY": 5, "O_CUSTKEY": 44485, "O_ORDERSTATUS": "F", "O_TOTALPRICE": "144659.20", "O_ORDERDATE": "1994-07-30" },"source":{"version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002", "ts_ms": 1695277757000, "snapshot": "last", "db": "mydb", "sequence": null, "table": "orders_new", "server_id": 0, "gtid": null, "file": "binlog.000008", "pos": 3693, "row": 0, "thread": null, "query": null},"op":"r","ts_ms":1695277757017,"transaction":null}"#.to_string(); let payload = r#"{ "payload": { "before": null, "after": { "O_ORDERKEY": 5, "O_CUSTKEY": 44485, "O_ORDERSTATUS": "F", "O_TOTALPRICE": "144659.20", "O_ORDERDATE": "1994-07-30" }, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002", "ts_ms": 1695277757000, "snapshot": "last", "db": "mydb", "sequence": null, "table": "orders_new", "server_id": 0, "gtid": null, "file": "binlog.000008", "pos": 3693, "row": 0, "thread": null, "query": null }, "op": "r", "ts_ms": 1695277757017, "transaction": null } }"#; diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index 824bc51e13b00..4aea8f63b26ea 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -20,7 +20,6 @@ use futures::stream::select_with_strategy; use futures::{stream, StreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::{DataChunk, Op, StreamChunk}; -use risingwave_common::catalog::Schema; use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::Datum; @@ -39,8 +38,8 @@ use crate::executor::backfill::utils::{ }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ - expect_first_barrier, Barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, - Message, Mutation, PkIndicesRef, StreamExecutorError, StreamExecutorResult, + expect_first_barrier, Barrier, BoxedMessageStream, Execute, Executor, Message, Mutation, + StreamExecutorError, StreamExecutorResult, }; use crate::task::{ActorId, CreateMviewProgress}; @@ -76,12 +75,10 @@ pub struct BackfillState { /// in the same worker, so that we can read uncommitted data from the upstream table without /// waiting. pub struct BackfillExecutor<S: StateStore> { - info: ExecutorInfo, - /// Upstream table upstream_table: StorageTable<S>, /// Upstream with the same schema with the upstream table. - upstream: BoxedExecutor, + upstream: Executor, /// Internal state table for persisting state of backfill state. state_table: Option<StateTable<S>>, @@ -110,9 +107,8 @@ where { #[allow(clippy::too_many_arguments)] pub fn new( - info: ExecutorInfo, upstream_table: StorageTable<S>, - upstream: BoxedExecutor, + upstream: Executor, state_table: Option<StateTable<S>>, output_indices: Vec<usize>, progress: CreateMviewProgress, @@ -122,7 +118,6 @@ where ) -> Self { let actor_id = progress.actor_id(); Self { - info, upstream_table, upstream, state_table, @@ -742,23 +737,11 @@ where } } -impl<S> Executor for BackfillExecutor<S> +impl<S> Execute for BackfillExecutor<S> where S: StateStore, { fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } diff --git a/src/stream/src/executor/barrier_recv.rs b/src/stream/src/executor/barrier_recv.rs index 66480ef1cb591..4ca07a80382a5 100644 --- a/src/stream/src/executor/barrier_recv.rs +++ b/src/stream/src/executor/barrier_recv.rs @@ -13,52 +13,37 @@ // limitations under the License. use futures::StreamExt; -use risingwave_common::catalog::Schema; use tokio::sync::mpsc::UnboundedReceiver; use tokio_stream::wrappers::UnboundedReceiverStream; use super::{ - ActorContext, ActorContextRef, Barrier, BoxedMessageStream, Executor, ExecutorInfo, Message, - PkIndices, PkIndicesRef, StreamExecutorError, + ActorContext, ActorContextRef, Barrier, BoxedMessageStream, Execute, Message, + StreamExecutorError, }; /// The executor only for receiving barrier from the meta service. It always resides in the leaves /// of the streaming graph. pub struct BarrierRecvExecutor { _ctx: ActorContextRef, - info: ExecutorInfo, /// The barrier receiver registered in the local barrier manager. barrier_receiver: UnboundedReceiver<Barrier>, } impl BarrierRecvExecutor { - pub fn new( - ctx: ActorContextRef, - info: ExecutorInfo, - barrier_receiver: UnboundedReceiver<Barrier>, - ) -> Self { + pub fn new(ctx: ActorContextRef, barrier_receiver: UnboundedReceiver<Barrier>) -> Self { Self { _ctx: ctx, - info, barrier_receiver, } } pub fn for_test(barrier_receiver: UnboundedReceiver<Barrier>) -> Self { - Self::new( - ActorContext::for_test(0), - ExecutorInfo { - schema: Schema::empty().clone(), - pk_indices: PkIndices::new(), - identity: "BarrierRecvExecutor".to_string(), - }, - barrier_receiver, - ) + Self::new(ActorContext::for_test(0), barrier_receiver) } } -impl Executor for BarrierRecvExecutor { +impl Execute for BarrierRecvExecutor { fn execute(self: Box<Self>) -> BoxedMessageStream { UnboundedReceiverStream::new(self.barrier_receiver) .map(|barrier| Ok(Message::Barrier(barrier))) @@ -69,18 +54,6 @@ impl Executor for BarrierRecvExecutor { })) .boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } #[cfg(test)] diff --git a/src/stream/src/executor/batch_query.rs b/src/stream/src/executor/batch_query.rs index 533a327432f43..7c92bcd732423 100644 --- a/src/stream/src/executor/batch_query.rs +++ b/src/stream/src/executor/batch_query.rs @@ -24,7 +24,7 @@ use risingwave_storage::table::collect_data_chunk; use risingwave_storage::StateStore; use super::error::StreamExecutorError; -use super::{Executor, ExecutorInfo, Message}; +use super::{Execute, Message}; use crate::executor::BoxedMessageStream; pub struct BatchQueryExecutor<S: StateStore> { @@ -34,18 +34,18 @@ pub struct BatchQueryExecutor<S: StateStore> { /// The number of tuples in one [`StreamChunk`] batch_size: usize, - info: ExecutorInfo, + schema: Schema, } impl<S> BatchQueryExecutor<S> where S: StateStore, { - pub fn new(table: StorageTable<S>, batch_size: usize, info: ExecutorInfo) -> Self { + pub fn new(table: StorageTable<S>, batch_size: usize, schema: Schema) -> Self { Self { table, batch_size, - info, + schema, } } @@ -62,7 +62,7 @@ where pin_mut!(iter); while let Some(data_chunk) = - collect_data_chunk(&mut iter, self.schema(), Some(self.batch_size)) + collect_data_chunk(&mut iter, &self.schema, Some(self.batch_size)) .instrument_await("batch_query_executor_collect_chunk") .await? { @@ -73,7 +73,7 @@ where } } -impl<S> Executor for BatchQueryExecutor<S> +impl<S> Execute for BatchQueryExecutor<S> where S: StateStore, { @@ -81,18 +81,6 @@ where unreachable!("should call `execute_with_epoch`") } - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - fn execute_with_epoch(self: Box<Self>, epoch: u64) -> BoxedMessageStream { self.execute_inner(epoch).boxed() } @@ -100,9 +88,6 @@ where #[cfg(test)] mod test { - - use std::vec; - use futures_async_stream::for_await; use super::*; @@ -114,15 +99,10 @@ mod test { let test_batch_count = 5; let table = gen_basic_table(test_batch_count * test_batch_size).await; - let info = ExecutorInfo { - schema: table.schema().clone(), - pk_indices: vec![0, 1], - identity: "BatchQuery".to_owned(), - }; - - let executor = Box::new(BatchQueryExecutor::new(table, test_batch_size, info)); - - let stream = executor.execute_with_epoch(u64::MAX); + let schema = table.schema().clone(); + let stream = BatchQueryExecutor::new(table, test_batch_size, schema) + .boxed() + .execute_with_epoch(u64::MAX); let mut batch_cnt = 0; #[for_await] diff --git a/src/stream/src/executor/chain.rs b/src/stream/src/executor/chain.rs index 0228f826a4bac..3e65bf878b552 100644 --- a/src/stream/src/executor/chain.rs +++ b/src/stream/src/executor/chain.rs @@ -14,10 +14,9 @@ use futures::StreamExt; use futures_async_stream::try_stream; -use risingwave_common::catalog::Schema; use super::error::StreamExecutorError; -use super::{expect_first_barrier, BoxedExecutor, Executor, ExecutorInfo, Message}; +use super::{expect_first_barrier, Execute, Executor, Message}; use crate::task::{ActorId, CreateMviewProgress}; /// [`ChainExecutor`] is an executor that enables synchronization between the existing stream and @@ -25,11 +24,9 @@ use crate::task::{ActorId, CreateMviewProgress}; /// feature. It pipes new data of existing MVs to newly created MV only all of the old data in the /// existing MVs are dispatched. pub struct ChainExecutor { - info: ExecutorInfo, + snapshot: Executor, - snapshot: BoxedExecutor, - - upstream: BoxedExecutor, + upstream: Executor, progress: CreateMviewProgress, @@ -41,14 +38,12 @@ pub struct ChainExecutor { impl ChainExecutor { pub fn new( - info: ExecutorInfo, - snapshot: BoxedExecutor, - upstream: BoxedExecutor, + snapshot: Executor, + upstream: Executor, progress: CreateMviewProgress, upstream_only: bool, ) -> Self { Self { - info, snapshot, upstream, actor_id: progress.actor_id(), @@ -104,22 +99,10 @@ impl ChainExecutor { } } -impl Executor for ChainExecutor { +impl Execute for ChainExecutor { fn execute(self: Box<Self>) -> super::BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } #[cfg(test)] @@ -135,9 +118,7 @@ mod test { use super::ChainExecutor; use crate::executor::test_utils::MockSource; - use crate::executor::{ - AddMutation, Barrier, Executor, ExecutorInfo, Message, Mutation, PkIndices, - }; + use crate::executor::{AddMutation, Barrier, Execute, Message, Mutation, PkIndices}; use crate::task::{CreateMviewProgress, LocalBarrierManager}; #[tokio::test] @@ -147,48 +128,35 @@ mod test { let actor_id = progress.actor_id(); let schema = Schema::new(vec![Field::unnamed(DataType::Int64)]); - let first = Box::new( - MockSource::with_chunks( - schema.clone(), - PkIndices::new(), - vec![ - StreamChunk::from_pretty("I\n + 1"), - StreamChunk::from_pretty("I\n + 2"), - ], - ) - .stop_on_finish(false), - ); - - let second = Box::new(MockSource::with_messages( - schema.clone(), - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1).with_mutation(Mutation::Add( - AddMutation { - adds: maplit::hashmap! { - 0 => vec![Dispatcher { - downstream_actor_id: vec![actor_id], - ..Default::default() - }], - }, - added_actors: maplit::hashset! { actor_id }, - splits: Default::default(), - pause: false, + let first = MockSource::with_chunks(vec![ + StreamChunk::from_pretty("I\n + 1"), + StreamChunk::from_pretty("I\n + 2"), + ]) + .stop_on_finish(false) + .into_executor(schema.clone(), PkIndices::new()); + + let second = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1).with_mutation(Mutation::Add( + AddMutation { + adds: maplit::hashmap! { + 0 => vec![Dispatcher { + downstream_actor_id: vec![actor_id], + ..Default::default() + }], }, - ))), - Message::Chunk(StreamChunk::from_pretty("I\n + 3")), - Message::Chunk(StreamChunk::from_pretty("I\n + 4")), - ], - )); - - let info = ExecutorInfo { - schema, - pk_indices: PkIndices::new(), - identity: "ChainExecutor".to_string(), - }; - let chain = ChainExecutor::new(info, first, second, progress, false); - - let mut chain = Box::new(chain).execute(); + added_actors: maplit::hashset! { actor_id }, + splits: Default::default(), + pause: false, + }, + ))), + Message::Chunk(StreamChunk::from_pretty("I\n + 3")), + Message::Chunk(StreamChunk::from_pretty("I\n + 4")), + ]) + .into_executor(schema.clone(), PkIndices::new()); + + let chain = ChainExecutor::new(first, second, progress, false); + + let mut chain = chain.boxed().execute(); chain.next().await; let mut count = 0; diff --git a/src/stream/src/executor/dedup/append_only_dedup.rs b/src/stream/src/executor/dedup/append_only_dedup.rs index 6049b963359ab..34b102ff1e030 100644 --- a/src/stream/src/executor/dedup/append_only_dedup.rs +++ b/src/stream/src/executor/dedup/append_only_dedup.rs @@ -19,7 +19,6 @@ use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::BitmapBuilder; -use risingwave_common::catalog::Schema; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_storage::StateStore; @@ -29,39 +28,39 @@ use crate::common::table::state_table::StateTable; use crate::executor::error::StreamExecutorError; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, PkIndicesRef, StreamExecutorResult, + expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, + StreamExecutorResult, }; use crate::task::AtomicU64Ref; /// [`AppendOnlyDedupExecutor`] drops any message that has duplicate pk columns with previous /// messages. It only accepts append-only input, and its output will be append-only as well. pub struct AppendOnlyDedupExecutor<S: StateStore> { - input: Option<BoxedExecutor>, + ctx: ActorContextRef, + + input: Option<Executor>, + dedup_cols: Vec<usize>, state_table: StateTable<S>, cache: DedupCache<OwnedRow>, - - info: ExecutorInfo, - ctx: ActorContextRef, } impl<S: StateStore> AppendOnlyDedupExecutor<S> { pub fn new( - input: BoxedExecutor, - state_table: StateTable<S>, - info: ExecutorInfo, ctx: ActorContextRef, + input: Executor, + dedup_cols: Vec<usize>, + state_table: StateTable<S>, watermark_epoch: AtomicU64Ref, metrics: Arc<StreamingMetrics>, ) -> Self { let metrics_info = MetricsInfo::new(metrics, state_table.table_id(), ctx.id, "AppendOnly Dedup"); Self { + ctx, input: Some(input), + dedup_cols, state_table, cache: DedupCache::new(watermark_epoch, metrics_info), - info, - ctx, } } @@ -90,7 +89,7 @@ impl<S: StateStore> AppendOnlyDedupExecutor<S> { .data_chunk() .rows_with_holes() .map(|row_ref| { - row_ref.map(|row| row.project(self.pk_indices()).to_owned_row()) + row_ref.map(|row| row.project(&self.dedup_cols).to_owned_row()) }) .collect_vec(); @@ -185,22 +184,10 @@ impl<S: StateStore> AppendOnlyDedupExecutor<S> { } } -impl<S: StateStore> Executor for AppendOnlyDedupExecutor<S> { +impl<S: StateStore> Execute for AppendOnlyDedupExecutor<S> { fn execute(self: Box<Self>) -> BoxedMessageStream { self.executor_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } #[cfg(test)] @@ -230,7 +217,8 @@ mod tests { Field::unnamed(DataType::Int64), Field::unnamed(DataType::Int64), ]); - let pk_indices = vec![0]; + let dedup_col_indices = vec![0]; + let pk_indices = dedup_col_indices.clone(); let order_types = vec![OrderType::ascending()]; let state_store = MemoryStateStore::new(); @@ -243,20 +231,17 @@ mod tests { ) .await; - let (mut tx, input) = MockSource::channel(schema.clone(), pk_indices.clone()); - let info = ExecutorInfo { - schema, - pk_indices, - identity: "AppendOnlyDedupExecutor".to_string(), - }; - let mut dedup_executor = Box::new(AppendOnlyDedupExecutor::new( - Box::new(input), - state_table, - info, + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, pk_indices); + let mut dedup_executor = AppendOnlyDedupExecutor::new( ActorContext::for_test(123), + source, + dedup_col_indices, + state_table, Arc::new(AtomicU64::new(0)), Arc::new(StreamingMetrics::unused()), - )) + ) + .boxed() .execute(); tx.push_barrier(1, false); diff --git a/src/stream/src/executor/dispatch.rs b/src/stream/src/executor/dispatch.rs index a21012c5f1664..ee347ee489258 100644 --- a/src/stream/src/executor/dispatch.rs +++ b/src/stream/src/executor/dispatch.rs @@ -35,17 +35,17 @@ use tokio::time::Instant; use tracing::{event, Instrument}; use super::exchange::output::{new_output, BoxedOutput}; -use super::{AddMutation, UpdateMutation, Watermark}; +use super::{AddMutation, Executor, UpdateMutation, Watermark}; use crate::error::StreamResult; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{Barrier, BoxedExecutor, Message, Mutation, StreamConsumer}; +use crate::executor::{Barrier, Message, Mutation, StreamConsumer}; use crate::task::{ActorId, DispatcherId, SharedContext}; /// [`DispatchExecutor`] consumes messages and send them into downstream actors. Usually, /// data chunks will be dispatched with some specified policy, while control message /// such as barriers will be distributed to all receivers. pub struct DispatchExecutor { - input: BoxedExecutor, + input: Executor, inner: DispatchExecutorInner, } @@ -341,7 +341,7 @@ impl DispatchExecutorInner { impl DispatchExecutor { pub fn new( - input: BoxedExecutor, + input: Executor, dispatchers: Vec<DispatcherImpl>, actor_id: u32, fragment_id: u32, @@ -1042,6 +1042,7 @@ mod tests { use crate::executor::exchange::output::Output; use crate::executor::exchange::permit::channel_for_test; use crate::executor::receiver::ReceiverExecutor; + use crate::executor::Execute; use crate::task::test_utils::helper_make_local_actor; #[derive(Debug)] @@ -1152,7 +1153,7 @@ mod tests { let (tx, rx) = channel_for_test(); let actor_id = 233; let fragment_id = 666; - let input = Box::new(ReceiverExecutor::for_test(rx)); + let input = Executor::new(Default::default(), ReceiverExecutor::for_test(rx).boxed()); let ctx = Arc::new(SharedContext::for_test()); let metrics = Arc::new(StreamingMetrics::unused()); diff --git a/src/stream/src/executor/dml.rs b/src/stream/src/executor/dml.rs index 8e0a186e9c81c..0463c695639d7 100644 --- a/src/stream/src/executor/dml.rs +++ b/src/stream/src/executor/dml.rs @@ -19,25 +19,20 @@ use either::Either; use futures::{StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; -use risingwave_common::catalog::{ColumnDesc, Schema, TableId, TableVersionId}; +use risingwave_common::catalog::{ColumnDesc, TableId, TableVersionId}; use risingwave_common::transaction::transaction_id::TxnId; use risingwave_common::transaction::transaction_message::TxnMsg; use risingwave_dml::dml_manager::DmlManagerRef; use super::error::StreamExecutorError; -use super::{ - expect_first_barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, Message, - Mutation, PkIndicesRef, -}; +use super::{expect_first_barrier, BoxedMessageStream, Execute, Executor, Message, Mutation}; use crate::common::StreamChunkBuilder; use crate::executor::stream_reader::StreamReaderWithPause; /// [`DmlExecutor`] accepts both stream data and batch data for data manipulation on a specific /// table. The two streams will be merged into one and then sent to downstream. pub struct DmlExecutor { - info: ExecutorInfo, - - upstream: BoxedExecutor, + upstream: Executor, /// Stores the information of batch data channels. dml_manager: DmlManagerRef, @@ -71,10 +66,8 @@ struct TxnBuffer { } impl DmlExecutor { - #[allow(clippy::too_many_arguments)] pub fn new( - info: ExecutorInfo, - upstream: BoxedExecutor, + upstream: Executor, dml_manager: DmlManagerRef, table_id: TableId, table_version_id: TableVersionId, @@ -82,7 +75,6 @@ impl DmlExecutor { chunk_size: usize, ) -> Self { Self { - info, upstream, dml_manager, table_id, @@ -277,22 +269,10 @@ impl DmlExecutor { } } -impl Executor for DmlExecutor { +impl Execute for DmlExecutor { fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } #[cfg(test)] @@ -300,7 +280,7 @@ mod tests { use std::sync::Arc; use risingwave_common::array::StreamChunk; - use risingwave_common::catalog::{ColumnId, Field, INITIAL_TABLE_VERSION_ID}; + use risingwave_common::catalog::{ColumnId, Field, Schema, INITIAL_TABLE_VERSION_ID}; use risingwave_common::test_prelude::StreamChunkTestExt; use risingwave_common::transaction::transaction_id::TxnId; use risingwave_common::types::DataType; @@ -326,23 +306,18 @@ mod tests { let pk_indices = vec![0]; let dml_manager = Arc::new(DmlManager::for_test()); - let (mut tx, source) = MockSource::channel(schema.clone(), pk_indices.clone()); - let info = ExecutorInfo { - schema, - pk_indices, - identity: "DmlExecutor".to_string(), - }; + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, pk_indices); - let dml_executor = Box::new(DmlExecutor::new( - info, - Box::new(source), + let dml_executor = DmlExecutor::new( + source, dml_manager.clone(), table_id, INITIAL_TABLE_VERSION_ID, column_descs, 1024, - )); - let mut dml_executor = dml_executor.execute(); + ); + let mut dml_executor = dml_executor.boxed().execute(); let stream_chunk1 = StreamChunk::from_pretty( " I I diff --git a/src/stream/src/executor/dynamic_filter.rs b/src/stream/src/executor/dynamic_filter.rs index d3299d99e8a33..6bcd3e97ed563 100644 --- a/src/stream/src/executor/dynamic_filter.rs +++ b/src/stream/src/executor/dynamic_filter.rs @@ -38,10 +38,7 @@ use risingwave_storage::StateStore; use super::barrier_align::*; use super::error::StreamExecutorError; use super::monitor::StreamingMetrics; -use super::{ - ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, Message, - PkIndicesRef, -}; +use super::{ActorContextRef, BoxedMessageStream, Execute, Executor, ExecutorInfo, Message}; use crate::common::table::state_table::{StateTable, WatermarkCacheParameterizedStateTable}; use crate::common::StreamChunkBuilder; use crate::executor::expect_first_barrier_from_aligned_stream; @@ -49,10 +46,12 @@ use crate::task::ActorEvalErrorReport; pub struct DynamicFilterExecutor<S: StateStore, const USE_WATERMARK_CACHE: bool> { ctx: ActorContextRef, - info: ExecutorInfo, - source_l: Option<BoxedExecutor>, - source_r: Option<BoxedExecutor>, + eval_error_report: ActorEvalErrorReport, + + schema: Schema, + source_l: Option<Executor>, + source_r: Option<Executor>, key_l: usize, comparator: ExprNodeType, left_table: WatermarkCacheParameterizedStateTable<S, USE_WATERMARK_CACHE>, @@ -71,9 +70,9 @@ impl<S: StateStore, const USE_WATERMARK_CACHE: bool> DynamicFilterExecutor<S, US #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, - source_l: BoxedExecutor, - source_r: BoxedExecutor, + info: &ExecutorInfo, + source_l: Executor, + source_r: Executor, key_l: usize, comparator: ExprNodeType, state_table_l: WatermarkCacheParameterizedStateTable<S, USE_WATERMARK_CACHE>, @@ -83,9 +82,14 @@ impl<S: StateStore, const USE_WATERMARK_CACHE: bool> DynamicFilterExecutor<S, US condition_always_relax: bool, cleaned_by_watermark: bool, ) -> Self { + let eval_error_report = ActorEvalErrorReport { + actor_context: ctx.clone(), + identity: Arc::from(info.identity.as_str()), + }; Self { ctx, - info, + eval_error_report, + schema: info.schema.clone(), source_l: Some(source_l), source_r: Some(source_r), key_l, @@ -263,7 +267,7 @@ impl<S: StateStore, const USE_WATERMARK_CACHE: bool> DynamicFilterExecutor<S, US } #[try_stream(ok = Message, error = StreamExecutorError)] - async fn into_stream(mut self) { + async fn execute_inner(mut self) { let input_l = self.source_l.take().unwrap(); let input_r = self.source_r.take().unwrap(); @@ -273,10 +277,7 @@ impl<S: StateStore, const USE_WATERMARK_CACHE: bool> DynamicFilterExecutor<S, US // The types are aligned by frontend. assert_eq!(l_data_type, r_data_type); let dynamic_cond = { - let eval_error_report = ActorEvalErrorReport { - actor_context: self.ctx.clone(), - identity: Arc::from(self.info.identity.as_str()), - }; + let eval_error_report = self.eval_error_report.clone(); move |literal: Datum| { literal.map(|scalar| { build_func_non_strict( @@ -319,7 +320,7 @@ impl<S: StateStore, const USE_WATERMARK_CACHE: bool> DynamicFilterExecutor<S, US yield Message::Barrier(barrier); let mut stream_chunk_builder = - StreamChunkBuilder::new(self.chunk_size, self.info.schema.data_types()); + StreamChunkBuilder::new(self.chunk_size, self.schema.data_types()); let watermark_can_clean_state = !matches!(self.comparator, LessThan | LessThanOrEqual); let mut unused_clean_hint = None; @@ -487,23 +488,11 @@ impl<S: StateStore, const USE_WATERMARK_CACHE: bool> DynamicFilterExecutor<S, US } } -impl<S: StateStore, const USE_WATERMARK_CACHE: bool> Executor +impl<S: StateStore, const USE_WATERMARK_CACHE: bool> Execute for DynamicFilterExecutor<S, USE_WATERMARK_CACHE> { fn execute(self: Box<Self>) -> BoxedMessageStream { - self.into_stream().boxed() - } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity + self.execute_inner().boxed() } } @@ -561,21 +550,20 @@ mod tests { let schema = Schema { fields: vec![Field::unnamed(DataType::Int64)], }; - let (tx_l, source_l) = MockSource::channel(schema.clone(), vec![0]); - let (tx_r, source_r) = MockSource::channel(schema, vec![]); - - let schema = source_l.schema().clone(); - let info = ExecutorInfo { - schema, - pk_indices: vec![0], - identity: "DynamicFilterExecutor".to_string(), - }; + let (tx_l, source_l) = MockSource::channel(); + let source_l = source_l.into_executor(schema.clone(), vec![0]); + let (tx_r, source_r) = MockSource::channel(); + let source_r = source_r.into_executor(schema, vec![]); let executor = DynamicFilterExecutor::<MemoryStateStore, false>::new( ActorContext::for_test(123), - info, - Box::new(source_l), - Box::new(source_r), + &ExecutorInfo { + schema: source_l.schema().clone(), + pk_indices: vec![0], + identity: "DynamicFilterExecutor".to_string(), + }, + source_l, + source_r, 0, comparator, mem_state_l, @@ -585,7 +573,7 @@ mod tests { always_relax, false, ); - (tx_l, tx_r, Box::new(executor).execute()) + (tx_l, tx_r, executor.boxed().execute()) } #[tokio::test] diff --git a/src/stream/src/executor/expand.rs b/src/stream/src/executor/expand.rs index cb8c45732d17f..2375ebca3d0df 100644 --- a/src/stream/src/executor/expand.rs +++ b/src/stream/src/executor/expand.rs @@ -16,26 +16,19 @@ use std::fmt::Debug; use futures::StreamExt; use futures_async_stream::try_stream; -use risingwave_common::array::{Array, I64Array}; -use risingwave_common::catalog::Schema; +use risingwave_common::array::{Array, I64Array, StreamChunk}; use super::error::StreamExecutorError; -use super::*; +use super::{BoxedMessageStream, Execute, Executor, Message}; pub struct ExpandExecutor { - info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, column_subsets: Vec<Vec<usize>>, } impl ExpandExecutor { - pub fn new( - info: ExecutorInfo, - input: Box<dyn Executor>, - column_subsets: Vec<Vec<usize>>, - ) -> Self { + pub fn new(input: Executor, column_subsets: Vec<Vec<usize>>) -> Self { Self { - info, input, column_subsets, } @@ -73,19 +66,7 @@ impl Debug for ExpandExecutor { } } -impl Executor for ExpandExecutor { - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - +impl Execute for ExpandExecutor { fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner().boxed() } @@ -100,7 +81,7 @@ mod tests { use super::ExpandExecutor; use crate::executor::test_utils::MockSource; - use crate::executor::{Executor, ExecutorInfo, PkIndices}; + use crate::executor::{Execute, PkIndices}; #[tokio::test] async fn test_expand() { @@ -111,29 +92,19 @@ mod tests { + 6 6 3 - 7 5 4", ); - let input_schema = Schema { - fields: vec![ + let source = MockSource::with_chunks(vec![chunk1]).into_executor( + Schema::new(vec![ Field::unnamed(DataType::Int64), Field::unnamed(DataType::Int64), Field::unnamed(DataType::Int64), - ], - }; - let source = MockSource::with_chunks(input_schema.clone(), PkIndices::new(), vec![chunk1]); - let schema = { - let mut fields = input_schema.into_fields(); - fields.extend(fields.clone()); - fields.push(Field::with_name(DataType::Int64, "flag")); - Schema::new(fields) - }; - let info = ExecutorInfo { - schema, - pk_indices: vec![], - identity: "ExpandExecutor".to_string(), - }; + ]), + PkIndices::new(), + ); let column_subsets = vec![vec![0, 1], vec![1, 2]]; - let expand = Box::new(ExpandExecutor::new(info, Box::new(source), column_subsets)); - let mut expand = expand.execute(); + let mut expand = ExpandExecutor::new(source, column_subsets) + .boxed() + .execute(); let chunk = expand.next().await.unwrap().unwrap().into_chunk().unwrap(); assert_eq!( diff --git a/src/stream/src/executor/filter.rs b/src/stream/src/executor/filter.rs index e51307b2ffee0..ffc8847c42abf 100644 --- a/src/stream/src/executor/filter.rs +++ b/src/stream/src/executor/filter.rs @@ -15,13 +15,17 @@ use std::fmt::{Debug, Formatter}; use std::sync::Arc; +use futures::StreamExt; +use futures_async_stream::try_stream; use risingwave_common::array::{Array, ArrayImpl, Op, StreamChunk}; use risingwave_common::buffer::BitmapBuilder; -use risingwave_common::catalog::Schema; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::expr::NonStrictExpression; -use super::*; +use super::{ + ActorContextRef, BoxedMessageStream, Execute, Executor, Message, StreamExecutorError, + StreamExecutorResult, +}; /// `FilterExecutor` filters data with the `expr`. The `expr` takes a chunk of data, /// and returns a boolean array on whether each item should be retained. And then, @@ -29,8 +33,7 @@ use super::*; /// to the result of the expression. pub struct FilterExecutor { _ctx: ActorContextRef, - info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, /// Expression of the current filter, note that the filter must always have the same output for /// the same input. @@ -38,15 +41,9 @@ pub struct FilterExecutor { } impl FilterExecutor { - pub fn new( - ctx: ActorContextRef, - info: ExecutorInfo, - input: Box<dyn Executor>, - expr: NonStrictExpression, - ) -> Self { + pub fn new(ctx: ActorContextRef, input: Executor, expr: NonStrictExpression) -> Self { Self { _ctx: ctx, - info, input, expr, } @@ -135,19 +132,7 @@ impl Debug for FilterExecutor { } } -impl Executor for FilterExecutor { - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - +impl Execute for FilterExecutor { fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner().boxed() } @@ -218,23 +203,14 @@ mod tests { ], }; let pk_indices = PkIndices::new(); - let source = - MockSource::with_chunks(schema.clone(), pk_indices.clone(), vec![chunk1, chunk2]); - let info = ExecutorInfo { - schema, - pk_indices, - identity: "FilterExecutor".to_string(), - }; + let source = MockSource::with_chunks(vec![chunk1, chunk2]) + .into_executor(schema.clone(), pk_indices.clone()); let test_expr = build_from_pretty("(greater_than:boolean $0:int8 $1:int8)"); - let filter = Box::new(FilterExecutor::new( - ActorContext::for_test(123), - info, - Box::new(source), - test_expr, - )); - let mut filter = filter.execute(); + let mut filter = FilterExecutor::new(ActorContext::for_test(123), source, test_expr) + .boxed() + .execute(); let chunk = filter.next().await.unwrap().unwrap().into_chunk().unwrap(); assert_eq!( diff --git a/src/stream/src/executor/flow_control.rs b/src/stream/src/executor/flow_control.rs index 53d0f15a602ae..4e4212a1e246e 100644 --- a/src/stream/src/executor/flow_control.rs +++ b/src/stream/src/executor/flow_control.rs @@ -15,11 +15,14 @@ use std::fmt::{Debug, Formatter}; use std::num::NonZeroU32; +use futures::StreamExt; +use futures_async_stream::try_stream; use governor::clock::MonotonicClock; use governor::{Quota, RateLimiter}; -use risingwave_common::catalog::Schema; -use super::*; +use super::{ + ActorContextRef, BoxedMessageStream, Execute, Executor, Message, Mutation, StreamExecutorError, +}; /// Flow Control Executor is used to control the rate of the input executor. /// @@ -31,27 +34,16 @@ use super::*; /// /// It is used to throttle problematic MVs that are consuming too much resources. pub struct FlowControlExecutor { - input: BoxedExecutor, + input: Executor, actor_ctx: ActorContextRef, - identity: String, rate_limit: Option<u32>, } impl FlowControlExecutor { - pub fn new( - input: Box<dyn Executor>, - actor_ctx: ActorContextRef, - rate_limit: Option<u32>, - ) -> Self { - let identity = if rate_limit.is_some() { - format!("{} (flow controlled)", input.identity()) - } else { - input.identity().to_owned() - }; + pub fn new(input: Executor, actor_ctx: ActorContextRef, rate_limit: Option<u32>) -> Self { Self { input, actor_ctx, - identity, rate_limit, } } @@ -128,20 +120,8 @@ impl Debug for FlowControlExecutor { } } -impl Executor for FlowControlExecutor { +impl Execute for FlowControlExecutor { fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - self.input.schema() - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - self.input.pk_indices() - } - - fn identity(&self) -> &str { - &self.identity - } } diff --git a/src/stream/src/executor/hash_agg.rs b/src/stream/src/executor/hash_agg.rs index 5078fd0bc5154..ccaa4d98ce11d 100644 --- a/src/stream/src/executor/hash_agg.rs +++ b/src/stream/src/executor/hash_agg.rs @@ -39,8 +39,7 @@ use super::aggregation::{ }; use super::sort_buffer::SortBuffer; use super::{ - expect_first_barrier, ActorContextRef, ExecutorInfo, PkIndicesRef, StreamExecutorResult, - Watermark, + expect_first_barrier, ActorContextRef, Executor, ExecutorInfo, StreamExecutorResult, Watermark, }; use crate::cache::{cache_may_stale, new_with_hasher, ManagedLruCache}; use crate::common::metrics::MetricsInfo; @@ -49,7 +48,7 @@ use crate::common::StreamChunkBuilder; use crate::error::StreamResult; use crate::executor::aggregation::AggGroup as GenericAggGroup; use crate::executor::error::StreamExecutorError; -use crate::executor::{BoxedMessageStream, Executor, Message}; +use crate::executor::{BoxedMessageStream, Execute, Message}; use crate::task::AtomicU64Ref; type AggGroup<S> = GenericAggGroup<S, OnlyOutputIfHasInput>; @@ -74,7 +73,7 @@ type AggGroupCache<K, S> = ManagedLruCache<K, Option<BoxedAggGroup<S>>, Precompu /// all modifications will be flushed to the storage backend. Meanwhile, the executor will go /// through `group_change_set`, and produce a stream chunk based on the state changes. pub struct HashAggExecutor<K: HashKey, S: StateStore> { - input: Box<dyn Executor>, + input: Executor, inner: ExecutorInner<K, S>, } @@ -193,27 +192,15 @@ impl ExecutionStats { } } -impl<K: HashKey, S: StateStore> Executor for HashAggExecutor<K, S> { +impl<K: HashKey, S: StateStore> Execute for HashAggExecutor<K, S> { fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.inner.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.inner.info.pk_indices - } - - fn identity(&self) -> &str { - &self.inner.info.identity - } } impl<K: HashKey, S: StateStore> HashAggExecutor<K, S> { pub fn new(args: AggExecutorArgs<S, HashAggExecutorExtraArgs>) -> StreamResult<Self> { - let input_info = args.input.info(); + let input_info = args.input.info().clone(); let group_key_len = args.extra.group_key_indices.len(); // NOTE: we assume the prefix of table pk is exactly the group key diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs index 471d382a763d0..3727da397d01f 100644 --- a/src/stream/src/executor/hash_join.rs +++ b/src/stream/src/executor/hash_join.rs @@ -22,7 +22,6 @@ use futures_async_stream::try_stream; use itertools::Itertools; use multimap::MultiMap; use risingwave_common::array::{Op, RowRef, StreamChunk}; -use risingwave_common::catalog::Schema; use risingwave_common::hash::{HashKey, NullBitmap}; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{DataType, DefaultOrd, ToOwnedDatum}; @@ -42,8 +41,7 @@ use super::join::{JoinTypePrimitive, SideTypePrimitive, *}; use super::monitor::StreamingMetrics; use super::watermark::*; use super::{ - ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, Message, - PkIndicesRef, Watermark, + ActorContextRef, BoxedMessageStream, Execute, Executor, ExecutorInfo, Message, Watermark, }; use crate::common::table::state_table::StateTable; use crate::executor::expect_first_barrier_from_aligned_stream; @@ -143,9 +141,9 @@ pub struct HashJoinExecutor<K: HashKey, S: StateStore, const T: JoinTypePrimitiv info: ExecutorInfo, /// Left input executor - input_l: Option<BoxedExecutor>, + input_l: Option<Executor>, /// Right input executor - input_r: Option<BoxedExecutor>, + input_r: Option<Executor>, /// The data types of the formed new columns actual_output_data_types: Vec<DataType>, /// The parameters of the left join executor @@ -191,22 +189,10 @@ impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> std::fmt::Debug } } -impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> Executor for HashJoinExecutor<K, S, T> { +impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> Execute for HashJoinExecutor<K, S, T> { fn execute(self: Box<Self>) -> BoxedMessageStream { self.into_stream().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } struct EqJoinArgs<'a, K: HashKey, S: StateStore> { @@ -227,8 +213,8 @@ impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> HashJoinExecutor<K, pub fn new( ctx: ActorContextRef, info: ExecutorInfo, - input_l: BoxedExecutor, - input_r: BoxedExecutor, + input_l: Executor, + input_r: Executor, params_l: JoinParams, params_r: JoinParams, null_safe: Vec<bool>, @@ -1156,8 +1142,10 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - let (tx_l, source_l) = MockSource::channel(schema.clone(), vec![1]); - let (tx_r, source_r) = MockSource::channel(schema, vec![1]); + let (tx_l, source_l) = MockSource::channel(); + let source_l = source_l.into_executor(schema.clone(), vec![1]); + let (tx_r, source_r) = MockSource::channel(); + let source_r = source_r.into_executor(schema, vec![1]); let params_l = JoinParams::new(vec![0], vec![1]); let params_r = JoinParams::new(vec![0], vec![1]); let cond = with_condition.then(|| create_cond(condition_text)); @@ -1200,8 +1188,8 @@ mod tests { let executor = HashJoinExecutor::<Key64, MemoryStateStore, T>::new( ActorContext::for_test(123), info, - Box::new(source_l), - Box::new(source_r), + source_l, + source_r, params_l, params_r, vec![null_safe], @@ -1217,7 +1205,7 @@ mod tests { Arc::new(StreamingMetrics::unused()), 1024, ); - (tx_l, tx_r, Box::new(executor).execute()) + (tx_l, tx_r, executor.boxed().execute()) } async fn create_classical_executor<const T: JoinTypePrimitive>( @@ -1238,8 +1226,10 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - let (tx_l, source_l) = MockSource::channel(schema.clone(), vec![0]); - let (tx_r, source_r) = MockSource::channel(schema, vec![0]); + let (tx_l, source_l) = MockSource::channel(); + let source_l = source_l.into_executor(schema.clone(), vec![0]); + let (tx_r, source_r) = MockSource::channel(); + let source_r = source_r.into_executor(schema, vec![0]); let params_l = JoinParams::new(vec![0, 1], vec![]); let params_r = JoinParams::new(vec![0, 1], vec![]); let cond = with_condition.then(|| create_cond(None)); @@ -1290,8 +1280,8 @@ mod tests { let executor = HashJoinExecutor::<Key128, MemoryStateStore, T>::new( ActorContext::for_test(123), info, - Box::new(source_l), - Box::new(source_r), + source_l, + source_r, params_l, params_r, vec![false], @@ -1307,7 +1297,7 @@ mod tests { Arc::new(StreamingMetrics::unused()), 1024, ); - (tx_l, tx_r, Box::new(executor).execute()) + (tx_l, tx_r, executor.boxed().execute()) } #[tokio::test] diff --git a/src/stream/src/executor/hop_window.rs b/src/stream/src/executor/hop_window.rs index 4bcca4d593072..801f3daa9f53a 100644 --- a/src/stream/src/executor/hop_window.rs +++ b/src/stream/src/executor/hop_window.rs @@ -23,13 +23,12 @@ use risingwave_expr::expr::NonStrictExpression; use risingwave_expr::ExprError; use super::error::StreamExecutorError; -use super::{ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, Message}; +use super::{ActorContextRef, Execute, Executor, Message}; use crate::common::StreamChunkBuilder; pub struct HopWindowExecutor { _ctx: ActorContextRef, - pub info: ExecutorInfo, - pub input: BoxedExecutor, + pub input: Executor, pub time_col_idx: usize, pub window_slide: Interval, pub window_size: Interval, @@ -43,8 +42,7 @@ impl HopWindowExecutor { #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, time_col_idx: usize, window_slide: Interval, window_size: Interval, @@ -55,7 +53,6 @@ impl HopWindowExecutor { ) -> Self { HopWindowExecutor { _ctx: ctx, - info, input, time_col_idx, window_slide, @@ -68,22 +65,10 @@ impl HopWindowExecutor { } } -impl Executor for HopWindowExecutor { +impl Execute for HopWindowExecutor { fn execute(self: Box<Self>) -> super::BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &risingwave_common::catalog::Schema { - &self.info.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } impl HopWindowExecutor { @@ -253,12 +238,13 @@ mod tests { use risingwave_expr::expr::test_utils::make_hop_window_expression; use risingwave_expr::expr::NonStrictExpression; + use super::*; use crate::executor::test_utils::MockSource; - use crate::executor::{ActorContext, Executor, ExecutorInfo, StreamChunk}; + use crate::executor::{ActorContext, Execute, StreamChunk}; const CHUNK_SIZE: usize = 256; - fn create_executor(output_indices: Vec<usize>) -> Box<dyn Executor> { + fn create_executor(output_indices: Vec<usize>) -> Box<dyn Execute> { let field1 = Field::unnamed(DataType::Int64); let field2 = Field::unnamed(DataType::Int64); let field3 = Field::with_name(DataType::Timestamp, "created_at"); @@ -278,7 +264,7 @@ mod tests { .replace('^', "2022-02-02T"), ); let input = - MockSource::with_chunks(schema.clone(), pk_indices.clone(), vec![chunk]).boxed(); + MockSource::with_chunks(vec![chunk]).into_executor(schema.clone(), pk_indices.clone()); let window_slide = Interval::from_minutes(15); let window_size = Interval::from_minutes(30); let window_offset = Interval::from_minutes(0); @@ -291,14 +277,8 @@ mod tests { ) .unwrap(); - super::HopWindowExecutor::new( + HopWindowExecutor::new( ActorContext::for_test(123), - ExecutorInfo { - // TODO: the schema is incorrect, but it seems useless here. - schema, - pk_indices, - identity: "HopWindowExecutor".to_string(), - }, input, 2, window_slide, diff --git a/src/stream/src/executor/integration_tests.rs b/src/stream/src/executor/integration_tests.rs index 0759f4bd2d813..691ba0f46f341 100644 --- a/src/stream/src/executor/integration_tests.rs +++ b/src/stream/src/executor/integration_tests.rs @@ -35,7 +35,7 @@ use crate::executor::receiver::ReceiverExecutor; use crate::executor::test_utils::agg_executor::{ generate_agg_schema, new_boxed_simple_agg_executor, }; -use crate::executor::{Executor, MergeExecutor, ProjectExecutor, StatelessSimpleAggExecutor}; +use crate::executor::{Execute, MergeExecutor, ProjectExecutor, StatelessSimpleAggExecutor}; use crate::task::{LocalBarrierManager, SharedContext}; /// This test creates a merger-dispatcher pair, and run a sum. Each chunk @@ -50,27 +50,25 @@ async fn test_merger_sum_aggr() { let actor_ctx = ActorContext::for_test(0); // `make_actor` build an actor to do local aggregation let make_actor = |input_rx| { - let _schema = Schema { + let input_schema = Schema { fields: vec![Field::unnamed(DataType::Int64)], }; - let input = ReceiverExecutor::for_test(input_rx); + let input = Executor::new( + ExecutorInfo { + schema: input_schema, + pk_indices: PkIndices::new(), + identity: "ReceiverExecutor".to_string(), + }, + ReceiverExecutor::for_test(input_rx).boxed(), + ); let agg_calls = vec![ AggCall::from_pretty("(count:int8)"), AggCall::from_pretty("(sum:int8 $0:int8)"), ]; let schema = generate_agg_schema(&input, &agg_calls, None); // for the local aggregator, we need two states: row count and sum - let aggregator = StatelessSimpleAggExecutor::new( - actor_ctx.clone(), - ExecutorInfo { - schema, - pk_indices: vec![], - identity: format!("StatelessSimpleAggExecutor {:X}", 1), - }, - input.boxed(), - agg_calls, - ) - .unwrap(); + let aggregator = + StatelessSimpleAggExecutor::new(actor_ctx.clone(), input, schema, agg_calls).unwrap(); let (tx, rx) = channel_for_test(); let consumer = SenderConsumer { input: aggregator.boxed(), @@ -108,13 +106,15 @@ async fn test_merger_sum_aggr() { // create a round robin dispatcher, which dispatches messages to the actors let (input, rx) = channel_for_test(); - let schema = Schema { - fields: vec![ - Field::unnamed(DataType::Int64), - Field::unnamed(DataType::Int64), - ], - }; - let receiver_op = Box::new(ReceiverExecutor::for_test(rx)); + let receiver_op = Executor::new( + ExecutorInfo { + // input schema of local simple agg + schema: Schema::new(vec![Field::unnamed(DataType::Int64)]), + pk_indices: PkIndices::new(), + identity: "ReceiverExecutor".to_string(), + }, + ReceiverExecutor::for_test(rx).boxed(), + ); let dispatcher = DispatchExecutor::new( receiver_op, vec![DispatcherImpl::RoundRobin(RoundRobinDataDispatcher::new( @@ -138,14 +138,25 @@ async fn test_merger_sum_aggr() { handles.push(tokio::spawn(actor.run())); // use a merge operator to collect data from dispatchers before sending them to aggregator - let merger = MergeExecutor::for_test(outputs, schema); + let merger = Executor::new( + ExecutorInfo { + // output schema of local simple agg + schema: Schema::new(vec![ + Field::unnamed(DataType::Int64), + Field::unnamed(DataType::Int64), + ]), + pk_indices: PkIndices::new(), + identity: "MergeExecutor".to_string(), + }, + MergeExecutor::for_test(outputs).boxed(), + ); // for global aggregator, we need to sum data and sum row count let is_append_only = false; let aggregator = new_boxed_simple_agg_executor( actor_ctx.clone(), MemoryStateStore::new(), - merger.boxed(), + merger, is_append_only, vec![ AggCall::from_pretty("(sum0:int8 $0:int8)"), @@ -160,13 +171,6 @@ async fn test_merger_sum_aggr() { let projection = ProjectExecutor::new( actor_ctx.clone(), - ExecutorInfo { - schema: Schema { - fields: vec![Field::unnamed(DataType::Int64)], - }, - pk_indices: vec![], - identity: format!("ProjectExecutor {:X}", 3), - }, aggregator, vec![ // TODO: use the new streaming_if_null expression here, and add `None` tests @@ -232,7 +236,7 @@ async fn test_merger_sum_aggr() { } struct MockConsumer { - input: BoxedExecutor, + input: Box<dyn Execute>, data: Arc<Mutex<Vec<StreamChunk>>>, } @@ -259,7 +263,7 @@ impl StreamConsumer for MockConsumer { /// `SenderConsumer` consumes data from input executor and send it into a channel. pub struct SenderConsumer { - input: BoxedExecutor, + input: Box<dyn Execute>, channel: BoxedOutput, } diff --git a/src/stream/src/executor/lookup.rs b/src/stream/src/executor/lookup.rs index 87cc163ea3dff..2c1de3170a801 100644 --- a/src/stream/src/executor/lookup.rs +++ b/src/stream/src/executor/lookup.rs @@ -14,11 +14,10 @@ use async_trait::async_trait; use futures::StreamExt; -use risingwave_common::catalog::Schema; use risingwave_common::types::DataType; use risingwave_storage::StateStore; -use crate::executor::{Barrier, BoxedMessageStream, Executor, PkIndicesRef}; +use crate::executor::{Barrier, BoxedMessageStream, Execute}; mod cache; mod sides; @@ -28,7 +27,7 @@ mod impl_; pub use impl_::LookupExecutorParams; -use super::{ActorContextRef, ExecutorInfo}; +use super::{ActorContextRef, Executor}; #[cfg(test)] mod tests; @@ -42,8 +41,6 @@ mod tests; pub struct LookupExecutor<S: StateStore> { ctx: ActorContextRef, - info: ExecutorInfo, - /// the data types of the produced data chunk inside lookup (before reordering) chunk_data_types: Vec<DataType>, @@ -54,10 +51,10 @@ pub struct LookupExecutor<S: StateStore> { stream: StreamJoinSide, /// The executor for arrangement. - arrangement_executor: Option<Box<dyn Executor>>, + arrangement_executor: Option<Executor>, /// The executor for stream. - stream_executor: Option<Box<dyn Executor>>, + stream_executor: Option<Executor>, /// The last received barrier. last_barrier: Option<Barrier>, @@ -83,20 +80,8 @@ pub struct LookupExecutor<S: StateStore> { } #[async_trait] -impl<S: StateStore> Executor for LookupExecutor<S> { +impl<S: StateStore> Execute for LookupExecutor<S> { fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } diff --git a/src/stream/src/executor/lookup/impl_.rs b/src/stream/src/executor/lookup/impl_.rs index a503a73a0c8a0..db2511e19e7bd 100644 --- a/src/stream/src/executor/lookup/impl_.rs +++ b/src/stream/src/executor/lookup/impl_.rs @@ -46,11 +46,11 @@ pub struct LookupExecutorParams<S: StateStore> { /// The side for arrangement. Currently, it should be a /// `MaterializeExecutor`. - pub arrangement: Box<dyn Executor>, + pub arrangement: Executor, /// The side for stream. It can be any stream, but it will generally be a /// `MaterializeExecutor`. - pub stream: Box<dyn Executor>, + pub stream: Executor, /// Should be the same as [`ColumnDesc`] in the arrangement. /// @@ -198,7 +198,6 @@ impl<S: StateStore> LookupExecutor<S> { Self { ctx, - info, chunk_data_types, last_barrier: None, stream_executor: Some(stream), diff --git a/src/stream/src/executor/lookup/sides.rs b/src/stream/src/executor/lookup/sides.rs index 26bfdd39ec401..6e64519503a93 100644 --- a/src/stream/src/executor/lookup/sides.rs +++ b/src/stream/src/executor/lookup/sides.rs @@ -211,10 +211,7 @@ pub async fn align_barrier(mut left: BoxedMessageStream, mut right: BoxedMessage /// * Barrier (prev = `[2`], current = `[3`]) /// * `[Msg`] Arrangement (batch) #[try_stream(ok = ArrangeMessage, error = StreamExecutorError)] -pub async fn stream_lookup_arrange_prev_epoch( - stream: Box<dyn Executor>, - arrangement: Box<dyn Executor>, -) { +pub async fn stream_lookup_arrange_prev_epoch(stream: Executor, arrangement: Executor) { let mut input = pin!(align_barrier(stream.execute(), arrangement.execute())); let mut arrange_buf = vec![]; let mut stream_side_end = false; @@ -295,10 +292,7 @@ pub async fn stream_lookup_arrange_prev_epoch( /// * `[Do`] lookup `a` in arrangement of epoch `[2`] (current epoch) /// * Barrier (prev = `[2`], current = `[3`]) #[try_stream(ok = ArrangeMessage, error = StreamExecutorError)] -pub async fn stream_lookup_arrange_this_epoch( - stream: Box<dyn Executor>, - arrangement: Box<dyn Executor>, -) { +pub async fn stream_lookup_arrange_this_epoch(stream: Executor, arrangement: Executor) { let mut input = pin!(align_barrier(stream.execute(), arrangement.execute())); let mut stream_buf = vec![]; let mut arrange_buf = vec![]; @@ -432,14 +426,16 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - let (mut tx_l, source_l) = MockSource::channel(schema.clone(), vec![1]); - let (tx_r, source_r) = MockSource::channel(schema, vec![1]); - - let mut stream = stream_lookup_arrange_this_epoch( - Box::new(source_l.stop_on_finish(false)), - Box::new(source_r.stop_on_finish(false)), - ) - .boxed(); + let (mut tx_l, source_l) = MockSource::channel(); + let source_l = source_l + .stop_on_finish(false) + .into_executor(schema.clone(), vec![1]); + let (tx_r, source_r) = MockSource::channel(); + let source_r = source_r + .stop_on_finish(false) + .into_executor(schema, vec![1]); + + let mut stream = stream_lookup_arrange_this_epoch(source_l, source_r).boxed(); // Simulate recovery test drop(tx_r); diff --git a/src/stream/src/executor/lookup/tests.rs b/src/stream/src/executor/lookup/tests.rs index 7d152e8a5f4e7..9b335fcca6428 100644 --- a/src/stream/src/executor/lookup/tests.rs +++ b/src/stream/src/executor/lookup/tests.rs @@ -30,8 +30,8 @@ use crate::executor::lookup::impl_::LookupExecutorParams; use crate::executor::lookup::LookupExecutor; use crate::executor::test_utils::*; use crate::executor::{ - ActorContext, Barrier, BoxedMessageStream, Executor, ExecutorInfo, MaterializeExecutor, - Message, PkIndices, + ActorContext, Barrier, BoxedMessageStream, Execute, Executor, ExecutorInfo, + MaterializeExecutor, Message, PkIndices, }; fn arrangement_col_descs() -> Vec<ColumnDesc> { @@ -68,10 +68,7 @@ fn arrangement_col_arrange_rules_join_key() -> Vec<ColumnOrder> { /// | + | 2337 | 8 | 3 | /// | - | 2333 | 6 | 3 | /// | b | | | 3 -> 4 | -async fn create_arrangement( - table_id: TableId, - memory_state_store: MemoryStateStore, -) -> Box<dyn Executor + Send> { +async fn create_arrangement(table_id: TableId, memory_state_store: MemoryStateStore) -> Executor { // Two columns of int32 type, the second column is arrange key. let columns = arrangement_col_descs(); @@ -101,30 +98,32 @@ async fn create_arrangement( .collect_vec(), ); - let source = MockSource::with_messages( - schema, - vec![0], - vec![ - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(chunk1), - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(chunk2), - Message::Barrier(Barrier::new_test_barrier(4)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk1), + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(4)), + ]) + .into_executor(schema, vec![0]); - Box::new( + Executor::new( + ExecutorInfo { + schema: source.schema().clone(), + pk_indices: source.pk_indices().to_vec(), + identity: "MaterializeExecutor".to_string(), + }, MaterializeExecutor::for_test( - Box::new(source), + source, memory_state_store, table_id, arrangement_col_arrange_rules(), column_ids, - 1, Arc::new(AtomicU64::new(0)), ConflictBehavior::NoCheck, ) - .await, + .await + .boxed(), ) } @@ -139,7 +138,7 @@ async fn create_arrangement( /// | b | | | 2 -> 3 | /// | - | 6 | 1 | 3 | /// | b | | | 3 -> 4 | -fn create_source() -> Box<dyn Executor + Send> { +fn create_source() -> Executor { let columns = vec![ ColumnDesc::new_atomic(DataType::Int64, "join_column", 1), ColumnDesc::new_atomic(DataType::Int64, "rowid_column", 2), @@ -163,19 +162,14 @@ fn create_source() -> Box<dyn Executor + Send> { .collect_vec(), ); - let source = MockSource::with_messages( - schema, - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(chunk1), - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(chunk2), - Message::Barrier(Barrier::new_test_barrier(4)), - ], - ); - - Box::new(source) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk1), + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(4)), + ]) + .into_executor(schema, PkIndices::new()) } async fn next_msg(buffer: &mut Vec<Message>, executor: &mut BoxedMessageStream) { diff --git a/src/stream/src/executor/lookup_union.rs b/src/stream/src/executor/lookup_union.rs index c52d562277459..2ffddbda45b46 100644 --- a/src/stream/src/executor/lookup_union.rs +++ b/src/stream/src/executor/lookup_union.rs @@ -18,35 +18,28 @@ use futures::future::{join_all, select, Either}; use futures::{FutureExt, SinkExt, StreamExt}; use futures_async_stream::try_stream; use itertools::Itertools; -use risingwave_common::catalog::Schema; use super::error::StreamExecutorError; -use super::*; -use crate::executor::{BoxedMessageStream, ExecutorInfo}; +use super::{Barrier, BoxedMessageStream, Execute, Executor, Message}; /// Merges data from multiple inputs with order. If `order = [2, 1, 0]`, then /// it will first pipe data from the third input; after the third input gets a barrier, it will then /// pipe the second, and finally the first. In the future we could have more efficient /// implementation. pub struct LookupUnionExecutor { - info: ExecutorInfo, - inputs: Vec<BoxedExecutor>, + inputs: Vec<Executor>, order: Vec<usize>, } impl std::fmt::Debug for LookupUnionExecutor { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("LookupUnionExecutor") - .field("schema", &self.info.schema) - .field("pk_indices", &self.info.pk_indices) - .finish() + f.debug_struct("LookupUnionExecutor").finish() } } impl LookupUnionExecutor { - pub fn new(info: ExecutorInfo, inputs: Vec<BoxedExecutor>, order: Vec<u32>) -> Self { + pub fn new(inputs: Vec<Executor>, order: Vec<u32>) -> Self { Self { - info, inputs, order: order.iter().map(|x| *x as _).collect(), } @@ -54,22 +47,10 @@ impl LookupUnionExecutor { } #[async_trait] -impl Executor for LookupUnionExecutor { +impl Execute for LookupUnionExecutor { fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } impl LookupUnionExecutor { @@ -154,52 +135,36 @@ mod tests { let schema = Schema { fields: vec![Field::unnamed(DataType::Int64)], }; - let source0 = MockSource::with_messages( - schema.clone(), - vec![0], - vec![ - Message::Chunk(StreamChunk::from_pretty("I\n + 1")), - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(StreamChunk::from_pretty("I\n + 2")), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(StreamChunk::from_pretty("I\n + 3")), - Message::Barrier(Barrier::new_test_barrier(3)), - ], - ) - .stop_on_finish(false); - let source1 = MockSource::with_messages( - schema.clone(), - vec![0], - vec![ - Message::Chunk(StreamChunk::from_pretty("I\n + 11")), - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(StreamChunk::from_pretty("I\n + 12")), - Message::Barrier(Barrier::new_test_barrier(2)), - ], - ) - .stop_on_finish(false); - let source2 = MockSource::with_messages( - schema, - vec![0], - vec![ - Message::Chunk(StreamChunk::from_pretty("I\n + 21")), - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(StreamChunk::from_pretty("I\n + 22")), - Message::Barrier(Barrier::new_test_barrier(2)), - ], - ) - .stop_on_finish(false); - - let executor = Box::new(LookupUnionExecutor::new( - ExecutorInfo { - schema: source0.schema().clone(), - pk_indices: vec![0], - identity: "LookupUnionExecutor".to_string(), - }, - vec![Box::new(source0), Box::new(source1), Box::new(source2)], - vec![2, 1, 0], - )) - .execute(); + let source0 = MockSource::with_messages(vec![ + Message::Chunk(StreamChunk::from_pretty("I\n + 1")), + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(StreamChunk::from_pretty("I\n + 2")), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(StreamChunk::from_pretty("I\n + 3")), + Message::Barrier(Barrier::new_test_barrier(3)), + ]) + .stop_on_finish(false) + .into_executor(schema.clone(), vec![0]); + let source1 = MockSource::with_messages(vec![ + Message::Chunk(StreamChunk::from_pretty("I\n + 11")), + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(StreamChunk::from_pretty("I\n + 12")), + Message::Barrier(Barrier::new_test_barrier(2)), + ]) + .stop_on_finish(false) + .into_executor(schema.clone(), vec![0]); + let source2 = MockSource::with_messages(vec![ + Message::Chunk(StreamChunk::from_pretty("I\n + 21")), + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(StreamChunk::from_pretty("I\n + 22")), + Message::Barrier(Barrier::new_test_barrier(2)), + ]) + .stop_on_finish(false) + .into_executor(schema, vec![0]); + + let executor = LookupUnionExecutor::new(vec![source0, source1, source2], vec![2, 1, 0]) + .boxed() + .execute(); let outputs: Vec<_> = executor.try_collect().await.unwrap(); assert_eq!( diff --git a/src/stream/src/executor/merge.rs b/src/stream/src/executor/merge.rs index 145a937fcdb2e..e5b88dd274e1e 100644 --- a/src/stream/src/executor/merge.rs +++ b/src/stream/src/executor/merge.rs @@ -20,7 +20,6 @@ use anyhow::Context as _; use futures::stream::{FusedStream, FuturesUnordered, StreamFuture}; use futures::{pin_mut, Stream, StreamExt}; use futures_async_stream::try_stream; -use risingwave_common::catalog::Schema; use tokio::time::Instant; use super::error::StreamExecutorError; @@ -38,9 +37,6 @@ pub struct MergeExecutor { /// The context of the actor. actor_context: ActorContextRef, - /// Logical Operator Info - info: ExecutorInfo, - /// Upstream channels. upstreams: Vec<BoxedInput>, @@ -61,7 +57,6 @@ impl MergeExecutor { #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, fragment_id: FragmentId, upstream_fragment_id: FragmentId, inputs: Vec<BoxedInput>, @@ -71,7 +66,6 @@ impl MergeExecutor { ) -> Self { Self { actor_context: ctx, - info, upstreams: inputs, fragment_id, upstream_fragment_id, @@ -81,17 +75,12 @@ impl MergeExecutor { } #[cfg(test)] - pub fn for_test(inputs: Vec<super::exchange::permit::Receiver>, schema: Schema) -> Self { + pub fn for_test(inputs: Vec<super::exchange::permit::Receiver>) -> Self { use super::exchange::input::LocalInput; use crate::executor::exchange::input::Input; Self::new( ActorContext::for_test(114), - ExecutorInfo { - schema, - pk_indices: vec![], - identity: "MergeExecutor".to_string(), - }, 514, 1919, inputs @@ -245,22 +234,10 @@ impl MergeExecutor { } } -impl Executor for MergeExecutor { +impl Execute for MergeExecutor { fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } /// A stream for merging messages from multiple upstreams. @@ -464,7 +441,7 @@ mod tests { use super::*; use crate::executor::exchange::input::RemoteInput; use crate::executor::exchange::permit::channel_for_test; - use crate::executor::{Barrier, Executor, Mutation}; + use crate::executor::{Barrier, Execute, Mutation}; use crate::task::test_utils::helper_make_local_actor; fn build_test_chunk(epoch: u64) -> StreamChunk { @@ -483,7 +460,7 @@ mod tests { txs.push(tx); rxs.push(rx); } - let merger = MergeExecutor::for_test(rxs, Schema::default()); + let merger = MergeExecutor::for_test(rxs); let mut handles = Vec::with_capacity(CHANNEL_NUMBER); let epochs = (10..1000u64).step_by(10).collect_vec(); @@ -556,8 +533,6 @@ mod tests { #[tokio::test] async fn test_configuration_change() { - let schema = Schema { fields: vec![] }; - let actor_id = 233; let (untouched, old, new) = (234, 235, 238); // upstream actors let ctx = Arc::new(SharedContext::for_test()); @@ -592,13 +567,8 @@ mod tests { .try_collect() .unwrap(); - let merge = MergeExecutor::new( + let mut merge = MergeExecutor::new( ActorContext::for_test(actor_id), - ExecutorInfo { - schema, - pk_indices: vec![], - identity: "MergeExecutor".to_string(), - }, fragment_id, upstream_fragment_id, inputs, @@ -609,8 +579,6 @@ mod tests { .boxed() .execute(); - pin_mut!(merge); - // 2. Take downstream receivers. let txs = [untouched, old, new] .into_iter() diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index 40ae252c03cf0..1e82768096f36 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -20,7 +20,6 @@ use await_tree::InstrumentAwait; use enum_as_inner::EnumAsInner; use futures::stream::BoxStream; use futures::{Stream, StreamExt}; -use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::StreamChunk; use risingwave_common::buffer::Bitmap; @@ -154,7 +153,6 @@ pub use wrapper::WrapperExecutor; use self::barrier_align::AlignedMessageStream; -pub type BoxedExecutor = Box<dyn Executor>; pub type MessageStreamItem = StreamExecutorResult<Message>; pub type BoxedMessageStream = BoxStream<'static, MessageStreamItem>; @@ -166,48 +164,27 @@ pub trait MessageStream = futures::Stream<Item = MessageStreamItem> + Send; /// Static information of an executor. #[derive(Debug, Default, Clone)] pub struct ExecutorInfo { - /// See [`Executor::schema`]. + /// The schema of the OUTPUT of the executor. pub schema: Schema, - /// See [`Executor::pk_indices`]. + /// The primary key indices of the OUTPUT of the executor. + /// Schema is used by both OLAP and streaming, therefore + /// pk indices are maintained independently. pub pk_indices: PkIndices, - /// See [`Executor::identity`]. + /// Identity of the executor. pub identity: String, } -/// `Executor` supports handling of control messages. -pub trait Executor: Send + 'static { +/// [`Execute`] describes the methods an executor should implement to handle control messages. +pub trait Execute: Send + 'static { fn execute(self: Box<Self>) -> BoxedMessageStream; - /// Return the schema of the OUTPUT of the executor. - fn schema(&self) -> &Schema; - - /// Return the primary key indices of the OUTPUT of the executor. - /// Schema is used by both OLAP and streaming, therefore - /// pk indices are maintained independently. - fn pk_indices(&self) -> PkIndicesRef<'_>; - - /// Identity of the executor. - fn identity(&self) -> &str; - fn execute_with_epoch(self: Box<Self>, _epoch: u64) -> BoxedMessageStream { self.execute() } - #[inline(always)] - fn info(&self) -> ExecutorInfo { - let schema = self.schema().to_owned(); - let pk_indices = self.pk_indices().to_owned(); - let identity = self.identity().to_owned(); - ExecutorInfo { - schema, - pk_indices, - identity, - } - } - - fn boxed(self) -> BoxedExecutor + fn boxed(self) -> Box<dyn Execute> where Self: Sized + Send + 'static, { @@ -215,12 +192,64 @@ pub trait Executor: Send + 'static { } } -impl std::fmt::Debug for BoxedExecutor { +/// [`Executor`] combines the static information ([`ExecutorInfo`]) and the executable object to +/// handle messages ([`Execute`]). +pub struct Executor { + info: ExecutorInfo, + execute: Box<dyn Execute>, +} + +impl Executor { + pub fn new(info: ExecutorInfo, execute: Box<dyn Execute>) -> Self { + Self { info, execute } + } + + pub fn info(&self) -> &ExecutorInfo { + &self.info + } + + pub fn schema(&self) -> &Schema { + &self.info.schema + } + + pub fn pk_indices(&self) -> PkIndicesRef<'_> { + &self.info.pk_indices + } + + pub fn identity(&self) -> &str { + &self.info.identity + } + + pub fn execute(self) -> BoxedMessageStream { + self.execute.execute() + } + + pub fn execute_with_epoch(self, epoch: u64) -> BoxedMessageStream { + self.execute.execute_with_epoch(epoch) + } +} + +impl std::fmt::Debug for Executor { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.write_str(self.identity()) } } +impl From<(ExecutorInfo, Box<dyn Execute>)> for Executor { + fn from((info, execute): (ExecutorInfo, Box<dyn Execute>)) -> Self { + Self::new(info, execute) + } +} + +impl<E> From<(ExecutorInfo, E)> for Executor +where + E: Execute, +{ + fn from((info, execute): (ExecutorInfo, E)) -> Self { + Self::new(info, execute.boxed()) + } +} + pub const INVALID_EPOCH: u64 = 0; type UpstreamFragmentId = FragmentId; diff --git a/src/stream/src/executor/mview/materialize.rs b/src/stream/src/executor/mview/materialize.rs index 69e238a476f74..c96f875765e8e 100644 --- a/src/stream/src/executor/mview/materialize.rs +++ b/src/stream/src/executor/mview/materialize.rs @@ -42,16 +42,16 @@ use crate::common::table::state_table::StateTableInner; use crate::executor::error::StreamExecutorError; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ - expect_first_barrier, ActorContext, ActorContextRef, AddMutation, BoxedExecutor, - BoxedMessageStream, Executor, ExecutorInfo, Message, Mutation, PkIndicesRef, - StreamExecutorResult, UpdateMutation, + expect_first_barrier, ActorContext, ActorContextRef, AddMutation, BoxedMessageStream, Execute, + Executor, Message, Mutation, StreamExecutorResult, UpdateMutation, }; use crate::task::{ActorId, AtomicU64Ref}; /// `MaterializeExecutor` materializes changes in stream into a materialized view on storage. pub struct MaterializeExecutor<S: StateStore, SD: ValueRowSerde> { - input: BoxedExecutor, - info: ExecutorInfo, + input: Executor, + + schema: Schema, state_table: StateTableInner<S, SD>, @@ -71,8 +71,8 @@ impl<S: StateStore, SD: ValueRowSerde> MaterializeExecutor<S, SD> { /// should be `None`. #[allow(clippy::too_many_arguments)] pub async fn new( - input: BoxedExecutor, - info: ExecutorInfo, + input: Executor, + schema: Schema, store: S, arrange_key: Vec<ColumnOrder>, actor_context: ActorContextRef, @@ -99,7 +99,7 @@ impl<S: StateStore, SD: ValueRowSerde> MaterializeExecutor<S, SD> { Self { input, - info, + schema, state_table, arrange_key_indices, actor_context, @@ -115,7 +115,7 @@ impl<S: StateStore, SD: ValueRowSerde> MaterializeExecutor<S, SD> { let actor_id_str = self.actor_context.id.to_string(); let fragment_id_str = self.actor_context.fragment_id.to_string(); - let data_types = self.schema().data_types().clone(); + let data_types = self.schema.data_types(); let mut input = self.input.execute(); let barrier = expect_first_barrier(&mut input).await?; @@ -264,12 +264,11 @@ impl<S: StateStore> MaterializeExecutor<S, BasicSerde> { /// Create a new `MaterializeExecutor` without distribution info for test purpose. #[allow(clippy::too_many_arguments)] pub async fn for_test( - input: BoxedExecutor, + input: Executor, store: S, table_id: TableId, keys: Vec<ColumnOrder>, column_ids: Vec<ColumnId>, - executor_id: u64, watermark_epoch: AtomicU64Ref, conflict_behavior: ConflictBehavior, ) -> Self { @@ -293,14 +292,10 @@ impl<S: StateStore> MaterializeExecutor<S, BasicSerde> { Self { input, + schema, state_table, arrange_key_indices: arrange_columns.clone(), actor_context: ActorContext::for_test(0), - info: ExecutorInfo { - schema, - pk_indices: arrange_columns, - identity: format!("MaterializeExecutor {:X}", executor_id), - }, materialize_cache: MaterializeCache::new(watermark_epoch, MetricsInfo::for_test()), conflict_behavior, } @@ -429,32 +424,15 @@ impl MaterializeBuffer { self.buffer } } -impl<S: StateStore, SD: ValueRowSerde> Executor for MaterializeExecutor<S, SD> { +impl<S: StateStore, SD: ValueRowSerde> Execute for MaterializeExecutor<S, SD> { fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - - fn info(&self) -> ExecutorInfo { - self.info.clone() - } } impl<S: StateStore, SD: ValueRowSerde> std::fmt::Debug for MaterializeExecutor<S, SD> { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("MaterializeExecutor") - .field("info", &self.info()) .field("arrange_key_indices", &self.arrange_key_indices) .finish() } @@ -676,17 +654,14 @@ mod tests { ); // Prepare stream executors. - let source = MockSource::with_messages( - schema.clone(), - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(chunk1), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(chunk2), - Message::Barrier(Barrier::new_test_barrier(3)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(3)), + ]) + .into_executor(schema.clone(), PkIndices::new()); let order_types = vec![OrderType::ascending()]; let column_descs = vec![ @@ -703,19 +678,17 @@ mod tests { vec![0, 1], ); - let mut materialize_executor = Box::new( - MaterializeExecutor::for_test( - Box::new(source), - memory_state_store, - table_id, - vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids, - 1, - Arc::new(AtomicU64::new(0)), - ConflictBehavior::NoCheck, - ) - .await, + let mut materialize_executor = MaterializeExecutor::for_test( + source, + memory_state_store, + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::NoCheck, ) + .await + .boxed() .execute(); materialize_executor.next().await.transpose().unwrap(); @@ -784,17 +757,14 @@ mod tests { ); // Prepare stream executors. - let source = MockSource::with_messages( - schema.clone(), - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(chunk1), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(chunk2), - Message::Barrier(Barrier::new_test_barrier(3)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(3)), + ]) + .into_executor(schema.clone(), PkIndices::new()); let order_types = vec![OrderType::ascending()]; let column_descs = vec![ @@ -811,19 +781,17 @@ mod tests { vec![0, 1], ); - let mut materialize_executor = Box::new( - MaterializeExecutor::for_test( - Box::new(source), - memory_state_store, - table_id, - vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids, - 1, - Arc::new(AtomicU64::new(0)), - ConflictBehavior::Overwrite, - ) - .await, + let mut materialize_executor = MaterializeExecutor::for_test( + source, + memory_state_store, + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::Overwrite, ) + .await + .boxed() .execute(); materialize_executor.next().await.transpose().unwrap(); @@ -880,18 +848,15 @@ mod tests { ); // Prepare stream executors. - let source = MockSource::with_messages( - schema.clone(), - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(chunk1), - Message::Chunk(chunk2), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(chunk3), - Message::Barrier(Barrier::new_test_barrier(3)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk3), + Message::Barrier(Barrier::new_test_barrier(3)), + ]) + .into_executor(schema.clone(), PkIndices::new()); let order_types = vec![OrderType::ascending()]; let column_descs = vec![ @@ -908,19 +873,17 @@ mod tests { vec![0, 1], ); - let mut materialize_executor = Box::new( - MaterializeExecutor::for_test( - Box::new(source), - memory_state_store, - table_id, - vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids, - 1, - Arc::new(AtomicU64::new(0)), - ConflictBehavior::Overwrite, - ) - .await, + let mut materialize_executor = MaterializeExecutor::for_test( + source, + memory_state_store, + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::Overwrite, ) + .await + .boxed() .execute(); materialize_executor.next().await.transpose().unwrap(); @@ -1012,19 +975,16 @@ mod tests { ); // Prepare stream executors. - let source = MockSource::with_messages( - schema.clone(), - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(chunk1), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(chunk2), - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(chunk3), - Message::Barrier(Barrier::new_test_barrier(4)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(chunk3), + Message::Barrier(Barrier::new_test_barrier(4)), + ]) + .into_executor(schema.clone(), PkIndices::new()); let order_types = vec![OrderType::ascending()]; let column_descs = vec![ @@ -1041,19 +1001,17 @@ mod tests { vec![0, 1], ); - let mut materialize_executor = Box::new( - MaterializeExecutor::for_test( - Box::new(source), - memory_state_store, - table_id, - vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids, - 1, - Arc::new(AtomicU64::new(0)), - ConflictBehavior::Overwrite, - ) - .await, + let mut materialize_executor = MaterializeExecutor::for_test( + source, + memory_state_store, + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::Overwrite, ) + .await + .boxed() .execute(); materialize_executor.next().await.transpose().unwrap(); @@ -1196,18 +1154,15 @@ mod tests { ); // Prepare stream executors. - let source = MockSource::with_messages( - schema.clone(), - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(chunk1), - Message::Chunk(chunk2), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(chunk3), - Message::Barrier(Barrier::new_test_barrier(3)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk3), + Message::Barrier(Barrier::new_test_barrier(3)), + ]) + .into_executor(schema.clone(), PkIndices::new()); let order_types = vec![OrderType::ascending()]; let column_descs = vec![ @@ -1224,19 +1179,17 @@ mod tests { vec![0, 1], ); - let mut materialize_executor = Box::new( - MaterializeExecutor::for_test( - Box::new(source), - memory_state_store, - table_id, - vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids, - 1, - Arc::new(AtomicU64::new(0)), - ConflictBehavior::IgnoreConflict, - ) - .await, + let mut materialize_executor = MaterializeExecutor::for_test( + source, + memory_state_store, + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::IgnoreConflict, ) + .await + .boxed() .execute(); materialize_executor.next().await.transpose().unwrap(); @@ -1307,15 +1260,12 @@ mod tests { ); // Prepare stream executors. - let source = MockSource::with_messages( - schema.clone(), - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(chunk1), - Message::Barrier(Barrier::new_test_barrier(2)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Barrier(Barrier::new_test_barrier(2)), + ]) + .into_executor(schema.clone(), PkIndices::new()); let order_types = vec![OrderType::ascending()]; let column_descs = vec![ @@ -1332,19 +1282,17 @@ mod tests { vec![0, 1], ); - let mut materialize_executor = Box::new( - MaterializeExecutor::for_test( - Box::new(source), - memory_state_store, - table_id, - vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids, - 1, - Arc::new(AtomicU64::new(0)), - ConflictBehavior::IgnoreConflict, - ) - .await, + let mut materialize_executor = MaterializeExecutor::for_test( + source, + memory_state_store, + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::IgnoreConflict, ) + .await + .boxed() .execute(); let _msg1 = materialize_executor .next() @@ -1426,19 +1374,16 @@ mod tests { ); // Prepare stream executors. - let source = MockSource::with_messages( - schema.clone(), - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(chunk1), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(chunk2), - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(chunk3), - Message::Barrier(Barrier::new_test_barrier(4)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(chunk3), + Message::Barrier(Barrier::new_test_barrier(4)), + ]) + .into_executor(schema.clone(), PkIndices::new()); let order_types = vec![OrderType::ascending()]; let column_descs = vec![ @@ -1455,19 +1400,17 @@ mod tests { vec![0, 1], ); - let mut materialize_executor = Box::new( - MaterializeExecutor::for_test( - Box::new(source), - memory_state_store, - table_id, - vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids, - 1, - Arc::new(AtomicU64::new(0)), - ConflictBehavior::IgnoreConflict, - ) - .await, + let mut materialize_executor = MaterializeExecutor::for_test( + source, + memory_state_store, + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::IgnoreConflict, ) + .await + .boxed() .execute(); materialize_executor.next().await.transpose().unwrap(); @@ -1635,21 +1578,20 @@ mod tests { .chain(iter::once(Message::Barrier(Barrier::new_test_barrier(2)))) .collect(); // Prepare stream executors. - let source = MockSource::with_messages(schema.clone(), PkIndices::new(), messages); - - let mut materialize_executor = Box::new( - MaterializeExecutor::for_test( - Box::new(source), - memory_state_store.clone(), - table_id, - vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids, - 1, - Arc::new(AtomicU64::new(0)), - conflict_behavior, - ) - .await, + let source = + MockSource::with_messages(messages).into_executor(schema.clone(), PkIndices::new()); + + let mut materialize_executor = MaterializeExecutor::for_test( + source, + memory_state_store.clone(), + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + Arc::new(AtomicU64::new(0)), + conflict_behavior, ) + .await + .boxed() .execute(); materialize_executor.expect_barrier().await; diff --git a/src/stream/src/executor/no_op.rs b/src/stream/src/executor/no_op.rs index f116eced8864d..ac12bf99d5d7f 100644 --- a/src/stream/src/executor/no_op.rs +++ b/src/stream/src/executor/no_op.rs @@ -12,44 +12,23 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::Schema; - -use super::{ - ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, PkIndicesRef, -}; +use super::{ActorContextRef, BoxedMessageStream, Execute, Executor}; /// No-op executor directly forwards the input stream. Currently used to break the multiple edges in /// the fragment graph. pub struct NoOpExecutor { _ctx: ActorContextRef, - info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, } impl NoOpExecutor { - pub fn new(ctx: ActorContextRef, info: ExecutorInfo, input: BoxedExecutor) -> Self { - Self { - _ctx: ctx, - info, - input, - } + pub fn new(ctx: ActorContextRef, input: Executor) -> Self { + Self { _ctx: ctx, input } } } -impl Executor for NoOpExecutor { +impl Execute for NoOpExecutor { fn execute(self: Box<Self>) -> BoxedMessageStream { self.input.execute() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } diff --git a/src/stream/src/executor/now.rs b/src/stream/src/executor/now.rs index 9d00646a9bcc1..755e48f325965 100644 --- a/src/stream/src/executor/now.rs +++ b/src/stream/src/executor/now.rs @@ -18,7 +18,6 @@ use std::ops::Bound::Unbounded; use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::catalog::Schema; use risingwave_common::row::{self, OwnedRow}; use risingwave_common::types::{DataType, Datum}; use risingwave_storage::StateStore; @@ -26,13 +25,12 @@ use tokio::sync::mpsc::UnboundedReceiver; use tokio_stream::wrappers::UnboundedReceiverStream; use super::{ - Barrier, BoxedMessageStream, Executor, ExecutorInfo, Message, Mutation, PkIndicesRef, - StreamExecutorError, Watermark, + Barrier, BoxedMessageStream, Execute, Message, Mutation, StreamExecutorError, Watermark, }; use crate::common::table::state_table::StateTable; pub struct NowExecutor<S: StateStore> { - info: ExecutorInfo, + data_types: Vec<DataType>, /// Receiver of barrier channel. barrier_receiver: UnboundedReceiver<Barrier>, @@ -42,24 +40,23 @@ pub struct NowExecutor<S: StateStore> { impl<S: StateStore> NowExecutor<S> { pub fn new( - info: ExecutorInfo, + data_types: Vec<DataType>, barrier_receiver: UnboundedReceiver<Barrier>, state_table: StateTable<S>, ) -> Self { Self { - info, + data_types, barrier_receiver, state_table, } } #[try_stream(ok = Message, error = StreamExecutorError)] - async fn into_stream(self) { + async fn execute_inner(self) { let Self { + data_types, barrier_receiver, mut state_table, - info, - .. } = self; // Whether the executor is paused. @@ -131,15 +128,12 @@ impl<S: StateStore> NowExecutor<S> { let row = row::once(×tamp); state_table.update(last_row, row); - StreamChunk::from_rows( - &[(Op::Delete, last_row), (Op::Insert, row)], - &info.schema.data_types(), - ) + StreamChunk::from_rows(&[(Op::Delete, last_row), (Op::Insert, row)], &data_types) } else { let row = row::once(×tamp); state_table.insert(row); - StreamChunk::from_rows(&[(Op::Insert, row)], &info.schema.data_types()) + StreamChunk::from_rows(&[(Op::Insert, row)], &data_types) }; yield Message::Chunk(stream_chunk); @@ -155,28 +149,16 @@ impl<S: StateStore> NowExecutor<S> { } } -impl<S: StateStore> Executor for NowExecutor<S> { +impl<S: StateStore> Execute for NowExecutor<S> { fn execute(self: Box<Self>) -> BoxedMessageStream { - self.into_stream().boxed() - } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity + self.execute_inner().boxed() } } #[cfg(test)] mod tests { use risingwave_common::array::StreamChunk; - use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, Schema, TableId}; + use risingwave_common::catalog::{ColumnDesc, ColumnId, TableId}; use risingwave_common::test_prelude::StreamChunkTestExt; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_storage::memory::MemoryStateStore; @@ -186,8 +168,7 @@ mod tests { use crate::common::table::state_table::StateTable; use crate::executor::test_utils::StreamExecutorTestExt; use crate::executor::{ - Barrier, BoxedMessageStream, Executor, ExecutorInfo, Mutation, StreamExecutorResult, - Watermark, + Barrier, BoxedMessageStream, Execute, Mutation, StreamExecutorResult, Watermark, }; #[tokio::test] @@ -410,22 +391,8 @@ mod tests { let (sender, barrier_receiver) = unbounded_channel(); - let schema = Schema::new(vec![Field { - data_type: DataType::Timestamptz, - name: String::from("now"), - sub_fields: vec![], - type_name: String::default(), - }]); - - let now_executor = NowExecutor::new( - ExecutorInfo { - schema, - pk_indices: vec![], - identity: "NowExecutor".to_string(), - }, - barrier_receiver, - state_table, - ); - (sender, Box::new(now_executor).execute()) + let now_executor = + NowExecutor::new(vec![DataType::Timestamptz], barrier_receiver, state_table); + (sender, now_executor.boxed().execute()) } } diff --git a/src/stream/src/executor/over_window/eowc.rs b/src/stream/src/executor/over_window/eowc.rs index 601a2f536aa00..32f1fafe9b8db 100644 --- a/src/stream/src/executor/over_window/eowc.rs +++ b/src/stream/src/executor/over_window/eowc.rs @@ -39,8 +39,8 @@ use crate::cache::{new_unbounded, ManagedLruCache}; use crate::common::metrics::MetricsInfo; use crate::common::table::state_table::StateTable; use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult, + expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, + StreamExecutorError, StreamExecutorResult, }; use crate::task::AtomicU64Ref; @@ -95,14 +95,14 @@ type PartitionCache = ManagedLruCache<MemcmpEncoded, Partition>; // TODO(rc): us /// - `WindowState` should output agg result for `curr output row`. /// - Recover: iterate through state table, push rows to `WindowState`, ignore ready windows. pub struct EowcOverWindowExecutor<S: StateStore> { - input: Box<dyn Executor>, + input: Executor, inner: ExecutorInner<S>, } struct ExecutorInner<S: StateStore> { actor_ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, calls: Vec<WindowFuncCall>, input_pk_indices: Vec<usize>, partition_key_indices: Vec<usize>, @@ -117,30 +117,18 @@ struct ExecutionVars<S: StateStore> { _phantom: PhantomData<S>, } -impl<S: StateStore> Executor for EowcOverWindowExecutor<S> { +impl<S: StateStore> Execute for EowcOverWindowExecutor<S> { fn execute(self: Box<Self>) -> BoxedMessageStream { self.executor_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.inner.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.inner.info.pk_indices - } - - fn identity(&self) -> &str { - &self.inner.info.identity - } } pub struct EowcOverWindowExecutorArgs<S: StateStore> { pub actor_ctx: ActorContextRef, - pub info: ExecutorInfo, - pub input: BoxedExecutor, + pub input: Executor, + pub schema: Schema, pub calls: Vec<WindowFuncCall>, pub partition_key_indices: Vec<usize>, pub order_key_index: usize, @@ -150,13 +138,13 @@ pub struct EowcOverWindowExecutorArgs<S: StateStore> { impl<S: StateStore> EowcOverWindowExecutor<S> { pub fn new(args: EowcOverWindowExecutorArgs<S>) -> Self { - let input_info = args.input.info(); + let input_info = args.input.info().clone(); Self { input: args.input, inner: ExecutorInner { actor_ctx: args.actor_ctx, - info: args.info, + schema: args.schema, calls: args.calls, input_pk_indices: input_info.pk_indices, partition_key_indices: args.partition_key_indices, @@ -237,7 +225,7 @@ impl<S: StateStore> EowcOverWindowExecutor<S> { vars: &mut ExecutionVars<S>, chunk: StreamChunk, ) -> StreamExecutorResult<Option<StreamChunk>> { - let mut builders = this.info.schema.create_array_builders(chunk.capacity()); // just an estimate + let mut builders = this.schema.create_array_builders(chunk.capacity()); // just an estimate // We assume that the input is sorted by order key. for record in chunk.records() { @@ -318,7 +306,7 @@ impl<S: StateStore> EowcOverWindowExecutor<S> { for key in keys_to_evict { let order_key = memcmp_encoding::decode_row( &key.order_key, - &[this.info.schema[this.order_key_index].data_type()], + &[this.schema[this.order_key_index].data_type()], &[OrderType::ascending()], )?; let state_row_pk = (&partition_key).chain(order_key).chain(key.pk); diff --git a/src/stream/src/executor/over_window/general.rs b/src/stream/src/executor/over_window/general.rs index df4e3ac5c84c2..a7245c57f368c 100644 --- a/src/stream/src/executor/over_window/general.rs +++ b/src/stream/src/executor/over_window/general.rs @@ -23,6 +23,7 @@ use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::stream_record::Record; use risingwave_common::array::{Op, RowRef, StreamChunk}; +use risingwave_common::catalog::Schema; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::session_config::OverWindowCachePolicy as CachePolicy; use risingwave_common::types::{DataType, DefaultOrdered}; @@ -46,8 +47,8 @@ use crate::common::StreamChunkBuilder; use crate::executor::monitor::StreamingMetrics; use crate::executor::over_window::over_partition::AffectedRange; use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, Message, - StreamExecutorError, StreamExecutorResult, + expect_first_barrier, ActorContextRef, Execute, Executor, Message, StreamExecutorError, + StreamExecutorResult, }; use crate::task::AtomicU64Ref; @@ -57,14 +58,14 @@ use crate::task::AtomicU64Ref; /// - State table schema = output schema, state table pk = `partition key | order key | input pk`. /// - Output schema = input schema + window function results. pub struct OverWindowExecutor<S: StateStore> { - input: Box<dyn Executor>, + input: Executor, inner: ExecutorInner<S>, } struct ExecutorInner<S: StateStore> { actor_ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, calls: Vec<WindowFuncCall>, partition_key_indices: Vec<usize>, order_key_indices: Vec<usize>, @@ -98,22 +99,10 @@ struct ExecutionStats { cache_lookup: u64, } -impl<S: StateStore> Executor for OverWindowExecutor<S> { +impl<S: StateStore> Execute for OverWindowExecutor<S> { fn execute(self: Box<Self>) -> crate::executor::BoxedMessageStream { self.executor_inner().boxed() } - - fn schema(&self) -> &risingwave_common::catalog::Schema { - &self.inner.info.schema - } - - fn pk_indices(&self) -> crate::executor::PkIndicesRef<'_> { - &self.inner.info.pk_indices - } - - fn identity(&self) -> &str { - &self.inner.info.identity - } } impl<S: StateStore> ExecutorInner<S> { @@ -145,10 +134,10 @@ impl<S: StateStore> ExecutorInner<S> { pub struct OverWindowExecutorArgs<S: StateStore> { pub actor_ctx: ActorContextRef, - pub info: ExecutorInfo, - pub input: BoxedExecutor, + pub input: Executor, + pub schema: Schema, pub calls: Vec<WindowFuncCall>, pub partition_key_indices: Vec<usize>, pub order_key_indices: Vec<usize>, @@ -164,7 +153,7 @@ pub struct OverWindowExecutorArgs<S: StateStore> { impl<S: StateStore> OverWindowExecutor<S> { pub fn new(args: OverWindowExecutorArgs<S>) -> Self { - let input_info = args.input.info(); + let input_info = args.input.info().clone(); let input_schema = &input_info.schema; let has_unbounded_frame = args @@ -195,7 +184,7 @@ impl<S: StateStore> OverWindowExecutor<S> { input: args.input, inner: ExecutorInner { actor_ctx: args.actor_ctx, - info: args.info, + schema: args.schema, calls: args.calls, partition_key_indices: args.partition_key_indices, order_key_indices: args.order_key_indices, @@ -328,8 +317,7 @@ impl<S: StateStore> OverWindowExecutor<S> { // `input pk` => `Record` let mut key_change_update_buffer: BTreeMap<DefaultOrdered<OwnedRow>, Record<OwnedRow>> = BTreeMap::new(); - let mut chunk_builder = - StreamChunkBuilder::new(this.chunk_size, this.info.schema.data_types()); + let mut chunk_builder = StreamChunkBuilder::new(this.chunk_size, this.schema.data_types()); // Prepare things needed by metrics. let actor_id = this.actor_ctx.id.to_string(); diff --git a/src/stream/src/executor/project.rs b/src/stream/src/executor/project.rs index efc381560e75d..c44d93e22639a 100644 --- a/src/stream/src/executor/project.rs +++ b/src/stream/src/executor/project.rs @@ -14,27 +14,30 @@ use std::fmt::{Debug, Formatter}; +use futures::StreamExt; +use futures_async_stream::try_stream; use multimap::MultiMap; use risingwave_common::array::StreamChunk; -use risingwave_common::catalog::Schema; use risingwave_common::row::{Row, RowExt}; -use risingwave_common::types::ToOwnedDatum; +use risingwave_common::types::{ScalarImpl, ToOwnedDatum}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::expr::NonStrictExpression; -use super::*; +use super::{ + ActorContextRef, BoxedMessageStream, Execute, Executor, Message, StreamExecutorError, + StreamExecutorResult, Watermark, +}; /// `ProjectExecutor` project data with the `expr`. The `expr` takes a chunk of data, /// and returns a new data chunk. And then, `ProjectExecutor` will insert, delete /// or update element into next operator according to the result of the expression. pub struct ProjectExecutor { - input: BoxedExecutor, + input: Executor, inner: Inner, } struct Inner { _ctx: ActorContextRef, - info: ExecutorInfo, /// Expressions of the current projection. exprs: Vec<NonStrictExpression>, @@ -55,8 +58,7 @@ impl ProjectExecutor { #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, - input: Box<dyn Executor>, + input: Executor, exprs: Vec<NonStrictExpression>, watermark_derivations: MultiMap<usize, usize>, nondecreasing_expr_indices: Vec<usize>, @@ -67,7 +69,6 @@ impl ProjectExecutor { input, inner: Inner { _ctx: ctx, - info, exprs, watermark_derivations, nondecreasing_expr_indices, @@ -86,19 +87,7 @@ impl Debug for ProjectExecutor { } } -impl Executor for ProjectExecutor { - fn schema(&self) -> &Schema { - &self.inner.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.inner.info.pk_indices - } - - fn identity(&self) -> &str { - &self.inner.info.identity - } - +impl Execute for ProjectExecutor { fn execute(self: Box<Self>) -> BoxedMessageStream { self.inner.execute(self.input).boxed() } @@ -142,8 +131,8 @@ impl Inner { ret.push(derived_watermark); } else { warn!( - "{} derive a NULL watermark with the expression {}!", - self.info.identity, out_col_idx + "a NULL watermark is derived with the expression {}!", + out_col_idx ); } } @@ -151,7 +140,7 @@ impl Inner { } #[try_stream(ok = Message, error = StreamExecutorError)] - async fn execute(mut self, input: BoxedExecutor) { + async fn execute(mut self, input: Executor) { #[for_await] for msg in input.execute() { let msg = msg?; @@ -241,28 +230,20 @@ mod tests { ], }; let pk_indices = vec![0]; - let (mut tx, source) = MockSource::channel(schema, pk_indices); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, pk_indices); let test_expr = build_from_pretty("(add:int8 $0:int8 $1:int8)"); - let info = ExecutorInfo { - schema: Schema { - fields: vec![Field::unnamed(DataType::Int64)], - }, - pk_indices: vec![], - identity: "ProjectExecutor".to_string(), - }; - - let project = Box::new(ProjectExecutor::new( + let project = ProjectExecutor::new( ActorContext::for_test(123), - info, - Box::new(source), + source, vec![test_expr], MultiMap::new(), vec![], 0.0, - )); - let mut project = project.execute(); + ); + let mut project = project.boxed().execute(); tx.push_barrier(1, false); let barrier = project.next().await.unwrap().unwrap(); @@ -329,34 +310,22 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - let (mut tx, source) = MockSource::channel(schema, PkIndices::new()); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, PkIndices::new()); let a_expr = build_from_pretty("(add:int8 $0:int8 1:int8)"); let b_expr = build_from_pretty("(subtract:int8 $0:int8 1:int8)"); let c_expr = NonStrictExpression::for_test(DummyNondecreasingExpr); - let info = ExecutorInfo { - schema: Schema { - fields: vec![ - Field::unnamed(DataType::Int64), - Field::unnamed(DataType::Int64), - Field::unnamed(DataType::Int64), - ], - }, - pk_indices: vec![], - identity: "ProjectExecutor".to_string(), - }; - - let project = Box::new(ProjectExecutor::new( + let project = ProjectExecutor::new( ActorContext::for_test(123), - info, - Box::new(source), + source, vec![a_expr, b_expr, c_expr], MultiMap::from_iter(vec![(0, 0), (0, 1)].into_iter()), vec![2], 0.0, - )); - let mut project = project.execute(); + ); + let mut project = project.boxed().execute(); tx.push_barrier(1, false); tx.push_int64_watermark(0, 100); diff --git a/src/stream/src/executor/project_set.rs b/src/stream/src/executor/project_set.rs index 5167a71db2cb8..9fadb5949dac2 100644 --- a/src/stream/src/executor/project_set.rs +++ b/src/stream/src/executor/project_set.rs @@ -20,7 +20,6 @@ use futures_async_stream::try_stream; use multimap::MultiMap; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::bail; -use risingwave_common::catalog::Schema; use risingwave_common::row::{Row, RowExt}; use risingwave_common::types::{DataType, Datum, DatumRef, ToOwnedDatum}; use risingwave_common::util::iter_util::ZipEqFast; @@ -28,10 +27,7 @@ use risingwave_expr::expr::{LogReport, NonStrictExpression}; use risingwave_expr::table_function::ProjectSetSelectItem; use super::error::StreamExecutorError; -use super::{ - ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, Message, PkIndicesRef, - StreamExecutorResult, Watermark, -}; +use super::{ActorContextRef, Execute, Executor, Message, StreamExecutorResult, Watermark}; use crate::common::StreamChunkBuilder; const PROJ_ROW_ID_OFFSET: usize = 1; @@ -40,13 +36,12 @@ const PROJ_ROW_ID_OFFSET: usize = 1; /// and returns a new data chunk. And then, `ProjectSetExecutor` will insert, delete /// or update element into next operator according to the result of the expression. pub struct ProjectSetExecutor { - input: BoxedExecutor, + input: Executor, inner: Inner, } struct Inner { _ctx: ActorContextRef, - info: ExecutorInfo, /// Expressions of the current project_section. select_list: Vec<ProjectSetSelectItem>, @@ -62,8 +57,7 @@ impl ProjectSetExecutor { #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, - input: Box<dyn Executor>, + input: Executor, select_list: Vec<ProjectSetSelectItem>, chunk_size: usize, watermark_derivations: MultiMap<usize, usize>, @@ -71,7 +65,6 @@ impl ProjectSetExecutor { ) -> Self { let inner = Inner { _ctx: ctx, - info, select_list, chunk_size, watermark_derivations, @@ -90,27 +83,15 @@ impl Debug for ProjectSetExecutor { } } -impl Executor for ProjectSetExecutor { +impl Execute for ProjectSetExecutor { fn execute(self: Box<Self>) -> super::BoxedMessageStream { self.inner.execute(self.input).boxed() } - - fn schema(&self) -> &Schema { - &self.inner.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.inner.info.pk_indices - } - - fn identity(&self) -> &str { - &self.inner.info.identity - } } impl Inner { #[try_stream(ok = Message, error = StreamExecutorError)] - async fn execute(self, input: BoxedExecutor) { + async fn execute(self, input: Executor) { assert!(!self.select_list.is_empty()); // First column will be `projected_row_id`, which represents the index in the // output table @@ -270,8 +251,8 @@ impl Inner { ret.push(derived_watermark); } else { warn!( - "{} derive a NULL watermark with the expression {}!", - self.info.identity, expr_idx + "a NULL watermark is derived with the expression {}!", + expr_idx ); } } diff --git a/src/stream/src/executor/rearranged_chain.rs b/src/stream/src/executor/rearranged_chain.rs index 1bb83009a203e..0b51bab665d41 100644 --- a/src/stream/src/executor/rearranged_chain.rs +++ b/src/stream/src/executor/rearranged_chain.rs @@ -19,12 +19,9 @@ use futures::stream::select_with_strategy; use futures::{stream, StreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; -use risingwave_common::catalog::Schema; use super::error::StreamExecutorError; -use super::{ - expect_first_barrier, Barrier, BoxedExecutor, Executor, ExecutorInfo, Message, MessageStream, -}; +use super::{expect_first_barrier, Barrier, Execute, Executor, Message, MessageStream}; use crate::task::{ActorId, CreateMviewProgress}; /// `ChainExecutor` is an executor that enables synchronization between the existing stream and @@ -35,11 +32,9 @@ use crate::task::{ActorId, CreateMviewProgress}; /// [`RearrangedChainExecutor`] resolves the latency problem when creating MV with a huge amount of /// existing data, by rearranging the barrier from the upstream. Check the design doc for details. pub struct RearrangedChainExecutor { - info: ExecutorInfo, + snapshot: Executor, - snapshot: BoxedExecutor, - - upstream: BoxedExecutor, + upstream: Executor, progress: CreateMviewProgress, @@ -84,14 +79,8 @@ impl RearrangedMessage { } impl RearrangedChainExecutor { - pub fn new( - info: ExecutorInfo, - snapshot: BoxedExecutor, - upstream: BoxedExecutor, - progress: CreateMviewProgress, - ) -> Self { + pub fn new(snapshot: Executor, upstream: Executor, progress: CreateMviewProgress) -> Self { Self { - info, snapshot, upstream, actor_id: progress.actor_id(), @@ -288,26 +277,10 @@ impl RearrangedChainExecutor { } } -impl Executor for RearrangedChainExecutor { +impl Execute for RearrangedChainExecutor { fn execute(self: Box<Self>) -> super::BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - - fn info(&self) -> ExecutorInfo { - self.info.clone() - } } // TODO: add new unit tests for rearranged chain diff --git a/src/stream/src/executor/receiver.rs b/src/stream/src/executor/receiver.rs index 273a3fcf1b339..62f2dd694f910 100644 --- a/src/stream/src/executor/receiver.rs +++ b/src/stream/src/executor/receiver.rs @@ -17,7 +17,6 @@ use anyhow::Context; use futures::StreamExt; use futures_async_stream::try_stream; use itertools::Itertools; -use risingwave_common::catalog::Schema; use tokio::time::Instant; use super::exchange::input::BoxedInput; @@ -25,17 +24,12 @@ use super::ActorContextRef; use crate::executor::exchange::input::new_input; use crate::executor::monitor::StreamingMetrics; use crate::executor::utils::ActorInputMetrics; -use crate::executor::{ - expect_first_barrier, BoxedMessageStream, Executor, ExecutorInfo, Message, PkIndicesRef, -}; +use crate::executor::{expect_first_barrier, BoxedMessageStream, Execute, Message}; use crate::task::{FragmentId, SharedContext}; /// `ReceiverExecutor` is used along with a channel. After creating a mpsc channel, /// there should be a `ReceiverExecutor` running in the background, so as to push /// messages down to the executors. pub struct ReceiverExecutor { - /// Logical Operator Info - info: ExecutorInfo, - /// Input from upstream. input: BoxedInput, @@ -57,10 +51,7 @@ pub struct ReceiverExecutor { impl std::fmt::Debug for ReceiverExecutor { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("ReceiverExecutor") - .field("schema", &self.info.schema) - .field("pk_indices", &self.info.pk_indices) - .finish() + f.debug_struct("ReceiverExecutor").finish() } } @@ -68,7 +59,6 @@ impl ReceiverExecutor { #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, fragment_id: FragmentId, upstream_fragment_id: FragmentId, input: BoxedInput, @@ -78,7 +68,6 @@ impl ReceiverExecutor { ) -> Self { Self { input, - info, actor_context: ctx, upstream_fragment_id, metrics, @@ -95,11 +84,6 @@ impl ReceiverExecutor { Self::new( ActorContext::for_test(114), - ExecutorInfo { - schema: Schema::default(), - pk_indices: vec![], - identity: "ReceiverExecutor".to_string(), - }, 514, 1919, LocalInput::new(input, 0).boxed_input(), @@ -110,7 +94,7 @@ impl ReceiverExecutor { } } -impl Executor for ReceiverExecutor { +impl Execute for ReceiverExecutor { fn execute(mut self: Box<Self>) -> BoxedMessageStream { let actor_id = self.actor_context.id; @@ -207,22 +191,6 @@ impl Executor for ReceiverExecutor { stream.boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - - fn info(&self) -> ExecutorInfo { - self.info.clone() - } } #[cfg(test)] @@ -235,13 +203,11 @@ mod tests { use risingwave_pb::stream_plan::update_mutation::MergeUpdate; use super::*; - use crate::executor::{ActorContext, Barrier, Executor, Mutation, UpdateMutation}; + use crate::executor::{ActorContext, Barrier, Execute, Mutation, UpdateMutation}; use crate::task::test_utils::helper_make_local_actor; #[tokio::test] async fn test_configuration_change() { - let schema = Schema { fields: vec![] }; - let actor_id = 233; let (old, new) = (114, 514); // old and new upstream actor id @@ -271,15 +237,8 @@ mod tests { ) .unwrap(); - let info = ExecutorInfo { - schema, - pk_indices: vec![], - identity: "ReceiverExecutor".to_string(), - }; - let receiver = ReceiverExecutor::new( ActorContext::for_test(actor_id), - info, fragment_id, upstream_fragment_id, input, diff --git a/src/stream/src/executor/row_id_gen.rs b/src/stream/src/executor/row_id_gen.rs index fe0ed6d908925..0a0fe1ffe84e1 100644 --- a/src/stream/src/executor/row_id_gen.rs +++ b/src/stream/src/executor/row_id_gen.rs @@ -19,23 +19,19 @@ use risingwave_common::array::{ Array, ArrayBuilder, ArrayRef, Op, SerialArrayBuilder, StreamChunk, }; use risingwave_common::buffer::Bitmap; -use risingwave_common::catalog::Schema; use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::types::Serial; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::row_id::RowIdGenerator; -use super::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, PkIndicesRef, -}; +use super::{expect_first_barrier, ActorContextRef, Execute, Executor}; use crate::executor::{Message, StreamExecutorError}; /// [`RowIdGenExecutor`] generates row id for data, where the user has not specified a pk. pub struct RowIdGenExecutor { ctx: ActorContextRef, - info: ExecutorInfo, - upstream: Option<BoxedExecutor>, + upstream: Option<Executor>, row_id_index: usize, @@ -45,14 +41,12 @@ pub struct RowIdGenExecutor { impl RowIdGenExecutor { pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, - upstream: BoxedExecutor, + upstream: Executor, row_id_index: usize, vnodes: Bitmap, ) -> Self { Self { ctx, - info, upstream: Some(upstream), row_id_index, row_id_generator: Self::new_generator(&vnodes), @@ -126,22 +120,10 @@ impl RowIdGenExecutor { } } -impl Executor for RowIdGenExecutor { +impl Execute for RowIdGenExecutor { fn execute(self: Box<Self>) -> super::BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } #[cfg(test)] @@ -154,7 +136,7 @@ mod tests { use super::*; use crate::executor::test_utils::MockSource; - use crate::executor::{ActorContext, Executor}; + use crate::executor::{ActorContext, Execute}; #[tokio::test] async fn test_row_id_gen_executor() { @@ -165,20 +147,16 @@ mod tests { let pk_indices = vec![0]; let row_id_index = 0; let row_id_generator = Bitmap::ones(VirtualNode::COUNT); - let (mut tx, upstream) = MockSource::channel(schema.clone(), pk_indices.clone()); - let row_id_gen_executor = Box::new(RowIdGenExecutor::new( + let (mut tx, upstream) = MockSource::channel(); + let upstream = upstream.into_executor(schema.clone(), pk_indices.clone()); + + let row_id_gen_executor = RowIdGenExecutor::new( ActorContext::for_test(233), - ExecutorInfo { - schema, - pk_indices, - identity: "RowIdGenExecutor".to_string(), - }, - Box::new(upstream), + upstream, row_id_index, row_id_generator, - )); - - let mut row_id_gen_executor = row_id_gen_executor.execute(); + ); + let mut row_id_gen_executor = row_id_gen_executor.boxed().execute(); // Init barrier tx.push_barrier(1, false); diff --git a/src/stream/src/executor/simple_agg.rs b/src/stream/src/executor/simple_agg.rs index b8915a070dbc5..a863fc0432876 100644 --- a/src/stream/src/executor/simple_agg.rs +++ b/src/stream/src/executor/simple_agg.rs @@ -47,7 +47,7 @@ use crate::task::AtomicU64Ref; /// Therefore, we "automatically" implemented a window function inside /// `SimpleAggExecutor`. pub struct SimpleAggExecutor<S: StateStore> { - input: Box<dyn Executor>, + input: Executor, inner: ExecutorInner<S>, } @@ -111,27 +111,15 @@ struct ExecutionVars<S: StateStore> { state_changed: bool, } -impl<S: StateStore> Executor for SimpleAggExecutor<S> { +impl<S: StateStore> Execute for SimpleAggExecutor<S> { fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.inner.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.inner.info.pk_indices - } - - fn identity(&self) -> &str { - &self.inner.info.identity - } } impl<S: StateStore> SimpleAggExecutor<S> { pub fn new(args: AggExecutorArgs<S, SimpleAggExecutorExtraArgs>) -> StreamResult<Self> { - let input_info = args.input.info(); + let input_info = args.input.info().clone(); Ok(Self { input: args.input, inner: ExecutorInner { @@ -335,7 +323,8 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - let (mut tx, source) = MockSource::channel(schema, vec![2]); // pk + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, vec![2]); tx.push_barrier(1, false); tx.push_barrier(2, false); tx.push_chunk(StreamChunk::from_pretty( @@ -364,7 +353,7 @@ mod tests { let simple_agg = new_boxed_simple_agg_executor( ActorContext::for_test(123), store, - Box::new(source), + source, false, agg_calls, 0, diff --git a/src/stream/src/executor/sink.rs b/src/stream/src/executor/sink.rs index 4e64c3456bc8f..7928308c3f94a 100644 --- a/src/stream/src/executor/sink.rs +++ b/src/stream/src/executor/sink.rs @@ -34,7 +34,7 @@ use risingwave_connector::sink::{ use thiserror_ext::AsReport; use super::error::{StreamExecutorError, StreamExecutorResult}; -use super::{BoxedExecutor, Executor, ExecutorInfo, Message, PkIndices}; +use super::{Execute, Executor, ExecutorInfo, Message, PkIndices}; use crate::executor::{ expect_first_barrier, ActorContextRef, BoxedMessageStream, MessageStream, Mutation, }; @@ -43,7 +43,7 @@ use crate::task::ActorId; pub struct SinkExecutor<F: LogStoreFactory> { actor_context: ActorContextRef, info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, sink: SinkImpl, input_columns: Vec<ColumnCatalog>, sink_param: SinkParam, @@ -83,7 +83,7 @@ impl<F: LogStoreFactory> SinkExecutor<F> { pub async fn new( actor_context: ActorContextRef, info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, sink_writer_param: SinkWriterParam, sink_param: SinkParam, columns: Vec<ColumnCatalog>, @@ -412,22 +412,10 @@ impl<F: LogStoreFactory> SinkExecutor<F> { } } -impl<F: LogStoreFactory> Executor for SinkExecutor<F> { +impl<F: LogStoreFactory> Execute for SinkExecutor<F> { fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } #[cfg(test)] @@ -475,28 +463,25 @@ mod test { .collect(); let pk_indices = vec![0]; - let mock = MockSource::with_messages( - schema.clone(), - pk_indices.clone(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( - " I I I + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( + " I I I + 3 2 1", - ))), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( - " I I I + ))), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( + " I I I U- 3 2 1 U+ 3 4 1 + 5 6 7", - ))), - Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( - " I I I + ))), + Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( + " I I I - 5 6 7", - ))), - ], - ); + ))), + ]) + .into_executor(schema.clone(), pk_indices.clone()); let sink_param = SinkParam { sink_id: 0.into(), @@ -522,7 +507,7 @@ mod test { let sink_executor = SinkExecutor::new( ActorContext::for_test(0), info, - Box::new(mock), + source, SinkWriterParam::for_test(), sink_param, columns.clone(), @@ -531,7 +516,7 @@ mod test { .await .unwrap(); - let mut executor = SinkExecutor::execute(Box::new(sink_executor)); + let mut executor = sink_executor.boxed().execute(); // Barrier message. executor.next().await.unwrap().unwrap(); @@ -598,32 +583,29 @@ mod test { .map(|column| Field::from(column.column_desc.clone())) .collect(); - let mock = MockSource::with_messages( - schema.clone(), - vec![0, 1], - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( - " I I I + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( + " I I I + 1 1 10", - ))), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( - " I I I + ))), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( + " I I I + 1 3 30", - ))), - Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( - " I I I + ))), + Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( + " I I I + 1 2 20 - 1 2 20", - ))), - Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( - " I I I + ))), + Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( + " I I I - 1 1 10", - ))), - Message::Barrier(Barrier::new_test_barrier(3)), - ], - ); + ))), + Message::Barrier(Barrier::new_test_barrier(3)), + ]) + .into_executor(schema.clone(), vec![0, 1]); let sink_param = SinkParam { sink_id: 0.into(), @@ -649,7 +631,7 @@ mod test { let sink_executor = SinkExecutor::new( ActorContext::for_test(0), info, - Box::new(mock), + source, SinkWriterParam::for_test(), sink_param, columns.clone(), @@ -658,7 +640,7 @@ mod test { .await .unwrap(); - let mut executor = SinkExecutor::execute(Box::new(sink_executor)); + let mut executor = sink_executor.boxed().execute(); // Barrier message. executor.next().await.unwrap().unwrap(); @@ -739,15 +721,12 @@ mod test { .collect(); let pk_indices = vec![0]; - let mock = MockSource::with_messages( - schema.clone(), - pk_indices.clone(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Barrier(Barrier::new_test_barrier(3)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Barrier(Barrier::new_test_barrier(3)), + ]) + .into_executor(schema.clone(), pk_indices.clone()); let sink_param = SinkParam { sink_id: 0.into(), @@ -773,7 +752,7 @@ mod test { let sink_executor = SinkExecutor::new( ActorContext::for_test(0), info, - Box::new(mock), + source, SinkWriterParam::for_test(), sink_param, columns, @@ -782,7 +761,7 @@ mod test { .await .unwrap(); - let mut executor = SinkExecutor::execute(Box::new(sink_executor)); + let mut executor = sink_executor.boxed().execute(); // Barrier message. assert_eq!( diff --git a/src/stream/src/executor/sort.rs b/src/stream/src/executor/sort.rs index f07967ce45299..72474eea32977 100644 --- a/src/stream/src/executor/sort.rs +++ b/src/stream/src/executor/sort.rs @@ -20,23 +20,23 @@ use risingwave_storage::StateStore; use super::sort_buffer::SortBuffer; use super::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, PkIndicesRef, StreamExecutorError, Watermark, + expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, + StreamExecutorError, Watermark, }; use crate::common::table::state_table::StateTable; use crate::common::StreamChunkBuilder; pub struct SortExecutor<S: StateStore> { - input: BoxedExecutor, + input: Executor, inner: ExecutorInner<S>, } pub struct SortExecutorArgs<S: StateStore> { pub actor_ctx: ActorContextRef, - pub info: ExecutorInfo, - pub input: BoxedExecutor, + pub input: Executor, + pub schema: Schema, pub buffer_table: StateTable<S>, pub chunk_size: usize, pub sort_column_index: usize, @@ -44,8 +44,8 @@ pub struct SortExecutorArgs<S: StateStore> { struct ExecutorInner<S: StateStore> { actor_ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, buffer_table: StateTable<S>, chunk_size: usize, sort_column_index: usize, @@ -55,22 +55,10 @@ struct ExecutionVars<S: StateStore> { buffer: SortBuffer<S>, } -impl<S: StateStore> Executor for SortExecutor<S> { +impl<S: StateStore> Execute for SortExecutor<S> { fn execute(self: Box<Self>) -> BoxedMessageStream { self.executor_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.inner.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.inner.info.pk_indices - } - - fn identity(&self) -> &str { - &self.inner.info.identity - } } impl<S: StateStore> SortExecutor<S> { @@ -79,7 +67,7 @@ impl<S: StateStore> SortExecutor<S> { input: args.input, inner: ExecutorInner { actor_ctx: args.actor_ctx, - info: args.info, + schema: args.schema, buffer_table: args.buffer_table, chunk_size: args.chunk_size, sort_column_index: args.sort_column_index, @@ -114,7 +102,7 @@ impl<S: StateStore> SortExecutor<S> { if col_idx == this.sort_column_index => { let mut chunk_builder = - StreamChunkBuilder::new(this.chunk_size, this.info.schema.data_types()); + StreamChunkBuilder::new(this.chunk_size, this.schema.data_types()); #[for_await] for row in vars @@ -172,7 +160,7 @@ mod tests { use super::*; use crate::executor::test_utils::{MessageSender, MockSource, StreamExecutorTestExt}; - use crate::executor::{ActorContext, BoxedMessageStream, Executor}; + use crate::executor::{ActorContext, BoxedMessageStream, Execute}; async fn create_executor<S: StateStore>( sort_column_index: usize, @@ -202,15 +190,12 @@ mod tests { ) .await; - let (tx, source) = MockSource::channel(input_schema, input_pk_indices); + let (tx, source) = MockSource::channel(); + let source = source.into_executor(input_schema, input_pk_indices); let sort_executor = SortExecutor::new(SortExecutorArgs { actor_ctx: ActorContext::for_test(123), - info: ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "SortExecutor".to_string(), - }, - input: source.boxed(), + schema: source.schema().clone(), + input: source, buffer_table, chunk_size: 1024, sort_column_index, diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 36513fc4aba36..73d1b5b42f1c6 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -22,7 +22,7 @@ use futures::stream::{self, StreamExt}; use futures::{pin_mut, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; -use risingwave_common::catalog::{ColumnId, Schema, TableId}; +use risingwave_common::catalog::{ColumnId, TableId}; use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{ScalarRef, ScalarRefImpl}; @@ -39,8 +39,13 @@ use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; use thiserror_ext::AsReport; +use super::{get_split_offset_col_idx, SourceStateTableHandler}; use crate::executor::stream_reader::StreamReaderWithPause; -use crate::executor::*; +use crate::executor::{ + expect_first_barrier, get_split_offset_mapping_from_chunk, prune_additional_cols, + ActorContextRef, BoxedMessageStream, Execute, Executor, Message, Mutation, StreamExecutorError, + StreamExecutorResult, StreamSourceCore, +}; const SPLIT_BATCH_SIZE: usize = 1000; @@ -48,13 +53,12 @@ type SplitBatch = Option<Vec<SplitImpl>>; pub struct FsFetchExecutor<S: StateStore, Src: OpendalSource> { actor_ctx: ActorContextRef, - info: ExecutorInfo, /// Streaming source for external stream_source_core: Option<StreamSourceCore<S>>, /// Upstream list executor. - upstream: Option<BoxedExecutor>, + upstream: Option<Executor>, // control options for connector level source_ctrl_opts: SourceCtrlOpts, @@ -69,15 +73,13 @@ impl<S: StateStore, Src: OpendalSource> FsFetchExecutor<S, Src> { #[allow(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, - info: ExecutorInfo, stream_source_core: StreamSourceCore<S>, - upstream: BoxedExecutor, + upstream: Executor, source_ctrl_opts: SourceCtrlOpts, connector_params: ConnectorParams, ) -> Self { Self { actor_ctx, - info, stream_source_core: Some(stream_source_core), upstream: Some(upstream), source_ctrl_opts, @@ -354,22 +356,10 @@ impl<S: StateStore, Src: OpendalSource> FsFetchExecutor<S, Src> { } } -impl<S: StateStore, Src: OpendalSource> Executor for FsFetchExecutor<S, Src> { +impl<S: StateStore, Src: OpendalSource> Execute for FsFetchExecutor<S, Src> { fn execute(self: Box<Self>) -> BoxedMessageStream { self.into_stream().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } impl<S: StateStore, Src: OpendalSource> Debug for FsFetchExecutor<S, Src> { @@ -378,7 +368,6 @@ impl<S: StateStore, Src: OpendalSource> Debug for FsFetchExecutor<S, Src> { f.debug_struct("FsFetchExecutor") .field("source_id", &core.source_id) .field("column_ids", &core.column_ids) - .field("pk_indices", &self.info.pk_indices) .finish() } else { f.debug_struct("FsFetchExecutor").finish() diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index 95894429e9361..3f2d13b8be76f 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -22,7 +22,6 @@ use anyhow::anyhow; use either::Either; use futures::{StreamExt, TryStreamExt}; use futures_async_stream::try_stream; -use risingwave_common::catalog::Schema; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_connector::error::ConnectorError; @@ -50,7 +49,6 @@ const WAIT_BARRIER_MULTIPLE_TIMES: u128 = 5; /// such as s3. pub struct FsSourceExecutor<S: StateStore> { actor_ctx: ActorContextRef, - info: ExecutorInfo, /// Streaming source for external stream_source_core: StreamSourceCore<S>, @@ -71,7 +69,6 @@ impl<S: StateStore> FsSourceExecutor<S> { #[allow(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, - info: ExecutorInfo, stream_source_core: StreamSourceCore<S>, metrics: Arc<StreamingMetrics>, barrier_receiver: UnboundedReceiver<Barrier>, @@ -80,7 +77,6 @@ impl<S: StateStore> FsSourceExecutor<S> { ) -> StreamResult<Self> { Ok(Self { actor_ctx, - info, stream_source_core, metrics, barrier_receiver: Some(barrier_receiver), @@ -481,22 +477,10 @@ impl<S: StateStore> FsSourceExecutor<S> { } } -impl<S: StateStore> Executor for FsSourceExecutor<S> { +impl<S: StateStore> Execute for FsSourceExecutor<S> { fn execute(self: Box<Self>) -> BoxedMessageStream { self.into_stream().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } impl<S: StateStore> Debug for FsSourceExecutor<S> { @@ -504,7 +488,6 @@ impl<S: StateStore> Debug for FsSourceExecutor<S> { f.debug_struct("FsSourceExecutor") .field("source_id", &self.stream_source_core.source_id) .field("column_ids", &self.stream_source_core.column_ids) - .field("pk_indices", &self.info.pk_indices) .finish() } } diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/list_executor.rs index f2a6e47ae8641..7996848a749e3 100644 --- a/src/stream/src/executor/source/list_executor.rs +++ b/src/stream/src/executor/source/list_executor.rs @@ -20,7 +20,6 @@ use either::Either; use futures::{StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::Op; -use risingwave_common::catalog::Schema; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_connector::source::reader::desc::{SourceDesc, SourceDescBuilder}; use risingwave_connector::source::SourceCtrlOpts; @@ -39,7 +38,6 @@ const CHUNK_SIZE: usize = 1024; #[allow(dead_code)] pub struct FsListExecutor<S: StateStore> { actor_ctx: ActorContextRef, - info: ExecutorInfo, /// Streaming source for external stream_source_core: Option<StreamSourceCore<S>>, @@ -64,7 +62,6 @@ impl<S: StateStore> FsListExecutor<S> { #[allow(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, - info: ExecutorInfo, stream_source_core: Option<StreamSourceCore<S>>, metrics: Arc<StreamingMetrics>, barrier_receiver: UnboundedReceiver<Barrier>, @@ -74,7 +71,6 @@ impl<S: StateStore> FsListExecutor<S> { ) -> Self { Self { actor_ctx, - info, stream_source_core, metrics, barrier_receiver: Some(barrier_receiver), @@ -193,22 +189,10 @@ impl<S: StateStore> FsListExecutor<S> { } } -impl<S: StateStore> Executor for FsListExecutor<S> { +impl<S: StateStore> Execute for FsListExecutor<S> { fn execute(self: Box<Self>) -> BoxedMessageStream { self.into_stream().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } impl<S: StateStore> Debug for FsListExecutor<S> { @@ -217,7 +201,6 @@ impl<S: StateStore> Debug for FsListExecutor<S> { f.debug_struct("FsListExecutor") .field("source_id", &core.source_id) .field("column_ids", &core.column_ids) - .field("pk_indices", &self.info.pk_indices) .finish() } else { f.debug_struct("FsListExecutor").finish() diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index e2567bb141492..a695b3c8b6b4b 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -43,7 +43,6 @@ const WAIT_BARRIER_MULTIPLE_TIMES: u128 = 5; pub struct SourceExecutor<S: StateStore> { actor_ctx: ActorContextRef, - info: ExecutorInfo, /// Streaming source for external stream_source_core: Option<StreamSourceCore<S>>, @@ -68,7 +67,6 @@ impl<S: StateStore> SourceExecutor<S> { #[allow(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, - info: ExecutorInfo, stream_source_core: Option<StreamSourceCore<S>>, metrics: Arc<StreamingMetrics>, barrier_receiver: UnboundedReceiver<Barrier>, @@ -78,7 +76,6 @@ impl<S: StateStore> SourceExecutor<S> { ) -> Self { Self { actor_ctx, - info, stream_source_core, metrics, barrier_receiver: Some(barrier_receiver), @@ -527,8 +524,7 @@ impl<S: StateStore> SourceExecutor<S> { // chunks. self_paused = true; tracing::warn!( - "source {} paused, wait barrier for {:?}", - self.info.identity, + "source paused, wait barrier for {:?}", last_barrier_time.elapsed() ); stream.pause_stream(); @@ -616,7 +612,7 @@ impl<S: StateStore> SourceExecutor<S> { } } -impl<S: StateStore> Executor for SourceExecutor<S> { +impl<S: StateStore> Execute for SourceExecutor<S> { fn execute(self: Box<Self>) -> BoxedMessageStream { if self.stream_source_core.is_some() { self.execute_with_stream_source().boxed() @@ -624,18 +620,6 @@ impl<S: StateStore> Executor for SourceExecutor<S> { self.execute_without_stream_source().boxed() } } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } impl<S: StateStore> Debug for SourceExecutor<S> { @@ -644,7 +628,6 @@ impl<S: StateStore> Debug for SourceExecutor<S> { f.debug_struct("SourceExecutor") .field("source_id", &core.source_id) .field("column_ids", &core.column_ids) - .field("pk_indices", &self.info.pk_indices) .finish() } else { f.debug_struct("SourceExecutor").finish() @@ -683,7 +666,6 @@ mod tests { fields: vec![Field::with_name(DataType::Int32, "sequence_int")], }; let row_id_index = None; - let pk_indices = vec![0]; let source_info = StreamSourceInfo { row_format: PbRowFormatType::Native as i32, ..Default::default() @@ -720,11 +702,6 @@ mod tests { let executor = SourceExecutor::new( ActorContext::for_test(0), - ExecutorInfo { - schema, - pk_indices, - identity: "SourceExecutor".to_string(), - }, Some(core), Arc::new(StreamingMetrics::unused()), barrier_rx, @@ -732,7 +709,7 @@ mod tests { SourceCtrlOpts::default(), ConnectorParams::default(), ); - let mut executor = Box::new(executor).execute(); + let mut executor = executor.boxed().execute(); let init_barrier = Barrier::new_test_barrier(1).with_mutation(Mutation::Add(AddMutation { adds: HashMap::new(), @@ -776,7 +753,6 @@ mod tests { fields: vec![Field::with_name(DataType::Int32, "v1")], }; let row_id_index = None; - let pk_indices = vec![0_usize]; let source_info = StreamSourceInfo { row_format: PbRowFormatType::Native as i32, ..Default::default() @@ -814,11 +790,6 @@ mod tests { let executor = SourceExecutor::new( ActorContext::for_test(0), - ExecutorInfo { - schema, - pk_indices, - identity: "SourceExecutor".to_string(), - }, Some(core), Arc::new(StreamingMetrics::unused()), barrier_rx, @@ -826,7 +797,7 @@ mod tests { SourceCtrlOpts::default(), ConnectorParams::default(), ); - let mut handler = Box::new(executor).execute(); + let mut handler = executor.boxed().execute(); let init_barrier = Barrier::new_test_barrier(1).with_mutation(Mutation::Add(AddMutation { adds: HashMap::new(), diff --git a/src/stream/src/executor/stateless_simple_agg.rs b/src/stream/src/executor/stateless_simple_agg.rs index 8a6334b7743b5..1ad25fe5c7f6c 100644 --- a/src/stream/src/executor/stateless_simple_agg.rs +++ b/src/stream/src/executor/stateless_simple_agg.rs @@ -16,7 +16,6 @@ use futures::StreamExt; use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::catalog::Schema; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::aggregate::{ build_retractable, AggCall, AggregateState, BoxedAggregateFunction, @@ -29,28 +28,16 @@ use crate::error::StreamResult; pub struct StatelessSimpleAggExecutor { _ctx: ActorContextRef, - pub(super) info: ExecutorInfo, - pub(super) input: Box<dyn Executor>, + pub(super) input: Executor, + pub(super) schema: Schema, pub(super) aggs: Vec<BoxedAggregateFunction>, pub(super) agg_calls: Vec<AggCall>, } -impl Executor for StatelessSimpleAggExecutor { +impl Execute for StatelessSimpleAggExecutor { fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } impl StatelessSimpleAggExecutor { @@ -73,7 +60,7 @@ impl StatelessSimpleAggExecutor { let StatelessSimpleAggExecutor { _ctx, input, - info, + schema, aggs, agg_calls, } = self; @@ -94,7 +81,7 @@ impl StatelessSimpleAggExecutor { if is_dirty { is_dirty = false; - let mut builders = info.schema.create_array_builders(1); + let mut builders = schema.create_array_builders(1); for ((agg, state), builder) in aggs .iter() .zip_eq_fast(states.iter_mut()) @@ -124,15 +111,15 @@ impl StatelessSimpleAggExecutor { impl StatelessSimpleAggExecutor { pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, - input: Box<dyn Executor>, + input: Executor, + schema: Schema, agg_calls: Vec<AggCall>, ) -> StreamResult<Self> { let aggs = agg_calls.iter().map(build_retractable).try_collect()?; Ok(StatelessSimpleAggExecutor { _ctx: ctx, - info, input, + schema, aggs, agg_calls, }) @@ -150,34 +137,24 @@ mod tests { use super::*; use crate::executor::test_utils::agg_executor::generate_agg_schema; use crate::executor::test_utils::MockSource; - use crate::executor::{Executor, StatelessSimpleAggExecutor}; + use crate::executor::{Execute, StatelessSimpleAggExecutor}; #[tokio::test] async fn test_no_chunk() { let schema = schema_test_utils::ii(); - let (mut tx, source) = MockSource::channel(schema, vec![2]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, vec![2]); tx.push_barrier(1, false); tx.push_barrier(2, false); tx.push_barrier(3, false); let agg_calls = vec![AggCall::from_pretty("(count:int8)")]; let schema = generate_agg_schema(&source, &agg_calls, None); - let info = ExecutorInfo { - schema, - pk_indices: vec![], - identity: "StatelessSimpleAggExecutor".to_string(), - }; - let simple_agg = Box::new( - StatelessSimpleAggExecutor::new( - ActorContext::for_test(123), - info, - Box::new(source), - agg_calls, - ) - .unwrap(), - ); - let mut simple_agg = simple_agg.execute(); + let simple_agg = + StatelessSimpleAggExecutor::new(ActorContext::for_test(123), source, schema, agg_calls) + .unwrap(); + let mut simple_agg = simple_agg.boxed().execute(); assert_matches!( simple_agg.next().await.unwrap().unwrap(), @@ -196,7 +173,8 @@ mod tests { #[tokio::test] async fn test_local_simple_agg() { let schema = schema_test_utils::iii(); - let (mut tx, source) = MockSource::channel(schema, vec![2]); // pk\ + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, vec![2]); tx.push_barrier(1, false); tx.push_chunk(StreamChunk::from_pretty( " I I I @@ -220,22 +198,11 @@ mod tests { AggCall::from_pretty("(sum:int8 $1:int8)"), ]; let schema = generate_agg_schema(&source, &agg_calls, None); - let info = ExecutorInfo { - schema, - pk_indices: vec![], - identity: "StatelessSimpleAggExecutor".to_string(), - }; - let simple_agg = Box::new( - StatelessSimpleAggExecutor::new( - ActorContext::for_test(123), - info, - Box::new(source), - agg_calls, - ) - .unwrap(), - ); - let mut simple_agg = simple_agg.execute(); + let simple_agg = + StatelessSimpleAggExecutor::new(ActorContext::for_test(123), source, schema, agg_calls) + .unwrap(); + let mut simple_agg = simple_agg.boxed().execute(); // Consume the init barrier simple_agg.next().await.unwrap().unwrap(); diff --git a/src/stream/src/executor/subscription.rs b/src/stream/src/executor/subscription.rs index f1a2923c9085c..dfe352909fd8e 100644 --- a/src/stream/src/executor/subscription.rs +++ b/src/stream/src/executor/subscription.rs @@ -16,15 +16,14 @@ use core::time::Duration; use futures::prelude::stream::StreamExt; use futures_async_stream::try_stream; -use risingwave_common::catalog::Schema; use risingwave_common::types::Timestamptz; use risingwave_common::util::epoch::Epoch; use risingwave_storage::store::LocalStateStore; use tokio::time::Instant; use super::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult, + expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, + StreamExecutorError, StreamExecutorResult, }; use crate::common::log_store_impl::kv_log_store::ReaderTruncationOffsetType; use crate::common::log_store_impl::subscription_log_store::SubscriptionLogStoreWriter; @@ -33,8 +32,7 @@ const EXECUTE_GC_INTERVAL: u64 = 3600; pub struct SubscriptionExecutor<LS: LocalStateStore> { actor_context: ActorContextRef, - info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, log_store: SubscriptionLogStoreWriter<LS>, retention_seconds: u64, } @@ -44,14 +42,12 @@ impl<LS: LocalStateStore> SubscriptionExecutor<LS> { #[expect(clippy::unused_async)] pub async fn new( actor_context: ActorContextRef, - info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, log_store: SubscriptionLogStoreWriter<LS>, retention_seconds: u64, ) -> StreamExecutorResult<Self> { Ok(Self { actor_context, - info, input, log_store, retention_seconds, @@ -115,24 +111,8 @@ impl<LS: LocalStateStore> SubscriptionExecutor<LS> { } } } -impl<LS: LocalStateStore> Executor for SubscriptionExecutor<LS> { +impl<LS: LocalStateStore> Execute for SubscriptionExecutor<LS> { fn execute(self: Box<Self>) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - - fn info(&self) -> ExecutorInfo { - self.info.clone() - } } diff --git a/src/stream/src/executor/subtask.rs b/src/stream/src/executor/subtask.rs index 0130a73fc411a..fea9644b151f3 100644 --- a/src/stream/src/executor/subtask.rs +++ b/src/stream/src/executor/subtask.rs @@ -20,7 +20,7 @@ use tokio::sync::mpsc::error::SendError; use tokio_stream::wrappers::ReceiverStream; use super::actor::spawn_blocking_drop_stream; -use super::{BoxedExecutor, Executor, ExecutorInfo, Message, MessageStreamItem}; +use super::{Execute, Executor, Message, MessageStreamItem}; use crate::task::ActorId; /// Handle used to drive the subtask. @@ -29,31 +29,13 @@ pub type SubtaskHandle = impl Future<Output = ()> + Send + 'static; /// The thin wrapper for subtask-wrapped executor, containing a channel to receive the messages from /// the subtask. pub struct SubtaskRxExecutor { - info: ExecutorInfo, - rx: mpsc::Receiver<MessageStreamItem>, } -impl Executor for SubtaskRxExecutor { +impl Execute for SubtaskRxExecutor { fn execute(self: Box<Self>) -> super::BoxedMessageStream { ReceiverStream::new(self.rx).boxed() } - - fn schema(&self) -> &risingwave_common::catalog::Schema { - &self.info.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - - fn info(&self) -> ExecutorInfo { - self.info.clone() - } } /// Wrap an executor into a subtask and a thin receiver executor, connected by a channel with a @@ -62,15 +44,9 @@ impl Executor for SubtaskRxExecutor { /// Used when there're multiple stateful executors in an actor. These subtasks can be concurrently /// executed to improve the I/O performance, while the computing resource can be still bounded to a /// single thread. -pub fn wrap(input: BoxedExecutor, actor_id: ActorId) -> (SubtaskHandle, SubtaskRxExecutor) { +pub fn wrap(input: Executor, actor_id: ActorId) -> (SubtaskHandle, SubtaskRxExecutor) { let (tx, rx) = mpsc::channel(1); - let rx_executor = SubtaskRxExecutor { - info: ExecutorInfo { - identity: "SubtaskRxExecutor".to_owned(), - ..input.info() - }, - rx, - }; + let rx_executor = SubtaskRxExecutor { rx }; let handle = async move { let mut input = input.execute(); diff --git a/src/stream/src/executor/temporal_join.rs b/src/stream/src/executor/temporal_join.rs index da0ac7b45dbdc..f6c614f41aef4 100644 --- a/src/stream/src/executor/temporal_join.rs +++ b/src/stream/src/executor/temporal_join.rs @@ -25,7 +25,6 @@ use futures_async_stream::{for_await, try_stream}; use local_stats_alloc::{SharedStatsAlloc, StatsAlloc}; use lru::DefaultHasher; use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::catalog::Schema; use risingwave_common::estimate_size::{EstimateSize, KvSize}; use risingwave_common::hash::{HashKey, NullBitmap}; use risingwave_common::row::{OwnedRow, Row, RowExt}; @@ -40,21 +39,21 @@ use risingwave_storage::StateStore; use super::join::{JoinType, JoinTypePrimitive}; use super::{ - Barrier, Executor, ExecutorInfo, Message, MessageStream, StreamExecutorError, + Barrier, Execute, ExecutorInfo, Message, MessageStream, StreamExecutorError, StreamExecutorResult, }; use crate::cache::{cache_may_stale, new_with_hasher_in, ManagedLruCache}; use crate::common::metrics::MetricsInfo; use crate::executor::join::builder::JoinStreamChunkBuilder; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{ActorContextRef, BoxedExecutor, Watermark}; +use crate::executor::{ActorContextRef, Executor, Watermark}; use crate::task::AtomicU64Ref; pub struct TemporalJoinExecutor<K: HashKey, S: StateStore, const T: JoinTypePrimitive> { ctx: ActorContextRef, info: ExecutorInfo, - left: BoxedExecutor, - right: BoxedExecutor, + left: Executor, + right: Executor, right_table: TemporalSide<K, S>, left_join_keys: Vec<usize>, right_join_keys: Vec<usize>, @@ -261,7 +260,7 @@ async fn internal_messages_until_barrier(stream: impl MessageStream, expected_ba // any number of `InternalMessage::Chunk(left_chunk)` and followed by // `InternalMessage::Barrier(right_chunks, barrier)`. #[try_stream(ok = InternalMessage, error = StreamExecutorError)] -async fn align_input(left: Box<dyn Executor>, right: Box<dyn Executor>) { +async fn align_input(left: Executor, right: Executor) { let mut left = pin!(left.execute()); let mut right = pin!(right.execute()); // Keep producing intervals until stream exhaustion or errors. @@ -313,8 +312,8 @@ impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> TemporalJoinExecutor pub fn new( ctx: ActorContextRef, info: ExecutorInfo, - left: BoxedExecutor, - right: BoxedExecutor, + left: Executor, + right: Executor, table: StorageTable<S>, left_join_keys: Vec<usize>, right_join_keys: Vec<usize>, @@ -475,22 +474,10 @@ impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> TemporalJoinExecutor } } -impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> Executor +impl<K: HashKey, S: StateStore, const T: JoinTypePrimitive> Execute for TemporalJoinExecutor<K, S, T> { fn execute(self: Box<Self>) -> super::BoxedMessageStream { self.into_stream().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } diff --git a/src/stream/src/executor/test_utils.rs b/src/stream/src/executor/test_utils.rs index cc7505164d154..0be0c9b5648b1 100644 --- a/src/stream/src/executor/test_utils.rs +++ b/src/stream/src/executor/test_utils.rs @@ -21,8 +21,8 @@ use tokio::sync::mpsc; use super::error::StreamExecutorError; use super::{ - Barrier, BoxedMessageStream, Executor, Message, MessageStream, PkIndices, StreamChunk, - StreamExecutorResult, Watermark, + Barrier, BoxedMessageStream, Execute, Executor, ExecutorInfo, Message, MessageStream, + StreamChunk, StreamExecutorResult, Watermark, }; pub mod prelude { @@ -40,12 +40,10 @@ pub mod prelude { pub use crate::common::table::state_table::StateTable; pub use crate::executor::test_utils::expr::build_from_pretty; pub use crate::executor::test_utils::{MessageSender, MockSource, StreamExecutorTestExt}; - pub use crate::executor::{ActorContext, BoxedMessageStream, Executor, PkIndices}; + pub use crate::executor::{ActorContext, BoxedMessageStream, Execute, PkIndices}; } pub struct MockSource { - schema: Schema, - pk_indices: PkIndices, rx: mpsc::UnboundedReceiver<Message>, /// Whether to send a `Stop` barrier on stream finish. @@ -107,20 +105,15 @@ impl MessageSender { impl std::fmt::Debug for MockSource { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("MockSource") - .field("schema", &self.schema) - .field("pk_indices", &self.pk_indices) - .finish() + f.debug_struct("MockSource").finish() } } impl MockSource { #[allow(dead_code)] - pub fn channel(schema: Schema, pk_indices: PkIndices) -> (MessageSender, Self) { + pub fn channel() -> (MessageSender, Self) { let (tx, rx) = mpsc::unbounded_channel(); let source = Self { - schema, - pk_indices, rx, stop_on_finish: true, }; @@ -128,16 +121,16 @@ impl MockSource { } #[allow(dead_code)] - pub fn with_messages(schema: Schema, pk_indices: PkIndices, msgs: Vec<Message>) -> Self { - let (tx, source) = Self::channel(schema, pk_indices); + pub fn with_messages(msgs: Vec<Message>) -> Self { + let (tx, source) = Self::channel(); for msg in msgs { tx.0.send(msg).unwrap(); } source } - pub fn with_chunks(schema: Schema, pk_indices: PkIndices, chunks: Vec<StreamChunk>) -> Self { - let (tx, source) = Self::channel(schema, pk_indices); + pub fn with_chunks(chunks: Vec<StreamChunk>) -> Self { + let (tx, source) = Self::channel(); for chunk in chunks { tx.0.send(Message::Chunk(chunk)).unwrap(); } @@ -153,6 +146,17 @@ impl MockSource { } } + pub fn into_executor(self, schema: Schema, pk_indices: Vec<usize>) -> Executor { + Executor::new( + ExecutorInfo { + schema, + pk_indices, + identity: "MockSource".to_string(), + }, + self.boxed(), + ) + } + #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(mut self: Box<Self>) { let mut epoch = 1; @@ -168,22 +172,10 @@ impl MockSource { } } -impl Executor for MockSource { +impl Execute for MockSource { fn execute(self: Box<Self>) -> super::BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.pk_indices - } - - fn identity(&self) -> &str { - "MockSource" - } } /// `row_nonnull` builds a `OwnedRow` with concrete values. @@ -295,14 +287,14 @@ pub mod agg_executor { }; use crate::executor::aggregation::AggStateStorage; use crate::executor::{ - ActorContext, ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, HashAggExecutor, - PkIndices, SimpleAggExecutor, + ActorContext, ActorContextRef, Executor, ExecutorInfo, HashAggExecutor, PkIndices, + SimpleAggExecutor, }; - /// Generate agg executor's schema from `input`, `agg_calls` and `group_key_indices`. + /// Generate aggExecuter's schema from `input`, `agg_calls` and `group_key_indices`. /// For [`crate::executor::HashAggExecutor`], the group key indices should be provided. pub fn generate_agg_schema( - input: &dyn Executor, + input_ref: &Executor, agg_calls: &[AggCall], group_key_indices: Option<&[usize]>, ) -> Schema { @@ -313,7 +305,7 @@ pub mod agg_executor { let fields = if let Some(key_indices) = group_key_indices { let keys = key_indices .iter() - .map(|idx| input.schema().fields[*idx].clone()); + .map(|idx| input_ref.schema().fields[*idx].clone()); keys.chain(aggs).collect() } else { @@ -331,7 +323,7 @@ pub mod agg_executor { agg_call: &AggCall, group_key_indices: &[usize], pk_indices: &[usize], - input_ref: &dyn Executor, + input_ref: &Executor, is_append_only: bool, ) -> AggStateStorage<S> { match agg_call.kind { @@ -402,7 +394,7 @@ pub mod agg_executor { table_id: TableId, agg_calls: &[AggCall], group_key_indices: &[usize], - input_ref: &dyn Executor, + input_ref: &Executor, ) -> StateTable<S> { let input_fields = input_ref.schema().fields(); @@ -441,7 +433,7 @@ pub mod agg_executor { #[allow(clippy::too_many_arguments)] pub async fn new_boxed_hash_agg_executor<S: StateStore>( store: S, - input: Box<dyn Executor>, + input: Executor, is_append_only: bool, agg_calls: Vec<AggCall>, row_count_index: usize, @@ -450,7 +442,7 @@ pub mod agg_executor { extreme_cache_size: usize, emit_on_window_close: bool, executor_id: u64, - ) -> Box<dyn Executor> { + ) -> Executor { let mut storages = Vec::with_capacity(agg_calls.iter().len()); for (idx, agg_call) in agg_calls.iter().enumerate() { storages.push( @@ -460,7 +452,7 @@ pub mod agg_executor { agg_call, &group_key_indices, &pk_indices, - input.as_ref(), + &input, is_append_only, ) .await, @@ -472,23 +464,23 @@ pub mod agg_executor { TableId::new(agg_calls.len() as u32), &agg_calls, &group_key_indices, - input.as_ref(), + &input, ) .await; - let schema = generate_agg_schema(input.as_ref(), &agg_calls, Some(&group_key_indices)); + let schema = generate_agg_schema(&input, &agg_calls, Some(&group_key_indices)); let info = ExecutorInfo { schema, pk_indices, identity: format!("HashAggExecutor {:X}", executor_id), }; - HashAggExecutor::<SerializedKey, S>::new(AggExecutorArgs { + let exec = HashAggExecutor::<SerializedKey, S>::new(AggExecutorArgs { version: PbAggNodeVersion::Max, input, actor_ctx: ActorContext::for_test(123), - info, + info: info.clone(), extreme_cache_size, @@ -506,21 +498,21 @@ pub mod agg_executor { emit_on_window_close, }, }) - .unwrap() - .boxed() + .unwrap(); + (info, exec).into() } #[allow(clippy::too_many_arguments)] pub async fn new_boxed_simple_agg_executor<S: StateStore>( actor_ctx: ActorContextRef, store: S, - input: BoxedExecutor, + input: Executor, is_append_only: bool, agg_calls: Vec<AggCall>, row_count_index: usize, pk_indices: PkIndices, executor_id: u64, - ) -> Box<dyn Executor> { + ) -> Executor { let storages = future::join_all(agg_calls.iter().enumerate().map(|(idx, agg_call)| { create_agg_state_storage( store.clone(), @@ -528,7 +520,7 @@ pub mod agg_executor { agg_call, &[], &pk_indices, - input.as_ref(), + &input, is_append_only, ) })) @@ -539,23 +531,23 @@ pub mod agg_executor { TableId::new(agg_calls.len() as u32), &agg_calls, &[], - input.as_ref(), + &input, ) .await; - let schema = generate_agg_schema(input.as_ref(), &agg_calls, None); + let schema = generate_agg_schema(&input, &agg_calls, None); let info = ExecutorInfo { schema, pk_indices, identity: format!("SimpleAggExecutor {:X}", executor_id), }; - SimpleAggExecutor::new(AggExecutorArgs { + let exec = SimpleAggExecutor::new(AggExecutorArgs { version: PbAggNodeVersion::Max, input, actor_ctx, - info, + info: info.clone(), extreme_cache_size: 1024, @@ -567,8 +559,8 @@ pub mod agg_executor { watermark_epoch: Arc::new(AtomicU64::new(0)), extra: SimpleAggExecutorExtraArgs {}, }) - .unwrap() - .boxed() + .unwrap(); + (info, exec).into() } } diff --git a/src/stream/src/executor/top_n/group_top_n.rs b/src/stream/src/executor/top_n/group_top_n.rs index ed0f64d098a7a..d4c0fa029341e 100644 --- a/src/stream/src/executor/top_n/group_top_n.rs +++ b/src/stream/src/executor/top_n/group_top_n.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::Bitmap; +use risingwave_common::catalog::Schema; use risingwave_common::hash::HashKey; use risingwave_common::row::RowExt; use risingwave_common::util::epoch::EpochPair; @@ -32,7 +33,7 @@ use crate::common::metrics::MetricsInfo; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; use crate::executor::error::StreamExecutorResult; -use crate::executor::{ActorContextRef, Executor, ExecutorInfo, PkIndices, Watermark}; +use crate::executor::{ActorContextRef, Executor, PkIndices, Watermark}; use crate::task::AtomicU64Ref; pub type GroupTopNExecutor<K, S, const WITH_TIES: bool> = @@ -41,9 +42,9 @@ pub type GroupTopNExecutor<K, S, const WITH_TIES: bool> = impl<K: HashKey, S: StateStore, const WITH_TIES: bool> GroupTopNExecutor<K, S, WITH_TIES> { #[allow(clippy::too_many_arguments)] pub fn new( - input: Box<dyn Executor>, + input: Executor, ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, storage_key: Vec<ColumnOrder>, offset_and_limit: (usize, usize), order_by: Vec<ColumnOrder>, @@ -55,7 +56,7 @@ impl<K: HashKey, S: StateStore, const WITH_TIES: bool> GroupTopNExecutor<K, S, W input, ctx: ctx.clone(), inner: InnerGroupTopNExecutor::new( - info, + schema, storage_key, offset_and_limit, order_by, @@ -69,7 +70,7 @@ impl<K: HashKey, S: StateStore, const WITH_TIES: bool> GroupTopNExecutor<K, S, W } pub struct InnerGroupTopNExecutor<K: HashKey, S: StateStore, const WITH_TIES: bool> { - info: ExecutorInfo, + schema: Schema, /// `LIMIT XXX`. None means no limit. limit: usize, @@ -97,7 +98,7 @@ pub struct InnerGroupTopNExecutor<K: HashKey, S: StateStore, const WITH_TIES: bo impl<K: HashKey, S: StateStore, const WITH_TIES: bool> InnerGroupTopNExecutor<K, S, WITH_TIES> { #[allow(clippy::too_many_arguments)] pub fn new( - info: ExecutorInfo, + schema: Schema, storage_key: Vec<ColumnOrder>, offset_and_limit: (usize, usize), order_by: Vec<ColumnOrder>, @@ -113,12 +114,11 @@ impl<K: HashKey, S: StateStore, const WITH_TIES: bool> InnerGroupTopNExecutor<K, "GroupTopN", ); - let cache_key_serde = - create_cache_key_serde(&storage_key, &info.schema, &order_by, &group_by); + let cache_key_serde = create_cache_key_serde(&storage_key, &schema, &order_by, &group_by); let managed_state = ManagedTopNState::<S>::new(state_table, cache_key_serde.clone()); Ok(Self { - info, + schema, offset: offset_and_limit.0, limit: offset_and_limit.1, managed_state, @@ -191,7 +191,7 @@ where .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .inc(); let mut topn_cache = - TopNCache::new(self.offset, self.limit, self.info().schema.data_types()); + TopNCache::new(self.offset, self.limit, self.schema.data_types()); self.managed_state .init_topn_cache(Some(group_key), &mut topn_cache) .await?; @@ -227,7 +227,7 @@ where .group_top_n_cached_entry_count .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .set(self.caches.len() as i64); - generate_output(res_rows, res_ops, &self.info().schema) + generate_output(res_rows, res_ops, &self.schema) } async fn flush_data(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { @@ -238,10 +238,6 @@ where self.managed_state.try_flush().await } - fn info(&self) -> &ExecutorInfo { - &self.info - } - fn update_epoch(&mut self, epoch: u64) { self.caches.update_epoch(epoch); } @@ -289,7 +285,7 @@ mod tests { use super::*; use crate::executor::test_utils::top_n_executor::create_in_memory_state_table; use crate::executor::test_utils::MockSource; - use crate::executor::{ActorContext, Barrier, Message}; + use crate::executor::{ActorContext, Barrier, Execute, Message}; fn create_schema() -> Schema { Schema { @@ -355,24 +351,21 @@ mod tests { vec![chunk0, chunk1, chunk2, chunk3] } - fn create_source() -> Box<MockSource> { + fn create_source() -> Executor { let mut chunks = create_stream_chunks(); let schema = create_schema(); - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(std::mem::take(&mut chunks[2])), - Message::Barrier(Barrier::new_test_barrier(4)), - Message::Chunk(std::mem::take(&mut chunks[3])), - Message::Barrier(Barrier::new_test_barrier(5)), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(std::mem::take(&mut chunks[2])), + Message::Barrier(Barrier::new_test_barrier(4)), + Message::Chunk(std::mem::take(&mut chunks[3])), + Message::Barrier(Barrier::new_test_barrier(5)), + ]) + .into_executor(schema, pk_indices()) } #[tokio::test] @@ -388,26 +381,20 @@ mod tests { &pk_indices(), ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), // this includes group key as prefix - identity: "GroupTopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - GroupTopNExecutor::<SerializedKey, MemoryStateStore, false>::new( - source as Box<dyn Executor>, - ActorContext::for_test(0), - info, - storage_key(), - (0, 2), - order_by_1(), - vec![1], - state_table, - Arc::new(AtomicU64::new(0)), - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = GroupTopNExecutor::<SerializedKey, MemoryStateStore, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (0, 2), + order_by_1(), + vec![1], + state_table, + Arc::new(AtomicU64::new(0)), + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -490,26 +477,20 @@ mod tests { &pk_indices(), ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), // this includes group key as prefix - identity: "GroupTopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - GroupTopNExecutor::<SerializedKey, MemoryStateStore, false>::new( - source as Box<dyn Executor>, - ActorContext::for_test(0), - info, - storage_key(), - (1, 2), - order_by_1(), - vec![1], - state_table, - Arc::new(AtomicU64::new(0)), - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = GroupTopNExecutor::<SerializedKey, MemoryStateStore, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (1, 2), + order_by_1(), + vec![1], + state_table, + Arc::new(AtomicU64::new(0)), + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -585,26 +566,20 @@ mod tests { &pk_indices(), ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), // this includes group key as prefix - identity: "GroupTopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - GroupTopNExecutor::<SerializedKey, MemoryStateStore, false>::new( - source as Box<dyn Executor>, - ActorContext::for_test(0), - info, - storage_key(), - (0, 2), - order_by_2(), - vec![1, 2], - state_table, - Arc::new(AtomicU64::new(0)), - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = GroupTopNExecutor::<SerializedKey, MemoryStateStore, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (0, 2), + order_by_2(), + vec![1, 2], + state_table, + Arc::new(AtomicU64::new(0)), + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); diff --git a/src/stream/src/executor/top_n/group_top_n_appendonly.rs b/src/stream/src/executor/top_n/group_top_n_appendonly.rs index 44522ddfb31ff..94f1df73770ab 100644 --- a/src/stream/src/executor/top_n/group_top_n_appendonly.rs +++ b/src/stream/src/executor/top_n/group_top_n_appendonly.rs @@ -16,6 +16,7 @@ use std::sync::Arc; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::Bitmap; +use risingwave_common::catalog::Schema; use risingwave_common::hash::HashKey; use risingwave_common::row::{RowDeserializer, RowExt}; use risingwave_common::util::epoch::EpochPair; @@ -31,7 +32,7 @@ use crate::common::metrics::MetricsInfo; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; use crate::executor::error::StreamExecutorResult; -use crate::executor::{ActorContextRef, Executor, ExecutorInfo, PkIndices, Watermark}; +use crate::executor::{ActorContextRef, Executor, PkIndices, Watermark}; use crate::task::AtomicU64Ref; /// If the input is append-only, `AppendOnlyGroupTopNExecutor` does not need @@ -45,9 +46,9 @@ impl<K: HashKey, S: StateStore, const WITH_TIES: bool> { #[allow(clippy::too_many_arguments)] pub fn new( - input: Box<dyn Executor>, + input: Executor, ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, storage_key: Vec<ColumnOrder>, offset_and_limit: (usize, usize), order_by: Vec<ColumnOrder>, @@ -59,7 +60,7 @@ impl<K: HashKey, S: StateStore, const WITH_TIES: bool> input, ctx: ctx.clone(), inner: InnerAppendOnlyGroupTopNExecutor::new( - info, + schema, storage_key, offset_and_limit, order_by, @@ -73,7 +74,7 @@ impl<K: HashKey, S: StateStore, const WITH_TIES: bool> } pub struct InnerAppendOnlyGroupTopNExecutor<K: HashKey, S: StateStore, const WITH_TIES: bool> { - info: ExecutorInfo, + schema: Schema, /// `LIMIT XXX`. None means no limit. limit: usize, @@ -103,7 +104,7 @@ impl<K: HashKey, S: StateStore, const WITH_TIES: bool> { #[allow(clippy::too_many_arguments)] pub fn new( - info: ExecutorInfo, + schema: Schema, storage_key: Vec<ColumnOrder>, offset_and_limit: (usize, usize), order_by: Vec<ColumnOrder>, @@ -119,12 +120,11 @@ impl<K: HashKey, S: StateStore, const WITH_TIES: bool> "GroupTopN", ); - let cache_key_serde = - create_cache_key_serde(&storage_key, &info.schema, &order_by, &group_by); + let cache_key_serde = create_cache_key_serde(&storage_key, &schema, &order_by, &group_by); let managed_state = ManagedTopNState::<S>::new(state_table, cache_key_serde.clone()); Ok(Self { - info, + schema, offset: offset_and_limit.0, limit: offset_and_limit.1, managed_state, @@ -147,7 +147,7 @@ where let mut res_rows = Vec::with_capacity(self.limit); let keys = K::build(&self.group_by, chunk.data_chunk())?; - let data_types = self.info().schema.data_types(); + let data_types = self.schema.data_types(); let row_deserializer = RowDeserializer::new(data_types.clone()); let table_id_str = self.managed_state.table().table_id().to_string(); let actor_id_str = self.ctx.id.to_string(); @@ -197,7 +197,7 @@ where .group_top_n_appendonly_cached_entry_count .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .set(self.caches.len() as i64); - generate_output(res_rows, res_ops, &self.info().schema) + generate_output(res_rows, res_ops, &self.schema) } async fn flush_data(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { @@ -208,10 +208,6 @@ where self.managed_state.try_flush().await } - fn info(&self) -> &ExecutorInfo { - &self.info - } - fn update_vnode_bitmap(&mut self, vnode_bitmap: Arc<Bitmap>) { let cache_may_stale = self.managed_state.update_vnode_bitmap(vnode_bitmap); if cache_may_stale { diff --git a/src/stream/src/executor/top_n/top_n_appendonly.rs b/src/stream/src/executor/top_n/top_n_appendonly.rs index 8da016fd8c7ac..095222939a6e8 100644 --- a/src/stream/src/executor/top_n/top_n_appendonly.rs +++ b/src/stream/src/executor/top_n/top_n_appendonly.rs @@ -13,6 +13,7 @@ // limitations under the License. use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::catalog::Schema; use risingwave_common::row::{RowDeserializer, RowExt}; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::sort_util::ColumnOrder; @@ -24,7 +25,7 @@ use super::{ManagedTopNState, TopNCache, NO_GROUP_KEY}; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; use crate::executor::error::StreamExecutorResult; -use crate::executor::{ActorContextRef, Executor, ExecutorInfo, PkIndices, Watermark}; +use crate::executor::{ActorContextRef, Executor, PkIndices, Watermark}; /// If the input is append-only, `AppendOnlyGroupTopNExecutor` does not need /// to keep all the rows seen. As long as a record @@ -38,9 +39,9 @@ pub type AppendOnlyTopNExecutor<S, const WITH_TIES: bool> = impl<S: StateStore, const WITH_TIES: bool> AppendOnlyTopNExecutor<S, WITH_TIES> { #[allow(clippy::too_many_arguments)] pub fn new( - input: Box<dyn Executor>, + input: Executor, ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, storage_key: Vec<ColumnOrder>, offset_and_limit: (usize, usize), order_by: Vec<ColumnOrder>, @@ -50,7 +51,7 @@ impl<S: StateStore, const WITH_TIES: bool> AppendOnlyTopNExecutor<S, WITH_TIES> input, ctx, inner: InnerAppendOnlyTopNExecutor::new( - info, + schema, storage_key, offset_and_limit, order_by, @@ -61,7 +62,7 @@ impl<S: StateStore, const WITH_TIES: bool> AppendOnlyTopNExecutor<S, WITH_TIES> } pub struct InnerAppendOnlyTopNExecutor<S: StateStore, const WITH_TIES: bool> { - info: ExecutorInfo, + schema: Schema, /// The storage key indices of the `TopNExecutor` storage_key_indices: PkIndices, @@ -80,7 +81,7 @@ pub struct InnerAppendOnlyTopNExecutor<S: StateStore, const WITH_TIES: bool> { impl<S: StateStore, const WITH_TIES: bool> InnerAppendOnlyTopNExecutor<S, WITH_TIES> { #[allow(clippy::too_many_arguments)] pub fn new( - info: ExecutorInfo, + schema: Schema, storage_key: Vec<ColumnOrder>, offset_and_limit: (usize, usize), order_by: Vec<ColumnOrder>, @@ -89,12 +90,12 @@ impl<S: StateStore, const WITH_TIES: bool> InnerAppendOnlyTopNExecutor<S, WITH_T let num_offset = offset_and_limit.0; let num_limit = offset_and_limit.1; - let cache_key_serde = create_cache_key_serde(&storage_key, &info.schema, &order_by, &[]); + let cache_key_serde = create_cache_key_serde(&storage_key, &schema, &order_by, &[]); let managed_state = ManagedTopNState::<S>::new(state_table, cache_key_serde.clone()); - let data_types = info.schema.data_types(); + let data_types = schema.data_types(); Ok(Self { - info, + schema, managed_state, storage_key_indices: storage_key.into_iter().map(|op| op.column_index).collect(), cache: TopNCache::new(num_offset, num_limit, data_types), @@ -111,7 +112,7 @@ where async fn apply_chunk(&mut self, chunk: StreamChunk) -> StreamExecutorResult<StreamChunk> { let mut res_ops = Vec::with_capacity(self.cache.limit); let mut res_rows = Vec::with_capacity(self.cache.limit); - let data_types = self.info().schema.data_types(); + let data_types = self.schema.data_types(); let row_deserializer = RowDeserializer::new(data_types); // apply the chunk to state table for (op, row_ref) in chunk.rows() { @@ -128,7 +129,7 @@ where )?; } - generate_output(res_rows, res_ops, &self.info().schema) + generate_output(res_rows, res_ops, &self.schema) } async fn flush_data(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { @@ -139,10 +140,6 @@ where self.managed_state.try_flush().await } - fn info(&self) -> &ExecutorInfo { - &self.info - } - async fn init(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { self.managed_state.init_epoch(epoch); self.managed_state @@ -170,7 +167,7 @@ mod tests { use super::AppendOnlyTopNExecutor; use crate::executor::test_utils::top_n_executor::create_in_memory_state_table; use crate::executor::test_utils::MockSource; - use crate::executor::{ActorContext, Barrier, Executor, ExecutorInfo, Message, PkIndices}; + use crate::executor::{ActorContext, Barrier, Execute, Executor, Message, PkIndices}; fn create_stream_chunks() -> Vec<StreamChunk> { let chunk1 = StreamChunk::from_pretty( @@ -223,21 +220,17 @@ mod tests { vec![0, 1] } - fn create_source() -> Box<MockSource> { + fn create_source() -> Executor { let mut chunks = create_stream_chunks(); - let schema = create_schema(); - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(std::mem::take(&mut chunks[2])), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(std::mem::take(&mut chunks[2])), + ]) + .into_executor(create_schema(), pk_indices()) } #[tokio::test] @@ -251,24 +244,18 @@ mod tests { ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "AppendOnlyTopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - AppendOnlyTopNExecutor::<_, false>::new( - source as Box<dyn Executor>, - ActorContext::for_test(0), - info, - storage_key, - (0, 5), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = AppendOnlyTopNExecutor::<_, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key, + (0, 5), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init epoch top_n_executor.next().await.unwrap().unwrap(); @@ -338,24 +325,18 @@ mod tests { ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "AppendOnlyTopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - AppendOnlyTopNExecutor::<_, false>::new( - source as Box<dyn Executor>, - ActorContext::for_test(0), - info, - storage_key(), - (3, 4), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = AppendOnlyTopNExecutor::<_, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (3, 4), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init epoch top_n_executor.next().await.unwrap().unwrap(); diff --git a/src/stream/src/executor/top_n/top_n_plain.rs b/src/stream/src/executor/top_n/top_n_plain.rs index 5df41cfca3a05..b09f13764ee79 100644 --- a/src/stream/src/executor/top_n/top_n_plain.rs +++ b/src/stream/src/executor/top_n/top_n_plain.rs @@ -13,6 +13,7 @@ // limitations under the License. use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::catalog::Schema; use risingwave_common::row::RowExt; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::sort_util::ColumnOrder; @@ -23,7 +24,7 @@ use super::{ManagedTopNState, TopNCache, TopNCacheTrait}; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; use crate::executor::error::StreamExecutorResult; -use crate::executor::{ActorContextRef, Executor, ExecutorInfo, PkIndices, Watermark}; +use crate::executor::{ActorContextRef, Executor, PkIndices, Watermark}; /// `TopNExecutor` works with input with modification, it keeps all the data /// records/rows that have been seen, and returns topN records overall. @@ -33,9 +34,9 @@ pub type TopNExecutor<S, const WITH_TIES: bool> = impl<S: StateStore, const WITH_TIES: bool> TopNExecutor<S, WITH_TIES> { #[allow(clippy::too_many_arguments)] pub fn new( - input: Box<dyn Executor>, + input: Executor, ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, storage_key: Vec<ColumnOrder>, offset_and_limit: (usize, usize), order_by: Vec<ColumnOrder>, @@ -45,7 +46,7 @@ impl<S: StateStore, const WITH_TIES: bool> TopNExecutor<S, WITH_TIES> { input, ctx, inner: InnerTopNExecutor::new( - info, + schema, storage_key, offset_and_limit, order_by, @@ -61,16 +62,16 @@ impl<S: StateStore> TopNExecutor<S, true> { #[allow(clippy::too_many_arguments)] #[cfg(test)] pub fn new_with_ties_for_test( - input: Box<dyn Executor>, + input: Executor, ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, storage_key: Vec<ColumnOrder>, offset_and_limit: (usize, usize), order_by: Vec<ColumnOrder>, state_table: StateTable<S>, ) -> StreamResult<Self> { let mut inner = - InnerTopNExecutor::new(info, storage_key, offset_and_limit, order_by, state_table)?; + InnerTopNExecutor::new(schema, storage_key, offset_and_limit, order_by, state_table)?; inner.cache.high_capacity = 2; @@ -79,7 +80,7 @@ impl<S: StateStore> TopNExecutor<S, true> { } pub struct InnerTopNExecutor<S: StateStore, const WITH_TIES: bool> { - info: ExecutorInfo, + schema: Schema, /// The storage key indices of the `TopNExecutor` storage_key_indices: PkIndices, @@ -103,7 +104,7 @@ impl<S: StateStore, const WITH_TIES: bool> InnerTopNExecutor<S, WITH_TIES> { /// into `CacheKey`. #[allow(clippy::too_many_arguments)] pub fn new( - info: ExecutorInfo, + schema: Schema, storage_key: Vec<ColumnOrder>, offset_and_limit: (usize, usize), order_by: Vec<ColumnOrder>, @@ -112,12 +113,12 @@ impl<S: StateStore, const WITH_TIES: bool> InnerTopNExecutor<S, WITH_TIES> { let num_offset = offset_and_limit.0; let num_limit = offset_and_limit.1; - let cache_key_serde = create_cache_key_serde(&storage_key, &info.schema, &order_by, &[]); + let cache_key_serde = create_cache_key_serde(&storage_key, &schema, &order_by, &[]); let managed_state = ManagedTopNState::<S>::new(state_table, cache_key_serde.clone()); - let data_types = info.schema.data_types(); + let data_types = schema.data_types(); Ok(Self { - info, + schema, managed_state, storage_key_indices: storage_key.into_iter().map(|op| op.column_index).collect(), cache: TopNCache::new(num_offset, num_limit, data_types), @@ -162,7 +163,7 @@ where } } } - generate_output(res_rows, res_ops, &self.info().schema) + generate_output(res_rows, res_ops, &self.schema) } async fn flush_data(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { @@ -173,10 +174,6 @@ where self.managed_state.try_flush().await } - fn info(&self) -> &ExecutorInfo { - &self.info - } - async fn init(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { self.managed_state.init_epoch(epoch); self.managed_state @@ -206,7 +203,7 @@ mod tests { mod test1 { use super::*; - use crate::executor::ActorContext; + use crate::executor::{ActorContext, Execute}; fn create_stream_chunks() -> Vec<StreamChunk> { let chunk1 = StreamChunk::from_pretty( " I I @@ -265,24 +262,21 @@ mod tests { vec![0, 1] } - fn create_source() -> Box<MockSource> { + fn create_source() -> Executor { let mut chunks = create_stream_chunks(); let schema = create_schema(); - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(std::mem::take(&mut chunks[2])), - Message::Barrier(Barrier::new_test_barrier(4)), - Message::Chunk(std::mem::take(&mut chunks[3])), - Message::Barrier(Barrier::new_test_barrier(5)), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(std::mem::take(&mut chunks[2])), + Message::Barrier(Barrier::new_test_barrier(4)), + Message::Chunk(std::mem::take(&mut chunks[3])), + Message::Barrier(Barrier::new_test_barrier(5)), + ]) + .into_executor(schema, pk_indices()) } #[tokio::test] @@ -295,24 +289,18 @@ mod tests { ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - TopNExecutor::<_, false>::new( - source as Box<dyn Executor>, - ActorContext::for_test(0), - info, - storage_key(), - (3, 1000), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = TopNExecutor::<_, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (3, 1000), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -396,24 +384,18 @@ mod tests { &pk_indices(), ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - TopNExecutor::<_, false>::new( - source as Box<dyn Executor>, - ActorContext::for_test(0), - info, - storage_key(), - (0, 4), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = TopNExecutor::<_, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (0, 4), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -509,24 +491,18 @@ mod tests { &pk_indices(), ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - TopNExecutor::<_, true>::new( - source as Box<dyn Executor>, - ActorContext::for_test(0), - info, - storage_key(), - (0, 4), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = TopNExecutor::<_, true>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (0, 4), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -621,24 +597,18 @@ mod tests { &pk_indices(), ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - TopNExecutor::<_, false>::new( - source as Box<dyn Executor>, - ActorContext::for_test(0), - info, - storage_key(), - (3, 4), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = TopNExecutor::<_, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (3, 4), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -715,9 +685,9 @@ mod tests { use super::*; use crate::executor::test_utils::top_n_executor::create_in_memory_state_table_from_state_store; - use crate::executor::ActorContext; + use crate::executor::{ActorContext, Execute}; - fn create_source_new() -> Box<MockSource> { + fn create_source_new() -> Executor { let mut chunks = vec![ StreamChunk::from_pretty( " I I I I @@ -746,21 +716,18 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Chunk(std::mem::take(&mut chunks[2])), - Message::Chunk(std::mem::take(&mut chunks[3])), - Message::Barrier(Barrier::new_test_barrier(2)), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Chunk(std::mem::take(&mut chunks[2])), + Message::Chunk(std::mem::take(&mut chunks[3])), + Message::Barrier(Barrier::new_test_barrier(2)), + ]) + .into_executor(schema, pk_indices()) } - fn create_source_new_before_recovery() -> Box<MockSource> { + fn create_source_new_before_recovery() -> Executor { let mut chunks = [ StreamChunk::from_pretty( " I I I I @@ -779,19 +746,16 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Barrier(Barrier::new_test_barrier(2)), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Barrier(Barrier::new_test_barrier(2)), + ]) + .into_executor(schema, pk_indices()) } - fn create_source_new_after_recovery() -> Box<MockSource> { + fn create_source_new_after_recovery() -> Executor { let mut chunks = [ StreamChunk::from_pretty( " I I I I @@ -812,16 +776,13 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Barrier(Barrier::new_test_barrier(4)), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Barrier(Barrier::new_test_barrier(4)), + ]) + .into_executor(schema, pk_indices()) } fn storage_key() -> Vec<ColumnOrder> { @@ -853,24 +814,18 @@ mod tests { &pk_indices(), ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - TopNExecutor::<_, false>::new( - source as Box<dyn Executor>, - ActorContext::for_test(0), - info, - storage_key(), - (1, 3), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = TopNExecutor::<_, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (1, 3), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -937,24 +892,18 @@ mod tests { ) .await; let source = create_source_new_before_recovery(); - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - TopNExecutor::<_, false>::new( - source as Box<dyn Executor>, - ActorContext::for_test(0), - info, - storage_key(), - (1, 3), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = TopNExecutor::<_, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (1, 3), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -997,24 +946,18 @@ mod tests { // recovery let source = create_source_new_after_recovery(); - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor_after_recovery = Box::new( - TopNExecutor::<_, false>::new( - source as Box<dyn Executor>, - ActorContext::for_test(0), - info, - storage_key(), - (1, 3), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor_after_recovery.execute(); + let schema = source.schema().clone(); + let top_n_executor_after_recovery = TopNExecutor::<_, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (1, 3), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor_after_recovery.boxed().execute(); // barrier assert_matches!( @@ -1057,9 +1000,9 @@ mod tests { use super::*; use crate::executor::test_utils::top_n_executor::create_in_memory_state_table_from_state_store; - use crate::executor::ActorContext; + use crate::executor::{ActorContext, Execute}; - fn create_source() -> Box<MockSource> { + fn create_source() -> Executor { let mut chunks = vec![ StreamChunk::from_pretty( " I I @@ -1094,18 +1037,15 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Chunk(std::mem::take(&mut chunks[2])), - Message::Chunk(std::mem::take(&mut chunks[3])), - Message::Barrier(Barrier::new_test_barrier(2)), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Chunk(std::mem::take(&mut chunks[2])), + Message::Chunk(std::mem::take(&mut chunks[3])), + Message::Barrier(Barrier::new_test_barrier(2)), + ]) + .into_executor(schema, pk_indices()) } fn storage_key() -> Vec<ColumnOrder> { @@ -1131,24 +1071,18 @@ mod tests { &pk_indices(), ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - TopNExecutor::new_with_ties_for_test( - source as Box<dyn Executor>, - ActorContext::for_test(0), - info, - storage_key(), - (0, 3), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = TopNExecutor::new_with_ties_for_test( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (0, 3), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -1208,7 +1142,7 @@ mod tests { ); } - fn create_source_before_recovery() -> Box<MockSource> { + fn create_source_before_recovery() -> Executor { let mut chunks = [ StreamChunk::from_pretty( " I I @@ -1235,19 +1169,16 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Barrier(Barrier::new_test_barrier(2)), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Barrier(Barrier::new_test_barrier(2)), + ]) + .into_executor(schema, pk_indices()) } - fn create_source_after_recovery() -> Box<MockSource> { + fn create_source_after_recovery() -> Executor { let mut chunks = [ StreamChunk::from_pretty( " I I @@ -1264,16 +1195,13 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Barrier(Barrier::new_test_barrier(4)), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Barrier(Barrier::new_test_barrier(4)), + ]) + .into_executor(schema, pk_indices()) } #[tokio::test] @@ -1287,24 +1215,18 @@ mod tests { ) .await; let source = create_source_before_recovery(); - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - TopNExecutor::new_with_ties_for_test( - source as Box<dyn Executor>, - ActorContext::for_test(0), - info, - storage_key(), - (0, 3), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = TopNExecutor::new_with_ties_for_test( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (0, 3), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -1350,24 +1272,18 @@ mod tests { // recovery let source = create_source_after_recovery(); - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor_after_recovery = Box::new( - TopNExecutor::new_with_ties_for_test( - source as Box<dyn Executor>, - ActorContext::for_test(0), - info, - storage_key(), - (0, 3), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor_after_recovery.execute(); + let schema = source.schema().clone(); + let top_n_executor_after_recovery = TopNExecutor::new_with_ties_for_test( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (0, 3), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor_after_recovery.boxed().execute(); // barrier assert_matches!( diff --git a/src/stream/src/executor/top_n/utils.rs b/src/stream/src/executor/top_n/utils.rs index b2310f0d352b1..bbd956cde2168 100644 --- a/src/stream/src/executor/top_n/utils.rs +++ b/src/stream/src/executor/top_n/utils.rs @@ -30,8 +30,8 @@ use risingwave_common::util::sort_util::ColumnOrder; use super::CacheKey; use crate::executor::error::{StreamExecutorError, StreamExecutorResult}; use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, PkIndicesRef, Watermark, + expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, + Watermark, }; pub trait TopNExecutorBase: Send + 'static { @@ -50,8 +50,6 @@ pub trait TopNExecutorBase: Send + 'static { /// Flush the buffered chunk to the storage backend. fn try_flush_data(&mut self) -> impl Future<Output = StreamExecutorResult<()>> + Send; - fn info(&self) -> &ExecutorInfo; - /// Update the vnode bitmap for the state table and manipulate the cache if necessary, only used /// by Group Top-N since it's distributed. fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc<Bitmap>) { @@ -72,34 +70,18 @@ pub trait TopNExecutorBase: Send + 'static { /// The struct wraps a [`TopNExecutorBase`] pub struct TopNExecutorWrapper<E> { - pub(super) input: BoxedExecutor, + pub(super) input: Executor, pub(super) ctx: ActorContextRef, pub(super) inner: E, } -impl<E> Executor for TopNExecutorWrapper<E> +impl<E> Execute for TopNExecutorWrapper<E> where E: TopNExecutorBase, { fn execute(self: Box<Self>) -> BoxedMessageStream { self.top_n_executor_execute().boxed() } - - fn schema(&self) -> &Schema { - &self.inner.info().schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.inner.info().pk_indices - } - - fn identity(&self) -> &str { - &self.inner.info().identity - } - - fn info(&self) -> ExecutorInfo { - self.inner.info().clone() - } } impl<E> TopNExecutorWrapper<E> diff --git a/src/stream/src/executor/union.rs b/src/stream/src/executor/union.rs index a3b3df6873e6b..acd35ce080cda 100644 --- a/src/stream/src/executor/union.rs +++ b/src/stream/src/executor/union.rs @@ -17,57 +17,37 @@ use std::pin::Pin; use std::task::{Context, Poll}; use futures::stream::{FusedStream, FuturesUnordered}; -use futures::StreamExt; +use futures::{Stream, StreamExt}; use futures_async_stream::try_stream; use pin_project::pin_project; -use risingwave_common::catalog::Schema; use super::watermark::BufferedWatermarks; -use super::*; -use crate::executor::{BoxedMessageStream, ExecutorInfo}; +use super::{ + Barrier, BoxedMessageStream, Execute, Executor, Message, MessageStreamItem, StreamExecutorError, +}; /// `UnionExecutor` merges data from multiple inputs. pub struct UnionExecutor { - info: ExecutorInfo, - inputs: Vec<BoxedExecutor>, + inputs: Vec<Executor>, } impl std::fmt::Debug for UnionExecutor { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("UnionExecutor") - .field("schema", &self.info.schema) - .field("pk_indices", &self.info.pk_indices) - .finish() + f.debug_struct("UnionExecutor").finish() } } impl UnionExecutor { - pub fn new(info: ExecutorInfo, inputs: Vec<BoxedExecutor>) -> Self { - Self { info, inputs } + pub fn new(inputs: Vec<Executor>) -> Self { + Self { inputs } } } -impl Executor for UnionExecutor { +impl Execute for UnionExecutor { fn execute(self: Box<Self>) -> BoxedMessageStream { let streams = self.inputs.into_iter().map(|e| e.execute()).collect(); merge(streams).boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - - fn info(&self) -> ExecutorInfo { - self.info.clone() - } } #[pin_project] @@ -170,8 +150,10 @@ mod tests { use async_stream::try_stream; use risingwave_common::array::stream_chunk::StreamChunkTestExt; use risingwave_common::array::StreamChunk; + use risingwave_common::types::{DataType, ScalarImpl}; use super::*; + use crate::executor::Watermark; #[tokio::test] async fn union() { diff --git a/src/stream/src/executor/utils.rs b/src/stream/src/executor/utils.rs index 8ff9f8f29576d..18b64633a3e12 100644 --- a/src/stream/src/executor/utils.rs +++ b/src/stream/src/executor/utils.rs @@ -13,40 +13,19 @@ // limitations under the License. use futures::StreamExt; -use risingwave_common::catalog::Schema; use risingwave_common::metrics::LabelGuardedIntCounter; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{BoxedMessageStream, Executor, ExecutorInfo, PkIndicesRef}; +use crate::executor::{BoxedMessageStream, Execute}; use crate::task::{ActorId, FragmentId}; #[derive(Default)] -pub struct DummyExecutor { - pub info: ExecutorInfo, -} +pub struct DummyExecutor; -impl DummyExecutor { - pub fn new(info: ExecutorInfo) -> Self { - Self { info } - } -} - -impl Executor for DummyExecutor { +impl Execute for DummyExecutor { fn execute(self: Box<Self>) -> BoxedMessageStream { futures::stream::pending().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } pub(crate) struct ActorInputMetrics { diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index 2a927f1a3a780..d7deee71f1443 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -24,10 +24,7 @@ use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::expr::NonStrictExpression; use tokio::sync::mpsc::UnboundedReceiver; -use super::{ - ActorContextRef, Barrier, BoxedMessageStream, Executor, ExecutorInfo, Message, PkIndicesRef, - StreamExecutorError, -}; +use super::{ActorContextRef, Barrier, BoxedMessageStream, Execute, Message, StreamExecutorError}; use crate::task::CreateMviewProgress; const DEFAULT_CHUNK_SIZE: usize = 1024; @@ -36,8 +33,8 @@ const DEFAULT_CHUNK_SIZE: usize = 1024; /// May refractor with `BarrierRecvExecutor` in the near future. pub struct ValuesExecutor { ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, // Receiver of barrier channel. barrier_receiver: UnboundedReceiver<Barrier>, progress: CreateMviewProgress, @@ -49,14 +46,14 @@ impl ValuesExecutor { /// Currently hard-code the `pk_indices` as the last column. pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, progress: CreateMviewProgress, rows: Vec<Vec<NonStrictExpression>>, barrier_receiver: UnboundedReceiver<Barrier>, ) -> Self { Self { ctx, - info, + schema, progress, barrier_receiver, rows: rows.into_iter(), @@ -64,9 +61,9 @@ impl ValuesExecutor { } #[try_stream(ok = Message, error = StreamExecutorError)] - async fn into_stream(self) { + async fn execute_inner(self) { let Self { - info, + schema, mut progress, mut barrier_receiver, mut rows, @@ -96,7 +93,7 @@ impl ValuesExecutor { } } - let cardinality = info.schema.len(); + let cardinality = schema.len(); ensure!(cardinality > 0); while !rows.is_empty() { // We need a one row chunk rather than an empty chunk because constant @@ -105,7 +102,7 @@ impl ValuesExecutor { let one_row_chunk = DataChunk::new_dummy(1); let chunk_size = DEFAULT_CHUNK_SIZE.min(rows.len()); - let mut array_builders = info.schema.create_array_builders(chunk_size); + let mut array_builders = schema.create_array_builders(chunk_size); for row in rows.by_ref().take(chunk_size) { for (expr, builder) in row.into_iter().zip_eq_fast(&mut array_builders) { let out = expr.eval_infallible(&one_row_chunk).await; @@ -135,21 +132,9 @@ impl ValuesExecutor { } } -impl Executor for ValuesExecutor { +impl Execute for ValuesExecutor { fn execute(self: Box<Self>) -> BoxedMessageStream { - self.into_stream().boxed() - } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity + self.execute_inner().boxed() } } @@ -167,7 +152,7 @@ mod tests { use super::ValuesExecutor; use crate::executor::test_utils::StreamExecutorTestExt; - use crate::executor::{ActorContext, AddMutation, Barrier, Executor, ExecutorInfo, Mutation}; + use crate::executor::{ActorContext, AddMutation, Barrier, Execute, Mutation}; use crate::task::{CreateMviewProgress, LocalBarrierManager}; #[tokio::test] @@ -206,15 +191,9 @@ mod tests { .iter() // for each column .map(|col| Field::unnamed(col.return_type())) .collect::<Schema>(); - let pk_indices = vec![schema.len() - 1]; - let info = ExecutorInfo { - schema, - pk_indices, - identity: "ValuesExecutor".to_string(), - }; let values_executor_struct = ValuesExecutor::new( ActorContext::for_test(actor_id), - info, + schema, progress, vec![exprs .into_iter() diff --git a/src/stream/src/executor/watermark_filter.rs b/src/stream/src/executor/watermark_filter.rs index 1aa671b4f671c..4875d8a53c2d1 100644 --- a/src/stream/src/executor/watermark_filter.rs +++ b/src/stream/src/executor/watermark_filter.rs @@ -36,9 +36,7 @@ use risingwave_storage::StateStore; use super::error::StreamExecutorError; use super::filter::FilterExecutor; -use super::{ - ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, Message, StreamExecutorResult, -}; +use super::{ActorContextRef, Execute, Executor, ExecutorInfo, Message, StreamExecutorResult}; use crate::common::table::state_table::StateTable; use crate::executor::{expect_first_barrier, Watermark}; use crate::task::ActorEvalErrorReport; @@ -48,59 +46,49 @@ use crate::task::ActorEvalErrorReport; /// filtered. pub struct WatermarkFilterExecutor<S: StateStore> { ctx: ActorContextRef, - info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, /// The expression used to calculate the watermark value. watermark_expr: NonStrictExpression, /// The column we should generate watermark and filter on. event_time_col_idx: usize, table: StateTable<S>, global_watermark_table: StorageTable<S>, + + eval_error_report: ActorEvalErrorReport, } impl<S: StateStore> WatermarkFilterExecutor<S> { pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, - input: BoxedExecutor, + info: &ExecutorInfo, + input: Executor, watermark_expr: NonStrictExpression, event_time_col_idx: usize, table: StateTable<S>, global_watermark_table: StorageTable<S>, ) -> Self { + let eval_error_report = ActorEvalErrorReport { + actor_context: ctx.clone(), + identity: Arc::from(info.identity.as_ref()), + }; + Self { ctx, - info, input, watermark_expr, event_time_col_idx, table, global_watermark_table, + eval_error_report, } } } -impl<S: StateStore> Executor for WatermarkFilterExecutor<S> { +impl<S: StateStore> Execute for WatermarkFilterExecutor<S> { fn execute(self: Box<Self>) -> super::BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &risingwave_common::catalog::Schema { - &self.info.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - - fn info(&self) -> ExecutorInfo { - self.info.clone() - } } impl<S: StateStore> WatermarkFilterExecutor<S> { @@ -111,16 +99,11 @@ impl<S: StateStore> WatermarkFilterExecutor<S> { event_time_col_idx, watermark_expr, ctx, - info, mut table, mut global_watermark_table, + eval_error_report, } = *self; - let eval_error_report = ActorEvalErrorReport { - actor_context: ctx.clone(), - identity: info.identity.into(), - }; - let watermark_type = watermark_expr.return_type(); assert_eq!( watermark_type, @@ -456,7 +439,7 @@ mod tests { async fn create_watermark_filter_executor( mem_state: MemoryStateStore, - ) -> (BoxedExecutor, MessageSender) { + ) -> (Box<dyn Execute>, MessageSender) { let schema = Schema { fields: vec![ Field::unnamed(DataType::Int16), // pk @@ -476,7 +459,8 @@ mod tests { ) .await; - let (tx, source) = MockSource::channel(schema, vec![0]); + let (tx, source) = MockSource::channel(); + let source = source.into_executor(schema, vec![0]); let info = ExecutorInfo { schema: source.schema().clone(), @@ -487,8 +471,8 @@ mod tests { ( WatermarkFilterExecutor::new( ActorContext::for_test(123), - info, - source.boxed(), + &info, + source, watermark_expr, 1, table, diff --git a/src/stream/src/executor/wrapper.rs b/src/stream/src/executor/wrapper.rs index ebd3f6d9bfa73..dddd94da5ab73 100644 --- a/src/stream/src/executor/wrapper.rs +++ b/src/stream/src/executor/wrapper.rs @@ -15,9 +15,8 @@ use std::sync::Arc; use futures::StreamExt; -use risingwave_common::catalog::Schema; -use super::*; +use super::{ActorContextRef, BoxedMessageStream, Execute, Executor, ExecutorInfo, MessageStream}; mod epoch_check; mod epoch_provide; @@ -27,7 +26,7 @@ mod update_check; /// [`WrapperExecutor`] will do some sanity checks and logging for the wrapped executor. pub struct WrapperExecutor { - input: BoxedExecutor, + input: Executor, actor_ctx: ActorContextRef, @@ -36,7 +35,7 @@ pub struct WrapperExecutor { impl WrapperExecutor { pub fn new( - input: BoxedExecutor, + input: Executor, actor_ctx: ActorContextRef, enable_executor_row_count: bool, ) -> Self { @@ -88,9 +87,9 @@ impl WrapperExecutor { } } -impl Executor for WrapperExecutor { +impl Execute for WrapperExecutor { fn execute(self: Box<Self>) -> BoxedMessageStream { - let info = Arc::new(self.input.info()); + let info = Arc::new(self.input.info().clone()); Self::wrap( self.enable_executor_row_count, info, @@ -101,7 +100,7 @@ impl Executor for WrapperExecutor { } fn execute_with_epoch(self: Box<Self>, epoch: u64) -> BoxedMessageStream { - let info = Arc::new(self.input.info()); + let info = Arc::new(self.input.info().clone()); Self::wrap( self.enable_executor_row_count, info, @@ -110,16 +109,4 @@ impl Executor for WrapperExecutor { ) .boxed() } - - fn schema(&self) -> &Schema { - self.input.schema() - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - self.input.pk_indices() - } - - fn identity(&self) -> &str { - self.input.identity() - } } diff --git a/src/stream/src/executor/wrapper/epoch_check.rs b/src/stream/src/executor/wrapper/epoch_check.rs index 732c67630a345..13d755dcbc60d 100644 --- a/src/stream/src/executor/wrapper/epoch_check.rs +++ b/src/stream/src/executor/wrapper/epoch_check.rs @@ -80,18 +80,18 @@ mod tests { use super::*; use crate::executor::test_utils::MockSource; - use crate::executor::Executor; #[tokio::test] async fn test_epoch_ok() { - let (mut tx, source) = MockSource::channel(Default::default(), vec![]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(Default::default(), vec![]); tx.push_barrier(1, false); tx.push_chunk(StreamChunk::default()); tx.push_barrier(2, false); tx.push_barrier(3, false); tx.push_barrier(4, false); - let checked = epoch_check(source.info().into(), source.boxed().execute()); + let checked = epoch_check(source.info().clone().into(), source.execute()); pin_mut!(checked); assert_matches!(checked.next().await.unwrap().unwrap(), Message::Barrier(b) if b.epoch.curr == 1); @@ -104,14 +104,15 @@ mod tests { #[should_panic] #[tokio::test] async fn test_epoch_bad() { - let (mut tx, source) = MockSource::channel(Default::default(), vec![]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(Default::default(), vec![]); tx.push_barrier(100, false); tx.push_chunk(StreamChunk::default()); tx.push_barrier(514, false); tx.push_barrier(514, false); tx.push_barrier(114, false); - let checked = epoch_check(source.info().into(), source.boxed().execute()); + let checked = epoch_check(source.info().clone().into(), source.execute()); pin_mut!(checked); assert_matches!(checked.next().await.unwrap().unwrap(), Message::Barrier(b) if b.epoch.curr == 100); @@ -125,11 +126,12 @@ mod tests { #[should_panic] #[tokio::test] async fn test_epoch_first_not_barrier() { - let (mut tx, source) = MockSource::channel(Default::default(), vec![]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(Default::default(), vec![]); tx.push_chunk(StreamChunk::default()); tx.push_barrier(114, false); - let checked = epoch_check(source.info().into(), source.boxed().execute()); + let checked = epoch_check(source.info().clone().into(), source.execute()); pin_mut!(checked); checked.next().await.unwrap().unwrap(); // should panic @@ -137,9 +139,11 @@ mod tests { #[tokio::test] async fn test_empty() { - let (_, mut source) = MockSource::channel(Default::default(), vec![]); - source = source.stop_on_finish(false); - let checked = epoch_check(source.info().into(), source.boxed().execute()); + let (_, source) = MockSource::channel(); + let source = source + .stop_on_finish(false) + .into_executor(Default::default(), vec![]); + let checked = epoch_check(source.info().clone().into(), source.execute()); pin_mut!(checked); assert!(checked.next().await.transpose().unwrap().is_none()); diff --git a/src/stream/src/executor/wrapper/schema_check.rs b/src/stream/src/executor/wrapper/schema_check.rs index 462ef5962042e..ad5f0acd3dfaa 100644 --- a/src/stream/src/executor/wrapper/schema_check.rs +++ b/src/stream/src/executor/wrapper/schema_check.rs @@ -62,7 +62,6 @@ mod tests { use super::*; use crate::executor::test_utils::MockSource; - use crate::executor::Executor; #[tokio::test] async fn test_schema_ok() { @@ -73,7 +72,8 @@ mod tests { ], }; - let (mut tx, source) = MockSource::channel(schema, vec![1]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, vec![1]); tx.push_chunk(StreamChunk::from_pretty( " I F + 100 200.0 @@ -82,7 +82,7 @@ mod tests { )); tx.push_barrier(1, false); - let checked = schema_check(source.info().into(), source.boxed().execute()); + let checked = schema_check(source.info().clone().into(), source.execute()); pin_mut!(checked); assert_matches!(checked.next().await.unwrap().unwrap(), Message::Chunk(_)); @@ -99,7 +99,8 @@ mod tests { ], }; - let (mut tx, source) = MockSource::channel(schema, vec![1]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, vec![1]); tx.push_chunk(StreamChunk::from_pretty( " I I + 100 200 @@ -108,7 +109,7 @@ mod tests { )); tx.push_barrier(1, false); - let checked = schema_check(source.info().into(), source.boxed().execute()); + let checked = schema_check(source.info().clone().into(), source.execute()); pin_mut!(checked); checked.next().await.unwrap().unwrap(); } diff --git a/src/stream/src/executor/wrapper/update_check.rs b/src/stream/src/executor/wrapper/update_check.rs index 4049960845282..37a2ca4f2a942 100644 --- a/src/stream/src/executor/wrapper/update_check.rs +++ b/src/stream/src/executor/wrapper/update_check.rs @@ -62,12 +62,12 @@ mod tests { use super::*; use crate::executor::test_utils::MockSource; - use crate::executor::Executor; #[should_panic] #[tokio::test] async fn test_not_next_to_each_other() { - let (mut tx, source) = MockSource::channel(Default::default(), vec![]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(Default::default(), vec![]); tx.push_chunk(StreamChunk::from_pretty( " I U- 114 @@ -76,7 +76,7 @@ mod tests { U+ 810", )); - let checked = update_check(source.info().into(), source.boxed().execute()); + let checked = update_check(source.info().clone().into(), source.execute()); pin_mut!(checked); checked.next().await.unwrap().unwrap(); // should panic @@ -85,13 +85,14 @@ mod tests { #[should_panic] #[tokio::test] async fn test_first_one_update_insert() { - let (mut tx, source) = MockSource::channel(Default::default(), vec![]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(Default::default(), vec![]); tx.push_chunk(StreamChunk::from_pretty( " I U+ 114", )); - let checked = update_check(source.info().into(), source.boxed().execute()); + let checked = update_check(source.info().clone().into(), source.execute()); pin_mut!(checked); checked.next().await.unwrap().unwrap(); // should panic @@ -100,7 +101,8 @@ mod tests { #[should_panic] #[tokio::test] async fn test_last_one_update_delete() { - let (mut tx, source) = MockSource::channel(Default::default(), vec![]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(Default::default(), vec![]); tx.push_chunk(StreamChunk::from_pretty( " I U- 114 @@ -108,7 +110,7 @@ mod tests { U- 1919810", )); - let checked = update_check(source.info().into(), source.boxed().execute()); + let checked = update_check(source.info().clone().into(), source.execute()); pin_mut!(checked); checked.next().await.unwrap().unwrap(); // should panic @@ -116,10 +118,11 @@ mod tests { #[tokio::test] async fn test_empty_chunk() { - let (mut tx, source) = MockSource::channel(Default::default(), vec![]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(Default::default(), vec![]); tx.push_chunk(StreamChunk::default()); - let checked = update_check(source.info().into(), source.boxed().execute()); + let checked = update_check(source.info().clone().into(), source.execute()); pin_mut!(checked); checked.next().await.unwrap().unwrap(); diff --git a/src/stream/src/from_proto/append_only_dedup.rs b/src/stream/src/from_proto/append_only_dedup.rs index 4ae2ee24ce798..020d13f3ff96b 100644 --- a/src/stream/src/from_proto/append_only_dedup.rs +++ b/src/stream/src/from_proto/append_only_dedup.rs @@ -20,7 +20,7 @@ use risingwave_storage::StateStore; use super::ExecutorBuilder; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; -use crate::executor::{AppendOnlyDedupExecutor, BoxedExecutor}; +use crate::executor::{AppendOnlyDedupExecutor, Executor}; use crate::task::ExecutorParams; pub struct AppendOnlyDedupExecutorBuilder; @@ -32,18 +32,19 @@ impl ExecutorBuilder for AppendOnlyDedupExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); let table = node.get_state_table()?; let vnodes = params.vnode_bitmap.map(Arc::new); let state_table = StateTable::from_table_catalog(table, store, vnodes).await; - Ok(Box::new(AppendOnlyDedupExecutor::new( + let exec = AppendOnlyDedupExecutor::new( + params.actor_context, input, + params.info.pk_indices.clone(), /* TODO(rc): should change to use `dedup_column_indices`, but need to check backward compatibility */ state_table, - params.info, - params.actor_context, params.watermark_epoch, params.executor_stats.clone(), - ))) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/barrier_recv.rs b/src/stream/src/from_proto/barrier_recv.rs index 032afde2ee1dc..b8c38f0f53d32 100644 --- a/src/stream/src/from_proto/barrier_recv.rs +++ b/src/stream/src/from_proto/barrier_recv.rs @@ -27,7 +27,7 @@ impl ExecutorBuilder for BarrierRecvExecutorBuilder { params: ExecutorParams, _node: &Self::Node, _store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { assert!( params.input.is_empty(), "barrier receiver should not have input" @@ -38,6 +38,7 @@ impl ExecutorBuilder for BarrierRecvExecutorBuilder { .local_barrier_manager .register_sender(params.actor_context.id, sender); - Ok(BarrierRecvExecutor::new(params.actor_context, params.info, barrier_receiver).boxed()) + let exec = BarrierRecvExecutor::new(params.actor_context, barrier_receiver); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/batch_query.rs b/src/stream/src/from_proto/batch_query.rs index 4fc492f7c984d..5e86d7e6d5b30 100644 --- a/src/stream/src/from_proto/batch_query.rs +++ b/src/stream/src/from_proto/batch_query.rs @@ -30,12 +30,12 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder { params: ExecutorParams, node: &Self::Node, state_store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { if node.table_desc.is_none() { // used in sharing cdc source backfill as a dummy batch plan node let mut info = params.info; info.identity = "DummyBatchQueryExecutor".to_string(); - return Ok(Box::new(DummyExecutor::new(info))); + return Ok((info, DummyExecutor).into()); } let table_desc: &StorageTableDesc = node.get_table_desc()?; @@ -55,9 +55,11 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder { ); assert_eq!(table.schema().data_types(), params.info.schema.data_types()); - let executor = - BatchQueryExecutor::new(table, params.env.config().developer.chunk_size, params.info); - - Ok(executor.boxed()) + let exec = BatchQueryExecutor::new( + table, + params.env.config().developer.chunk_size, + params.info.schema.clone(), + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/cdc_filter.rs b/src/stream/src/from_proto/cdc_filter.rs index 5747e1894d4a8..673f6a1c5e712 100644 --- a/src/stream/src/from_proto/cdc_filter.rs +++ b/src/stream/src/from_proto/cdc_filter.rs @@ -28,11 +28,12 @@ impl ExecutorBuilder for CdcFilterExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); let search_condition = build_non_strict_from_prost(node.get_search_condition()?, params.eval_error_report)?; - Ok(FilterExecutor::new(params.actor_context, params.info, input, search_condition).boxed()) + let exec = FilterExecutor::new(params.actor_context, input, search_condition); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/dml.rs b/src/stream/src/from_proto/dml.rs index d7f21fda7bf2d..cc8836890ce7f 100644 --- a/src/stream/src/from_proto/dml.rs +++ b/src/stream/src/from_proto/dml.rs @@ -20,7 +20,7 @@ use risingwave_storage::StateStore; use super::ExecutorBuilder; use crate::error::StreamResult; use crate::executor::dml::DmlExecutor; -use crate::executor::BoxedExecutor; +use crate::executor::Executor; use crate::task::ExecutorParams; pub struct DmlExecutorBuilder; @@ -32,19 +32,19 @@ impl ExecutorBuilder for DmlExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [upstream]: [_; 1] = params.input.try_into().unwrap(); let table_id = TableId::new(node.table_id); let column_descs = node.column_descs.iter().map(Into::into).collect_vec(); - Ok(Box::new(DmlExecutor::new( - params.info, + let exec = DmlExecutor::new( upstream, params.env.dml_manager_ref(), table_id, node.table_version_id, column_descs, params.env.config().developer.chunk_size, - ))) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/dynamic_filter.rs b/src/stream/src/from_proto/dynamic_filter.rs index babb2680a9575..c09fd767ad9ef 100644 --- a/src/stream/src/from_proto/dynamic_filter.rs +++ b/src/stream/src/from_proto/dynamic_filter.rs @@ -32,7 +32,7 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [source_l, source_r]: [_; 2] = params.input.try_into().unwrap(); let key_l = node.get_left_key() as usize; @@ -62,7 +62,7 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder { let left_table = node.get_left_table()?; let cleaned_by_watermark = left_table.get_cleaned_by_watermark(); - if cleaned_by_watermark { + let exec = if cleaned_by_watermark { let state_table_l = WatermarkCacheStateTable::from_table_catalog( node.get_left_table()?, store, @@ -70,9 +70,9 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder { ) .await; - Ok(Box::new(DynamicFilterExecutor::new( + DynamicFilterExecutor::new( params.actor_context, - params.info, + ¶ms.info, source_l, source_r, key_l, @@ -83,14 +83,15 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder { params.env.config().developer.chunk_size, condition_always_relax, cleaned_by_watermark, - ))) + ) + .boxed() } else { let state_table_l = StateTable::from_table_catalog(node.get_left_table()?, store, Some(vnodes)).await; - Ok(Box::new(DynamicFilterExecutor::new( + DynamicFilterExecutor::new( params.actor_context, - params.info, + ¶ms.info, source_l, source_r, key_l, @@ -101,7 +102,10 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder { params.env.config().developer.chunk_size, condition_always_relax, cleaned_by_watermark, - ))) - } + ) + .boxed() + }; + + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/eowc_over_window.rs b/src/stream/src/from_proto/eowc_over_window.rs index b2136f51c3970..4f6e873d7bcf3 100644 --- a/src/stream/src/from_proto/eowc_over_window.rs +++ b/src/stream/src/from_proto/eowc_over_window.rs @@ -21,9 +21,7 @@ use risingwave_storage::StateStore; use super::ExecutorBuilder; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; -use crate::executor::{ - BoxedExecutor, EowcOverWindowExecutor, EowcOverWindowExecutorArgs, Executor, -}; +use crate::executor::{EowcOverWindowExecutor, EowcOverWindowExecutorArgs, Executor}; use crate::task::ExecutorParams; pub struct EowcOverWindowExecutorBuilder; @@ -35,7 +33,7 @@ impl ExecutorBuilder for EowcOverWindowExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); let calls: Vec<_> = node .get_calls() @@ -56,18 +54,18 @@ impl ExecutorBuilder for EowcOverWindowExecutorBuilder { let state_table = StateTable::from_table_catalog_inconsistent_op(node.get_state_table()?, store, vnodes) .await; - Ok(EowcOverWindowExecutor::new(EowcOverWindowExecutorArgs { + let exec = EowcOverWindowExecutor::new(EowcOverWindowExecutorArgs { actor_ctx: params.actor_context, - info: params.info, input, + schema: params.info.schema.clone(), calls, partition_key_indices, order_key_index, state_table, watermark_epoch: params.watermark_epoch, - }) - .boxed()) + }); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/expand.rs b/src/stream/src/from_proto/expand.rs index 40bb972cdbd33..34d4313e24cb7 100644 --- a/src/stream/src/from_proto/expand.rs +++ b/src/stream/src/from_proto/expand.rs @@ -26,7 +26,7 @@ impl ExecutorBuilder for ExpandExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); let column_subsets = node .column_subsets @@ -39,6 +39,6 @@ impl ExecutorBuilder for ExpandExecutorBuilder { .collect_vec() }) .collect_vec(); - Ok(ExpandExecutor::new(params.info, input, column_subsets).boxed()) + Ok((params.info, ExpandExecutor::new(input, column_subsets)).into()) } } diff --git a/src/stream/src/from_proto/filter.rs b/src/stream/src/from_proto/filter.rs index a82b9cc65dea7..a4efb953064b8 100644 --- a/src/stream/src/from_proto/filter.rs +++ b/src/stream/src/from_proto/filter.rs @@ -27,11 +27,12 @@ impl ExecutorBuilder for FilterExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); let search_condition = build_non_strict_from_prost(node.get_search_condition()?, params.eval_error_report)?; - Ok(FilterExecutor::new(params.actor_context, params.info, input, search_condition).boxed()) + let exec = FilterExecutor::new(params.actor_context, input, search_condition); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/group_top_n.rs b/src/stream/src/from_proto/group_top_n.rs index a90e13d3f496b..51512cb9d94d3 100644 --- a/src/stream/src/from_proto/group_top_n.rs +++ b/src/stream/src/from_proto/group_top_n.rs @@ -14,6 +14,7 @@ use std::sync::Arc; +use risingwave_common::catalog::Schema; use risingwave_common::hash::{HashKey, HashKeyDispatcher}; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::ColumnOrder; @@ -21,7 +22,7 @@ use risingwave_pb::stream_plan::GroupTopNNode; use super::*; use crate::common::table::state_table::StateTable; -use crate::executor::{ActorContextRef, AppendOnlyGroupTopNExecutor, GroupTopNExecutor}; +use crate::executor::{ActorContextRef, AppendOnlyGroupTopNExecutor, Executor, GroupTopNExecutor}; use crate::task::AtomicU64Ref; pub struct GroupTopNExecutorBuilder<const APPEND_ONLY: bool>; @@ -33,7 +34,7 @@ impl<const APPEND_ONLY: bool> ExecutorBuilder for GroupTopNExecutorBuilder<APPEN params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let group_by: Vec<usize> = node .get_group_key() .iter() @@ -61,7 +62,7 @@ impl<const APPEND_ONLY: bool> ExecutorBuilder for GroupTopNExecutorBuilder<APPEN let args = GroupTopNExecutorDispatcherArgs { input, ctx: params.actor_context, - info: params.info, + schema: params.info.schema.clone(), storage_key, offset_and_limit: (node.offset as usize, node.limit as usize), order_by, @@ -73,14 +74,14 @@ impl<const APPEND_ONLY: bool> ExecutorBuilder for GroupTopNExecutorBuilder<APPEN with_ties: node.with_ties, append_only: APPEND_ONLY, }; - args.dispatch() + Ok((params.info, args.dispatch()?).into()) } } struct GroupTopNExecutorDispatcherArgs<S: StateStore> { - input: BoxedExecutor, + input: Executor, ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, storage_key: Vec<ColumnOrder>, offset_and_limit: (usize, usize), order_by: Vec<ColumnOrder>, @@ -94,7 +95,7 @@ struct GroupTopNExecutorDispatcherArgs<S: StateStore> { } impl<S: StateStore> HashKeyDispatcher for GroupTopNExecutorDispatcherArgs<S> { - type Output = StreamResult<BoxedExecutor>; + type Output = StreamResult<Box<dyn Execute>>; fn dispatch_impl<K: HashKey>(self) -> Self::Output { macro_rules! build { @@ -102,7 +103,7 @@ impl<S: StateStore> HashKeyDispatcher for GroupTopNExecutorDispatcherArgs<S> { Ok($excutor::<K, S, $with_ties>::new( self.input, self.ctx, - self.info, + self.schema, self.storage_key, self.offset_and_limit, self.order_by, diff --git a/src/stream/src/from_proto/hash_agg.rs b/src/stream/src/from_proto/hash_agg.rs index 52390b6806d0e..3331640519617 100644 --- a/src/stream/src/from_proto/hash_agg.rs +++ b/src/stream/src/from_proto/hash_agg.rs @@ -35,7 +35,7 @@ pub struct HashAggExecutorDispatcherArgs<S: StateStore> { } impl<S: StateStore> HashKeyDispatcher for HashAggExecutorDispatcherArgs<S> { - type Output = StreamResult<BoxedExecutor>; + type Output = StreamResult<Box<dyn Execute>>; fn dispatch_impl<K: HashKey>(self) -> Self::Output { Ok(HashAggExecutor::<K, S>::new(self.args)?.boxed()) @@ -55,7 +55,7 @@ impl ExecutorBuilder for HashAggExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let group_key_indices = node .get_group_key() .iter() @@ -93,13 +93,13 @@ impl ExecutorBuilder for HashAggExecutorBuilder { build_distinct_dedup_table_from_proto(node.get_distinct_dedup_tables(), store, vnodes) .await; - HashAggExecutorDispatcherArgs { + let exec = HashAggExecutorDispatcherArgs { args: AggExecutorArgs { version: node.version(), input, actor_ctx: params.actor_context, - info: params.info, + info: params.info.clone(), extreme_cache_size: params.env.config().developer.unsafe_extreme_cache_size, @@ -122,6 +122,7 @@ impl ExecutorBuilder for HashAggExecutorBuilder { }, group_key_types, } - .dispatch() + .dispatch()?; + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/hash_join.rs b/src/stream/src/from_proto/hash_join.rs index 0c50f26f941b6..ad69fd1ecd5ac 100644 --- a/src/stream/src/from_proto/hash_join.rs +++ b/src/stream/src/from_proto/hash_join.rs @@ -27,7 +27,7 @@ use super::*; use crate::common::table::state_table::StateTable; use crate::executor::hash_join::*; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{ActorContextRef, JoinType}; +use crate::executor::{ActorContextRef, Executor, JoinType}; use crate::task::AtomicU64Ref; pub struct HashJoinExecutorBuilder; @@ -39,7 +39,7 @@ impl ExecutorBuilder for HashJoinExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let is_append_only = node.is_append_only; let vnodes = Arc::new(params.vnode_bitmap.expect("vnodes not set for hash join")); @@ -136,7 +136,7 @@ impl ExecutorBuilder for HashJoinExecutorBuilder { let args = HashJoinExecutorDispatcherArgs { ctx: params.actor_context, - info: params.info, + info: params.info.clone(), source_l, source_r, params_l, @@ -157,15 +157,16 @@ impl ExecutorBuilder for HashJoinExecutorBuilder { chunk_size: params.env.config().developer.chunk_size, }; - args.dispatch() + let exec = args.dispatch()?; + Ok((params.info, exec).into()) } } struct HashJoinExecutorDispatcherArgs<S: StateStore> { ctx: ActorContextRef, info: ExecutorInfo, - source_l: Box<dyn Executor>, - source_r: Box<dyn Executor>, + source_l: Executor, + source_r: Executor, params_l: JoinParams, params_r: JoinParams, null_safe: Vec<bool>, @@ -185,34 +186,33 @@ struct HashJoinExecutorDispatcherArgs<S: StateStore> { } impl<S: StateStore> HashKeyDispatcher for HashJoinExecutorDispatcherArgs<S> { - type Output = StreamResult<BoxedExecutor>; + type Output = StreamResult<Box<dyn Execute>>; fn dispatch_impl<K: HashKey>(self) -> Self::Output { /// This macro helps to fill the const generic type parameter. macro_rules! build { ($join_type:ident) => { - Ok(Box::new( - HashJoinExecutor::<K, S, { JoinType::$join_type }>::new( - self.ctx, - self.info, - self.source_l, - self.source_r, - self.params_l, - self.params_r, - self.null_safe, - self.output_indices, - self.cond, - self.inequality_pairs, - self.state_table_l, - self.degree_state_table_l, - self.state_table_r, - self.degree_state_table_r, - self.lru_manager, - self.is_append_only, - self.metrics, - self.chunk_size, - ), - )) + Ok(HashJoinExecutor::<K, S, { JoinType::$join_type }>::new( + self.ctx, + self.info, + self.source_l, + self.source_r, + self.params_l, + self.params_r, + self.null_safe, + self.output_indices, + self.cond, + self.inequality_pairs, + self.state_table_l, + self.degree_state_table_l, + self.state_table_r, + self.degree_state_table_r, + self.lru_manager, + self.is_append_only, + self.metrics, + self.chunk_size, + ) + .boxed()) }; } match self.join_type_proto { diff --git a/src/stream/src/from_proto/hop_window.rs b/src/stream/src/from_proto/hop_window.rs index a56cbc2d8c976..2598ae927608e 100644 --- a/src/stream/src/from_proto/hop_window.rs +++ b/src/stream/src/from_proto/hop_window.rs @@ -27,16 +27,8 @@ impl ExecutorBuilder for HopWindowExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { - let ExecutorParams { - actor_context, - info, - input, - env, - .. - } = params; - - let input = input.into_iter().next().unwrap(); + ) -> StreamResult<Executor> { + let input = params.input.into_iter().next().unwrap(); // TODO: reuse the schema derivation with frontend. let output_indices = node .get_output_indices() @@ -59,11 +51,10 @@ impl ExecutorBuilder for HopWindowExecutorBuilder { let window_slide = node.get_window_slide()?.into(); let window_size = node.get_window_size()?.into(); - let chunk_size = env.config().developer.chunk_size; + let chunk_size = params.env.config().developer.chunk_size; - Ok(HopWindowExecutor::new( - actor_context, - info, + let exec = HopWindowExecutor::new( + params.actor_context, input, time_col, window_slide, @@ -72,7 +63,7 @@ impl ExecutorBuilder for HopWindowExecutorBuilder { window_end_exprs, output_indices, chunk_size, - ) - .boxed()) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/lookup.rs b/src/stream/src/from_proto/lookup.rs index 1f873f31127c5..dc7f7e3c49dfe 100644 --- a/src/stream/src/from_proto/lookup.rs +++ b/src/stream/src/from_proto/lookup.rs @@ -30,7 +30,7 @@ impl ExecutorBuilder for LookupExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let lookup = node; let [stream, arrangement]: [_; 2] = params.input.try_into().unwrap(); @@ -69,9 +69,9 @@ impl ExecutorBuilder for LookupExecutorBuilder { table_desc, ); - Ok(Box::new(LookupExecutor::new(LookupExecutorParams { + let exec = LookupExecutor::new(LookupExecutorParams { ctx: params.actor_context, - info: params.info, + info: params.info.clone(), arrangement, stream, arrangement_col_descs, @@ -83,6 +83,7 @@ impl ExecutorBuilder for LookupExecutorBuilder { storage_table, watermark_epoch: params.watermark_epoch, chunk_size: params.env.config().developer.chunk_size, - }))) + }); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/lookup_union.rs b/src/stream/src/from_proto/lookup_union.rs index e9cc0ed33311a..93e847609cae2 100644 --- a/src/stream/src/from_proto/lookup_union.rs +++ b/src/stream/src/from_proto/lookup_union.rs @@ -26,7 +26,8 @@ impl ExecutorBuilder for LookupUnionExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { - Ok(LookupUnionExecutor::new(params.info, params.input, node.order.clone()).boxed()) + ) -> StreamResult<Executor> { + let exec = LookupUnionExecutor::new(params.input, node.order.clone()); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/merge.rs b/src/stream/src/from_proto/merge.rs index aa0a6a9d2bb92..eded1f59d294e 100644 --- a/src/stream/src/from_proto/merge.rs +++ b/src/stream/src/from_proto/merge.rs @@ -27,7 +27,7 @@ impl ExecutorBuilder for MergeExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let upstreams = node.get_upstream_actor_id(); let upstream_fragment_id = node.get_upstream_fragment_id(); @@ -56,10 +56,9 @@ impl ExecutorBuilder for MergeExecutorBuilder { DispatcherType::NoShuffle => true, }; - if always_single_input { - Ok(ReceiverExecutor::new( + let exec = if always_single_input { + ReceiverExecutor::new( params.actor_context, - params.info, params.fragment_id, upstream_fragment_id, inputs.into_iter().exactly_one().unwrap(), @@ -67,11 +66,10 @@ impl ExecutorBuilder for MergeExecutorBuilder { params.operator_id, params.executor_stats.clone(), ) - .boxed()) + .boxed() } else { - Ok(MergeExecutor::new( + MergeExecutor::new( params.actor_context, - params.info, params.fragment_id, upstream_fragment_id, inputs, @@ -79,7 +77,8 @@ impl ExecutorBuilder for MergeExecutorBuilder { params.operator_id, params.executor_stats.clone(), ) - .boxed()) - } + .boxed() + }; + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/mod.rs b/src/stream/src/from_proto/mod.rs index 46ec1a6cc8e59..c00a9fd844df7 100644 --- a/src/stream/src/from_proto/mod.rs +++ b/src/stream/src/from_proto/mod.rs @@ -93,7 +93,7 @@ use self::top_n::*; use self::union::*; use self::watermark_filter::WatermarkFilterBuilder; use crate::error::StreamResult; -use crate::executor::{BoxedExecutor, Executor, ExecutorInfo}; +use crate::executor::{Execute, Executor, ExecutorInfo}; use crate::from_proto::subscription::SubscriptionExecutorBuilder; use crate::from_proto::values::ValuesExecutorBuilder; use crate::task::ExecutorParams; @@ -101,12 +101,12 @@ use crate::task::ExecutorParams; trait ExecutorBuilder { type Node; - /// Create a [`BoxedExecutor`] from [`StreamNode`]. + /// Create an [`Executor`] from [`StreamNode`]. fn new_boxed_executor( params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> impl std::future::Future<Output = StreamResult<BoxedExecutor>> + Send; + ) -> impl std::future::Future<Output = StreamResult<Executor>> + Send; } macro_rules! build_executor { @@ -127,7 +127,7 @@ pub async fn create_executor( params: ExecutorParams, node: &StreamNode, store: impl StateStore, -) -> StreamResult<BoxedExecutor> { +) -> StreamResult<Executor> { build_executor! { params, node, diff --git a/src/stream/src/from_proto/mview.rs b/src/stream/src/from_proto/mview.rs index ecc54b20d8535..80aab576435ca 100644 --- a/src/stream/src/from_proto/mview.rs +++ b/src/stream/src/from_proto/mview.rs @@ -32,7 +32,7 @@ impl ExecutorBuilder for MaterializeExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); let order_key = node @@ -51,7 +51,7 @@ impl ExecutorBuilder for MaterializeExecutorBuilder { ($SD:ident) => { MaterializeExecutor::<_, $SD>::new( input, - params.info, + params.info.schema.clone(), store, order_key, params.actor_context, @@ -66,13 +66,13 @@ impl ExecutorBuilder for MaterializeExecutorBuilder { }; } - let executor = if versioned { + let exec = if versioned { new_executor!(ColumnAwareSerde) } else { new_executor!(BasicSerde) }; - Ok(executor) + Ok((params.info, exec).into()) } } @@ -85,7 +85,7 @@ impl ExecutorBuilder for ArrangeExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); let keys = node @@ -102,9 +102,9 @@ impl ExecutorBuilder for ArrangeExecutorBuilder { let vnodes = params.vnode_bitmap.map(Arc::new); let conflict_behavior = ConflictBehavior::from_protobuf(&table.handle_pk_conflict_behavior()); - let executor = MaterializeExecutor::<_, BasicSerde>::new( + let exec = MaterializeExecutor::<_, BasicSerde>::new( input, - params.info, + params.info.schema.clone(), store, keys, params.actor_context, @@ -116,6 +116,6 @@ impl ExecutorBuilder for ArrangeExecutorBuilder { ) .await; - Ok(executor.boxed()) + Ok((params.info, exec.boxed()).into()) } } diff --git a/src/stream/src/from_proto/no_op.rs b/src/stream/src/from_proto/no_op.rs index 606b584d9109c..31494d771b903 100644 --- a/src/stream/src/from_proto/no_op.rs +++ b/src/stream/src/from_proto/no_op.rs @@ -17,7 +17,7 @@ use risingwave_storage::StateStore; use super::ExecutorBuilder; use crate::error::StreamResult; -use crate::executor::{BoxedExecutor, Executor, NoOpExecutor}; +use crate::executor::{Executor, NoOpExecutor}; use crate::task::ExecutorParams; pub struct NoOpExecutorBuilder; @@ -29,8 +29,8 @@ impl ExecutorBuilder for NoOpExecutorBuilder { params: ExecutorParams, _node: &NoOpNode, _store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); - Ok(NoOpExecutor::new(params.actor_context, params.info, input).boxed()) + Ok((params.info, NoOpExecutor::new(params.actor_context, input)).into()) } } diff --git a/src/stream/src/from_proto/now.rs b/src/stream/src/from_proto/now.rs index f622cdef1343e..06de0cacb7197 100644 --- a/src/stream/src/from_proto/now.rs +++ b/src/stream/src/from_proto/now.rs @@ -19,7 +19,7 @@ use tokio::sync::mpsc::unbounded_channel; use super::ExecutorBuilder; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; -use crate::executor::{BoxedExecutor, NowExecutor}; +use crate::executor::{Executor, NowExecutor}; use crate::task::ExecutorParams; pub struct NowExecutorBuilder; @@ -31,7 +31,7 @@ impl ExecutorBuilder for NowExecutorBuilder { params: ExecutorParams, node: &NowNode, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let (sender, barrier_receiver) = unbounded_channel(); params .local_barrier_manager @@ -40,10 +40,11 @@ impl ExecutorBuilder for NowExecutorBuilder { let state_table = StateTable::from_table_catalog(node.get_state_table()?, store, None).await; - Ok(Box::new(NowExecutor::new( - params.info, + let exec = NowExecutor::new( + params.info.schema.data_types(), barrier_receiver, state_table, - ))) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/over_window.rs b/src/stream/src/from_proto/over_window.rs index d2fc2893b6135..f7ca73c183a81 100644 --- a/src/stream/src/from_proto/over_window.rs +++ b/src/stream/src/from_proto/over_window.rs @@ -23,7 +23,7 @@ use risingwave_storage::StateStore; use super::ExecutorBuilder; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; -use crate::executor::{BoxedExecutor, Executor, OverWindowExecutor, OverWindowExecutorArgs}; +use crate::executor::{Executor, OverWindowExecutor, OverWindowExecutorArgs}; use crate::task::ExecutorParams; pub struct OverWindowExecutorBuilder; @@ -35,7 +35,7 @@ impl ExecutorBuilder for OverWindowExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); let calls: Vec<_> = node .get_calls() @@ -60,12 +60,12 @@ impl ExecutorBuilder for OverWindowExecutorBuilder { )); let state_table = StateTable::from_table_catalog(node.get_state_table()?, store, vnodes).await; - Ok(OverWindowExecutor::new(OverWindowExecutorArgs { + let exec = OverWindowExecutor::new(OverWindowExecutorArgs { actor_ctx: params.actor_context, - info: params.info, input, + schema: params.info.schema.clone(), calls, partition_key_indices, order_key_indices, @@ -79,7 +79,7 @@ impl ExecutorBuilder for OverWindowExecutorBuilder { cache_policy: OverWindowCachePolicy::from_protobuf( node.get_cache_policy().unwrap_or_default(), ), - }) - .boxed()) + }); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/project.rs b/src/stream/src/from_proto/project.rs index 6fc9ada39aed8..d7f96c4dffcbf 100644 --- a/src/stream/src/from_proto/project.rs +++ b/src/stream/src/from_proto/project.rs @@ -30,7 +30,7 @@ impl ExecutorBuilder for ProjectExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); let project_exprs: Vec<_> = node .get_select_list() @@ -60,15 +60,14 @@ impl ExecutorBuilder for ProjectExecutorBuilder { ) }); let materialize_selectivity_threshold = if extremely_light { 0.0 } else { 0.5 }; - Ok(ProjectExecutor::new( + let exec = ProjectExecutor::new( params.actor_context, - params.info, input, project_exprs, watermark_derivations, nondecreasing_expr_indices, materialize_selectivity_threshold, - ) - .boxed()) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/project_set.rs b/src/stream/src/from_proto/project_set.rs index b98e9e467ccf2..c2338394b33ef 100644 --- a/src/stream/src/from_proto/project_set.rs +++ b/src/stream/src/from_proto/project_set.rs @@ -29,7 +29,7 @@ impl ExecutorBuilder for ProjectSetExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); let select_list: Vec<_> = node .get_select_list() @@ -55,15 +55,14 @@ impl ExecutorBuilder for ProjectSetExecutorBuilder { .collect(); let chunk_size = params.env.config().developer.chunk_size; - Ok(ProjectSetExecutor::new( + let exec = ProjectSetExecutor::new( params.actor_context, - params.info, input, select_list, chunk_size, watermark_derivations, nondecreasing_expr_indices, - ) - .boxed()) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/row_id_gen.rs b/src/stream/src/from_proto/row_id_gen.rs index 1333a99aebfa5..ab87566547ac6 100644 --- a/src/stream/src/from_proto/row_id_gen.rs +++ b/src/stream/src/from_proto/row_id_gen.rs @@ -18,7 +18,7 @@ use risingwave_storage::StateStore; use super::ExecutorBuilder; use crate::error::StreamResult; use crate::executor::row_id_gen::RowIdGenExecutor; -use crate::executor::BoxedExecutor; +use crate::executor::Executor; use crate::task::ExecutorParams; pub struct RowIdGenExecutorBuilder; @@ -30,19 +30,18 @@ impl ExecutorBuilder for RowIdGenExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [upstream]: [_; 1] = params.input.try_into().unwrap(); tracing::debug!("row id gen executor: {:?}", params.vnode_bitmap); let vnodes = params .vnode_bitmap .expect("vnodes not set for row id gen executor"); - let executor = RowIdGenExecutor::new( + let exec = RowIdGenExecutor::new( params.actor_context, - params.info, upstream, node.row_id_index as _, vnodes, ); - Ok(Box::new(executor)) + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/simple_agg.rs b/src/stream/src/from_proto/simple_agg.rs index 209a08daf6a22..16809edb8bcaf 100644 --- a/src/stream/src/from_proto/simple_agg.rs +++ b/src/stream/src/from_proto/simple_agg.rs @@ -34,7 +34,7 @@ impl ExecutorBuilder for SimpleAggExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); let agg_calls: Vec<AggCall> = node .get_agg_calls() @@ -55,12 +55,12 @@ impl ExecutorBuilder for SimpleAggExecutorBuilder { build_distinct_dedup_table_from_proto(node.get_distinct_dedup_tables(), store, None) .await; - Ok(SimpleAggExecutor::new(AggExecutorArgs { + let exec = SimpleAggExecutor::new(AggExecutorArgs { version: node.version(), input, actor_ctx: params.actor_context, - info: params.info, + info: params.info.clone(), extreme_cache_size: params.env.config().developer.unsafe_extreme_cache_size, @@ -71,7 +71,8 @@ impl ExecutorBuilder for SimpleAggExecutorBuilder { distinct_dedup_tables, watermark_epoch: params.watermark_epoch, extra: SimpleAggExecutorExtraArgs {}, - })? - .boxed()) + })?; + + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs index 87d63cd80f599..884925d4a35ab 100644 --- a/src/stream/src/from_proto/sink.rs +++ b/src/stream/src/from_proto/sink.rs @@ -103,7 +103,7 @@ impl ExecutorBuilder for SinkExecutorBuilder { params: ExecutorParams, node: &Self::Node, state_store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input_executor]: [_; 1] = params.input.try_into().unwrap(); let sink_desc = node.sink_desc.as_ref().unwrap(); @@ -189,23 +189,22 @@ impl ExecutorBuilder for SinkExecutorBuilder { connector, sink_id.sink_id, params.executor_id ); - match node.log_store_type() { + let exec = match node.log_store_type() { // Default value is the normal in memory log store to be backward compatible with the // previously unset value SinkLogStoreType::InMemoryLogStore | SinkLogStoreType::Unspecified => { let factory = BoundedInMemLogStoreFactory::new(1); - Ok(Box::new( - SinkExecutor::new( - params.actor_context, - params.info, - input_executor, - sink_write_param, - sink_param, - columns, - factory, - ) - .await?, - )) + SinkExecutor::new( + params.actor_context, + params.info.clone(), + input_executor, + sink_write_param, + sink_param, + columns, + factory, + ) + .await? + .boxed() } SinkLogStoreType::KvLogStore => { let metrics = KvLogStoreMetrics::new( @@ -230,19 +229,20 @@ impl ExecutorBuilder for SinkExecutorBuilder { pk_info, ); - Ok(Box::new( - SinkExecutor::new( - params.actor_context, - params.info, - input_executor, - sink_write_param, - sink_param, - columns, - factory, - ) - .await?, - )) + SinkExecutor::new( + params.actor_context, + params.info.clone(), + input_executor, + sink_write_param, + sink_param, + columns, + factory, + ) + .await? + .boxed() } - } + }; + + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/sort.rs b/src/stream/src/from_proto/sort.rs index dcb59f77f9549..e901d66e5d8ff 100644 --- a/src/stream/src/from_proto/sort.rs +++ b/src/stream/src/from_proto/sort.rs @@ -29,18 +29,19 @@ impl ExecutorBuilder for SortExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); let vnodes = Arc::new(params.vnode_bitmap.expect("vnodes not set for sort")); let state_table = StateTable::from_table_catalog(node.get_state_table()?, store, Some(vnodes)).await; - Ok(Box::new(SortExecutor::new(SortExecutorArgs { + let exec = SortExecutor::new(SortExecutorArgs { actor_ctx: params.actor_context, - info: params.info, + schema: params.info.schema.clone(), input, buffer_table: state_table, chunk_size: params.env.config().developer.chunk_size, sort_column_index: node.sort_column_index as _, - }))) + }); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs index 2cedce5a8cd0a..355f7d874dec8 100644 --- a/src/stream/src/from_proto/source/fs_fetch.rs +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -25,7 +25,7 @@ use risingwave_storage::StateStore; use crate::error::StreamResult; use crate::executor::{ - BoxedExecutor, Executor, FlowControlExecutor, FsFetchExecutor, SourceStateTableHandler, + Execute, Executor, FlowControlExecutor, FsFetchExecutor, SourceStateTableHandler, StreamSourceCore, }; use crate::from_proto::ExecutorBuilder; @@ -40,7 +40,7 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [upstream]: [_; 1] = params.input.try_into().unwrap(); let source = node.node_inner.as_ref().unwrap(); @@ -89,11 +89,10 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { state_table_handler, ); - let executor = match properties { + let exec = match properties { risingwave_connector::source::ConnectorProperties::Gcs(_) => { FsFetchExecutor::<_, OpendalGcs>::new( params.actor_context.clone(), - params.info, stream_source_core, upstream, source_ctrl_opts, @@ -104,7 +103,6 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { risingwave_connector::source::ConnectorProperties::OpendalS3(_) => { FsFetchExecutor::<_, OpendalS3>::new( params.actor_context.clone(), - params.info, stream_source_core, upstream, source_ctrl_opts, @@ -115,7 +113,6 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { risingwave_connector::source::ConnectorProperties::PosixFs(_) => { FsFetchExecutor::<_, OpendalPosixFs>::new( params.actor_context.clone(), - params.info, stream_source_core, upstream, source_ctrl_opts, @@ -125,7 +122,11 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { } _ => unreachable!(), }; + let mut info = params.info.clone(); + info.identity = format!("{} (flow controlled)", info.identity); + let rate_limit = source.rate_limit.map(|x| x as _); - Ok(FlowControlExecutor::new(executor, params.actor_context, rate_limit).boxed()) + let exec = FlowControlExecutor::new((info, exec).into(), params.actor_context, rate_limit); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index 8ce6b88b0196b..7870bab10151b 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -46,7 +46,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let (sender, barrier_receiver) = unbounded_channel(); params .local_barrier_manager @@ -54,7 +54,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { let system_params = params.env.system_params_manager_ref().get_params(); if let Some(source) = &node.source_inner { - let executor = { + let exec = { let source_id = TableId::new(source.source_id); let source_name = source.source_name.clone(); let mut source_info = source.get_info()?.clone(); @@ -205,7 +205,6 @@ impl ExecutorBuilder for SourceExecutorBuilder { #[expect(deprecated)] crate::executor::FsSourceExecutor::new( params.actor_context.clone(), - params.info, stream_source_core, params.executor_stats, barrier_receiver, @@ -216,7 +215,6 @@ impl ExecutorBuilder for SourceExecutorBuilder { } else if is_fs_v2_connector { FsListExecutor::new( params.actor_context.clone(), - params.info.clone(), Some(stream_source_core), params.executor_stats.clone(), barrier_receiver, @@ -228,7 +226,6 @@ impl ExecutorBuilder for SourceExecutorBuilder { } else { SourceExecutor::new( params.actor_context.clone(), - params.info.clone(), Some(stream_source_core), params.executor_stats.clone(), barrier_receiver, @@ -239,14 +236,18 @@ impl ExecutorBuilder for SourceExecutorBuilder { .boxed() } }; + let mut info = params.info.clone(); + info.identity = format!("{} (flow controlled)", info.identity); + let rate_limit = source.rate_limit.map(|x| x as _); - Ok(FlowControlExecutor::new(executor, params.actor_context, rate_limit).boxed()) + let exec = + FlowControlExecutor::new((info, exec).into(), params.actor_context, rate_limit); + Ok((params.info, exec).into()) } else { // If there is no external stream source, then no data should be persisted. We pass a // `PanicStateStore` type here for indication. - Ok(SourceExecutor::<PanicStateStore>::new( + let exec = SourceExecutor::<PanicStateStore>::new( params.actor_context, - params.info, None, params.executor_stats, barrier_receiver, @@ -254,8 +255,8 @@ impl ExecutorBuilder for SourceExecutorBuilder { // we don't expect any data in, so no need to set chunk_sizes SourceCtrlOpts::default(), params.env.connector_params(), - ) - .boxed()) + ); + Ok((params.info, exec).into()) } } } diff --git a/src/stream/src/from_proto/stateless_simple_agg.rs b/src/stream/src/from_proto/stateless_simple_agg.rs index 87617b42f83e4..2ce5e612737e8 100644 --- a/src/stream/src/from_proto/stateless_simple_agg.rs +++ b/src/stream/src/from_proto/stateless_simple_agg.rs @@ -27,7 +27,7 @@ impl ExecutorBuilder for StatelessSimpleAggExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); let agg_calls: Vec<AggCall> = node .get_agg_calls() @@ -35,9 +35,12 @@ impl ExecutorBuilder for StatelessSimpleAggExecutorBuilder { .map(AggCall::from_protobuf) .try_collect()?; - Ok( - StatelessSimpleAggExecutor::new(params.actor_context, params.info, input, agg_calls)? - .boxed(), - ) + let exec = StatelessSimpleAggExecutor::new( + params.actor_context, + input, + params.info.schema.clone(), + agg_calls, + )?; + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/stream_cdc_scan.rs b/src/stream/src/from_proto/stream_cdc_scan.rs index 66e08c2bb4695..2736fdd712cb7 100644 --- a/src/stream/src/from_proto/stream_cdc_scan.rs +++ b/src/stream/src/from_proto/stream_cdc_scan.rs @@ -23,7 +23,7 @@ use risingwave_pb::stream_plan::StreamCdcScanNode; use super::*; use crate::common::table::state_table::StateTable; -use crate::executor::{CdcBackfillExecutor, ExternalStorageTable, FlowControlExecutor}; +use crate::executor::{CdcBackfillExecutor, Executor, ExternalStorageTable, FlowControlExecutor}; pub struct StreamCdcScanExecutorBuilder; @@ -34,7 +34,7 @@ impl ExecutorBuilder for StreamCdcScanExecutorBuilder { params: ExecutorParams, node: &Self::Node, state_store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [upstream]: [_; 1] = params.input.try_into().unwrap(); let output_indices = node @@ -95,9 +95,8 @@ impl ExecutorBuilder for StreamCdcScanExecutorBuilder { .map(|x| std::cmp::min(x as usize, chunk_size)) .unwrap_or(chunk_size); - let executor = CdcBackfillExecutor::new( + let exec = CdcBackfillExecutor::new( params.actor_context.clone(), - params.info, external_table, upstream, output_indices, @@ -106,14 +105,15 @@ impl ExecutorBuilder for StreamCdcScanExecutorBuilder { state_table, backfill_chunk_size, disable_backfill, - ) - .boxed(); + ); + let mut info = params.info.clone(); + info.identity = format!("{} (flow controlled)", info.identity); - Ok(FlowControlExecutor::new( - executor, + let exec = FlowControlExecutor::new( + (info, exec).into(), params.actor_context, node.rate_limit.map(|x| x as _), - ) - .boxed()) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/stream_scan.rs b/src/stream/src/from_proto/stream_scan.rs index c3f15802afc8d..5f0ae484e04c8 100644 --- a/src/stream/src/from_proto/stream_scan.rs +++ b/src/stream/src/from_proto/stream_scan.rs @@ -37,7 +37,7 @@ impl ExecutorBuilder for StreamScanExecutorBuilder { params: ExecutorParams, node: &Self::Node, state_store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [upstream, snapshot]: [_; 2] = params.input.try_into().unwrap(); // For reporting the progress. let progress = params @@ -50,13 +50,13 @@ impl ExecutorBuilder for StreamScanExecutorBuilder { .map(|&i| i as usize) .collect_vec(); - let executor = match node.stream_scan_type() { + let exec = match node.stream_scan_type() { StreamScanType::Chain | StreamScanType::UpstreamOnly => { let upstream_only = matches!(node.stream_scan_type(), StreamScanType::UpstreamOnly); - ChainExecutor::new(params.info, snapshot, upstream, progress, upstream_only).boxed() + ChainExecutor::new(snapshot, upstream, progress, upstream_only).boxed() } StreamScanType::Rearrange => { - RearrangedChainExecutor::new(params.info, snapshot, upstream, progress).boxed() + RearrangedChainExecutor::new(snapshot, upstream, progress).boxed() } StreamScanType::Backfill => { @@ -83,7 +83,6 @@ impl ExecutorBuilder for StreamScanExecutorBuilder { StorageTable::new_partial(state_store.clone(), column_ids, vnodes, table_desc); BackfillExecutor::new( - params.info, upstream_table, upstream, state_table, @@ -126,7 +125,6 @@ impl ExecutorBuilder for StreamScanExecutorBuilder { ) .await; ArrangementBackfillExecutor::<_, $SD>::new( - params.info, upstream_table, upstream, state_table, @@ -147,11 +145,14 @@ impl ExecutorBuilder for StreamScanExecutorBuilder { } StreamScanType::Unspecified => unreachable!(), }; - Ok(FlowControlExecutor::new( - executor, + let mut info = params.info.clone(); + info.identity = format!("{} (flow controlled)", info.identity); + + let exec = FlowControlExecutor::new( + (info, exec).into(), params.actor_context, node.rate_limit.map(|x| x as _), - ) - .boxed()) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/subscription.rs b/src/stream/src/from_proto/subscription.rs index 4d5e6eb6855b1..ff0c1f8c1d084 100644 --- a/src/stream/src/from_proto/subscription.rs +++ b/src/stream/src/from_proto/subscription.rs @@ -21,7 +21,7 @@ use crate::common::log_store_impl::kv_log_store::serde::LogStoreRowSerde; use crate::common::log_store_impl::kv_log_store::KV_LOG_STORE_V2_INFO; use crate::common::log_store_impl::subscription_log_store::SubscriptionLogStoreWriter; use crate::error::StreamResult; -use crate::executor::{BoxedExecutor, SubscriptionExecutor}; +use crate::executor::{Executor, SubscriptionExecutor}; pub struct SubscriptionExecutorBuilder; @@ -32,7 +32,7 @@ impl ExecutorBuilder for SubscriptionExecutorBuilder { params: crate::task::ExecutorParams, node: &Self::Node, state_store: impl risingwave_storage::StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); let table_id = TableId::new(node.log_store_table.as_ref().unwrap().id); let local_state_store = state_store @@ -61,15 +61,13 @@ impl ExecutorBuilder for SubscriptionExecutorBuilder { let log_store_identity = format!("subscription-executor[{}]", params.executor_id); let log_store = SubscriptionLogStoreWriter::new(table_id, local_state_store, serde, log_store_identity); - Ok(Box::new( - SubscriptionExecutor::new( - params.actor_context, - params.info, - input, - log_store, - node.retention_seconds, - ) - .await?, - )) + let exec = SubscriptionExecutor::new( + params.actor_context, + input, + log_store, + node.retention_seconds, + ) + .await?; + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/temporal_join.rs b/src/stream/src/from_proto/temporal_join.rs index 7c91b8c1d261d..15badec97e5cc 100644 --- a/src/stream/src/from_proto/temporal_join.rs +++ b/src/stream/src/from_proto/temporal_join.rs @@ -35,7 +35,7 @@ impl ExecutorBuilder for TemporalJoinExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let table_desc: &StorageTableDesc = node.get_table_desc()?; let table = { let column_ids = table_desc @@ -101,7 +101,7 @@ impl ExecutorBuilder for TemporalJoinExecutorBuilder { let dispatcher_args = TemporalJoinExecutorDispatcherArgs { ctx: params.actor_context, - info: params.info, + info: params.info.clone(), left: source_l, right: source_r, right_table: table, @@ -119,15 +119,15 @@ impl ExecutorBuilder for TemporalJoinExecutorBuilder { join_key_data_types, }; - dispatcher_args.dispatch() + Ok((params.info, dispatcher_args.dispatch()?).into()) } } struct TemporalJoinExecutorDispatcherArgs<S: StateStore> { ctx: ActorContextRef, info: ExecutorInfo, - left: BoxedExecutor, - right: BoxedExecutor, + left: Executor, + right: Executor, right_table: StorageTable<S>, left_join_keys: Vec<usize>, right_join_keys: Vec<usize>, @@ -144,7 +144,7 @@ struct TemporalJoinExecutorDispatcherArgs<S: StateStore> { } impl<S: StateStore> HashKeyDispatcher for TemporalJoinExecutorDispatcherArgs<S> { - type Output = StreamResult<BoxedExecutor>; + type Output = StreamResult<Box<dyn Execute>>; fn dispatch_impl<K: HashKey>(self) -> Self::Output { /// This macro helps to fill the const generic type parameter. diff --git a/src/stream/src/from_proto/top_n.rs b/src/stream/src/from_proto/top_n.rs index 5c74f9f97e0eb..10071534e57df 100644 --- a/src/stream/src/from_proto/top_n.rs +++ b/src/stream/src/from_proto/top_n.rs @@ -30,7 +30,7 @@ impl<const APPEND_ONLY: bool> ExecutorBuilder for TopNExecutorBuilder<APPEND_ONL params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); let table = node.get_table()?; @@ -52,7 +52,7 @@ impl<const APPEND_ONLY: bool> ExecutorBuilder for TopNExecutorBuilder<APPEND_ONL Ok($excutor::<_, $with_ties>::new( input, params.actor_context, - params.info, + params.info.schema.clone(), storage_key, (node.offset as usize, node.limit as usize), order_by, @@ -62,11 +62,12 @@ impl<const APPEND_ONLY: bool> ExecutorBuilder for TopNExecutorBuilder<APPEND_ONL }; } - match (APPEND_ONLY, node.with_ties) { + let exec: StreamResult<Box<dyn Execute>> = match (APPEND_ONLY, node.with_ties) { (true, true) => build!(AppendOnlyTopNExecutor, true), (true, false) => build!(AppendOnlyTopNExecutor, false), (false, true) => build!(TopNExecutor, true), (false, false) => build!(TopNExecutor, false), - } + }; + Ok((params.info, exec?).into()) } } diff --git a/src/stream/src/from_proto/union.rs b/src/stream/src/from_proto/union.rs index e11ea8f3f2d21..1360f92c1c83b 100644 --- a/src/stream/src/from_proto/union.rs +++ b/src/stream/src/from_proto/union.rs @@ -26,7 +26,7 @@ impl ExecutorBuilder for UnionExecutorBuilder { params: ExecutorParams, _node: &Self::Node, _store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { - Ok(UnionExecutor::new(params.info, params.input).boxed()) + ) -> StreamResult<Executor> { + Ok((params.info, UnionExecutor::new(params.input)).into()) } } diff --git a/src/stream/src/from_proto/values.rs b/src/stream/src/from_proto/values.rs index b3ed75e4f0903..28e9ca086414b 100644 --- a/src/stream/src/from_proto/values.rs +++ b/src/stream/src/from_proto/values.rs @@ -20,7 +20,7 @@ use tokio::sync::mpsc::unbounded_channel; use super::ExecutorBuilder; use crate::error::StreamResult; -use crate::executor::{BoxedExecutor, ValuesExecutor}; +use crate::executor::{Executor, ValuesExecutor}; use crate::task::ExecutorParams; /// Build a `ValuesExecutor` for stream. As is a leaf, current workaround registers a `sender` for @@ -34,7 +34,7 @@ impl ExecutorBuilder for ValuesExecutorBuilder { params: ExecutorParams, node: &ValuesNode, _store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let (sender, barrier_receiver) = unbounded_channel(); params .local_barrier_manager @@ -55,12 +55,13 @@ impl ExecutorBuilder for ValuesExecutorBuilder { .collect_vec() }) .collect_vec(); - Ok(Box::new(ValuesExecutor::new( + let exec = ValuesExecutor::new( params.actor_context, - params.info, + params.info.schema.clone(), progress, rows, barrier_receiver, - ))) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/watermark_filter.rs b/src/stream/src/from_proto/watermark_filter.rs index cc3382bcc65fc..f01695e991487 100644 --- a/src/stream/src/from_proto/watermark_filter.rs +++ b/src/stream/src/from_proto/watermark_filter.rs @@ -34,7 +34,7 @@ impl ExecutorBuilder for WatermarkFilterBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { let [input]: [_; 1] = params.input.try_into().unwrap(); let watermark_descs = node.get_watermark_descs().clone(); let [watermark_desc]: [_; 1] = watermark_descs.try_into().unwrap(); @@ -63,15 +63,15 @@ impl ExecutorBuilder for WatermarkFilterBuilder { let table = StateTable::from_table_catalog_inconsistent_op(&table, store, Some(vnodes)).await; - Ok(WatermarkFilterExecutor::new( + let exec = WatermarkFilterExecutor::new( params.actor_context, - params.info, + ¶ms.info, input, watermark_expr, event_time_col_idx, table, global_watermark_table, - ) - .boxed()) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index 11eb9a44290cf..ad93575771ea2 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -44,7 +44,10 @@ use super::{unique_executor_id, unique_operator_id, BarrierCompleteResult}; use crate::error::StreamResult; use crate::executor::monitor::StreamingMetrics; use crate::executor::subtask::SubtaskHandle; -use crate::executor::*; +use crate::executor::{ + Actor, ActorContext, ActorContextRef, Barrier, DispatchExecutor, DispatcherImpl, Executor, + ExecutorInfo, WrapperExecutor, +}; use crate::from_proto::create_executor; use crate::task::barrier_manager::{LocalBarrierEvent, LocalBarrierWorker}; use crate::task::{ @@ -102,7 +105,7 @@ pub struct ExecutorParams { pub op_info: String, /// The input executor. - pub input: Vec<BoxedExecutor>, + pub input: Vec<Executor>, /// FragmentId of the actor pub fragment_id: FragmentId, @@ -351,7 +354,7 @@ impl StreamActorManager { /// Create dispatchers with downstream information registered before fn create_dispatcher( &self, - input: BoxedExecutor, + input: Executor, dispatchers: &[stream_plan::Dispatcher], actor_id: ActorId, fragment_id: FragmentId, @@ -384,7 +387,7 @@ impl StreamActorManager { vnode_bitmap: Option<Bitmap>, has_stateful: bool, subtasks: &mut Vec<SubtaskHandle>, - ) -> StreamResult<BoxedExecutor> { + ) -> StreamResult<Executor> { // The "stateful" here means that the executor may issue read operations to the state store // massively and continuously. Used to decide whether to apply the optimization of subtasks. fn is_stateful_executor(stream_node: &StreamNode) -> bool { @@ -435,21 +438,22 @@ impl StreamActorManager { let schema: Schema = node.fields.iter().map(Field::from).collect(); let identity = format!("{} {:X}", node.get_node_body().unwrap(), executor_id); + let info = ExecutorInfo { + schema, + pk_indices, + identity, + }; + let eval_error_report = ActorEvalErrorReport { actor_context: actor_context.clone(), - identity: identity.clone().into(), + identity: info.identity.clone().into(), }; // Build the executor with params. let executor_params = ExecutorParams { env: env.clone(), - info: ExecutorInfo { - schema: schema.clone(), - pk_indices: pk_indices.clone(), - identity: identity.clone(), - }, - + info: info.clone(), executor_id, operator_id, op_info, @@ -465,26 +469,14 @@ impl StreamActorManager { }; let executor = create_executor(executor_params, node, store).await?; - assert_eq!( - executor.pk_indices(), - &pk_indices, - "`pk_indices` of {} not consistent with what derived by optimizer", - executor.identity() - ); - assert_eq!( - executor.schema(), - &schema, - "`schema` of {} not consistent with what derived by optimizer", - executor.identity() - ); // Wrap the executor for debug purpose. - let executor = WrapperExecutor::new( + let wrapped = WrapperExecutor::new( executor, actor_context.clone(), env.config().developer.enable_executor_row_count, - ) - .boxed(); + ); + let executor = (info, wrapped).into(); // If there're multiple stateful executors in this actor, we will wrap it into a subtask. let executor = if has_stateful && is_stateful { @@ -510,7 +502,7 @@ impl StreamActorManager { env: StreamEnvironment, actor_context: &ActorContextRef, vnode_bitmap: Option<Bitmap>, - ) -> StreamResult<(BoxedExecutor, Vec<SubtaskHandle>)> { + ) -> StreamResult<(Executor, Vec<SubtaskHandle>)> { let mut subtasks = vec![]; let executor = dispatch_state_store!(env.state_store(), store, { diff --git a/src/stream/tests/integration_tests/eowc_over_window.rs b/src/stream/tests/integration_tests/eowc_over_window.rs index 6941d86f3a759..cc674e556ab5e 100644 --- a/src/stream/tests/integration_tests/eowc_over_window.rs +++ b/src/stream/tests/integration_tests/eowc_over_window.rs @@ -14,9 +14,7 @@ use risingwave_expr::aggregate::{AggArgs, AggKind}; use risingwave_expr::window_function::{Frame, FrameBound, WindowFuncCall, WindowFuncKind}; -use risingwave_stream::executor::{ - EowcOverWindowExecutor, EowcOverWindowExecutorArgs, ExecutorInfo, -}; +use risingwave_stream::executor::{EowcOverWindowExecutor, EowcOverWindowExecutorArgs}; use crate::prelude::*; @@ -54,7 +52,6 @@ async fn create_executor<S: StateStore>( }); Schema { fields } }; - let output_pk_indices = vec![2]; let state_table = StateTable::new_without_distribution_inconsistent_op( store, @@ -65,17 +62,14 @@ async fn create_executor<S: StateStore>( ) .await; - let (tx, source) = MockSource::channel(input_schema, input_pk_indices.clone()); + let (tx, source) = MockSource::channel(); + let source = source.into_executor(input_schema, input_pk_indices.clone()); let executor = EowcOverWindowExecutor::new(EowcOverWindowExecutorArgs { actor_ctx: ActorContext::for_test(123), - info: ExecutorInfo { - schema: output_schema, - pk_indices: output_pk_indices, - identity: "EowcOverWindowExecutor".to_string(), - }, - input: source.boxed(), + input: source, + schema: output_schema, calls, partition_key_indices, order_key_index, diff --git a/src/stream/tests/integration_tests/hash_agg.rs b/src/stream/tests/integration_tests/hash_agg.rs index bf82a2986bf7d..aecf0ad4d5251 100644 --- a/src/stream/tests/integration_tests/hash_agg.rs +++ b/src/stream/tests/integration_tests/hash_agg.rs @@ -38,10 +38,11 @@ async fn test_hash_agg_count_sum() { AggCall::from_pretty("(sum:int8 $2:int8)"), ]; - let (mut tx, source) = MockSource::channel(schema, PkIndices::new()); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, PkIndices::new()); let hash_agg = new_boxed_hash_agg_executor( store, - Box::new(source), + source, false, agg_calls, 0, @@ -116,10 +117,11 @@ async fn test_hash_agg_min() { AggCall::from_pretty("(min:int8 $1:int8)"), ]; - let (mut tx, source) = MockSource::channel(schema, vec![2]); // pk + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, vec![2]); let hash_agg = new_boxed_hash_agg_executor( store, - Box::new(source), + source, false, agg_calls, 0, @@ -191,10 +193,11 @@ async fn test_hash_agg_min_append_only() { AggCall::from_pretty("(min:int8 $1:int8)"), ]; - let (mut tx, source) = MockSource::channel(schema, vec![2]); // pk + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, vec![2]); let hash_agg = new_boxed_hash_agg_executor( store, - Box::new(source), + source, true, // is append only agg_calls, 0, @@ -266,10 +269,11 @@ async fn test_hash_agg_emit_on_window_close() { let agg_calls = vec![AggCall::from_pretty("(count:int8)")]; let create_executor = || async { - let (tx, source) = MockSource::channel(input_schema.clone(), PkIndices::new()); + let (tx, source) = MockSource::channel(); + let source = source.into_executor(input_schema.clone(), PkIndices::new()); let hash_agg = new_boxed_hash_agg_executor( store.clone(), - Box::new(source), + source, false, agg_calls.clone(), 0, diff --git a/src/stream/tests/integration_tests/hop_window.rs b/src/stream/tests/integration_tests/hop_window.rs index f0bd65c84b69e..97e2886dbecda 100644 --- a/src/stream/tests/integration_tests/hop_window.rs +++ b/src/stream/tests/integration_tests/hop_window.rs @@ -16,7 +16,7 @@ use risingwave_common::types::test_utils::IntervalTestExt; use risingwave_common::types::{Interval, Timestamp}; use risingwave_expr::expr::test_utils::make_hop_window_expression; use risingwave_expr::expr::NonStrictExpression; -use risingwave_stream::executor::{ExecutorInfo, HopWindowExecutor}; +use risingwave_stream::executor::HopWindowExecutor; use crate::prelude::*; @@ -24,12 +24,15 @@ const TIME_COL_IDX: usize = 2; const CHUNK_SIZE: usize = 256; fn create_executor(output_indices: Vec<usize>) -> (MessageSender, BoxedMessageStream) { - let field1 = Field::unnamed(DataType::Int64); - let field2 = Field::unnamed(DataType::Int64); - let field3 = Field::with_name(DataType::Timestamp, "created_at"); - let schema = Schema::new(vec![field1, field2, field3]); - let pk_indices = vec![0]; - let (tx, source) = MockSource::channel(schema.clone(), pk_indices.clone()); + let (tx, source) = MockSource::channel(); + let source = source.into_executor( + Schema::new(vec![ + Field::unnamed(DataType::Int64), + Field::unnamed(DataType::Int64), + Field::with_name(DataType::Timestamp, "created_at"), + ]), + vec![0], + ); let window_slide = Interval::from_minutes(15); let window_size = Interval::from_minutes(30); @@ -47,12 +50,7 @@ fn create_executor(output_indices: Vec<usize>) -> (MessageSender, BoxedMessageSt tx, HopWindowExecutor::new( ActorContext::for_test(123), - ExecutorInfo { - schema, - pk_indices, - identity: "HopWindowExecutor".to_string(), - }, - Box::new(source), + source, TIME_COL_IDX, window_slide, window_size, diff --git a/src/stream/tests/integration_tests/over_window.rs b/src/stream/tests/integration_tests/over_window.rs index 78203d79a87ac..0be8e1848e9cd 100644 --- a/src/stream/tests/integration_tests/over_window.rs +++ b/src/stream/tests/integration_tests/over_window.rs @@ -18,7 +18,7 @@ use risingwave_expr::window_function::{ Frame, FrameBound, FrameExclusion, WindowFuncCall, WindowFuncKind, }; use risingwave_stream::executor::monitor::StreamingMetrics; -use risingwave_stream::executor::{ExecutorInfo, OverWindowExecutor, OverWindowExecutorArgs}; +use risingwave_stream::executor::{OverWindowExecutor, OverWindowExecutorArgs}; use crate::prelude::*; @@ -63,7 +63,6 @@ async fn create_executor<S: StateStore>( }); Schema { fields } }; - let output_pk_indices = vec![2]; let state_table = StateTable::new_without_distribution( store, @@ -74,17 +73,14 @@ async fn create_executor<S: StateStore>( ) .await; - let (tx, source) = MockSource::channel(input_schema, input_pk_indices.clone()); + let (tx, source) = MockSource::channel(); + let source = source.into_executor(input_schema, input_pk_indices.clone()); let executor = OverWindowExecutor::new(OverWindowExecutorArgs { actor_ctx: ActorContext::for_test(123), - info: ExecutorInfo { - schema: output_schema, - pk_indices: output_pk_indices, - identity: "OverWindowExecutor".to_string(), - }, - input: source.boxed(), + input: source, + schema: output_schema, calls, partition_key_indices, order_key_indices, diff --git a/src/stream/tests/integration_tests/project_set.rs b/src/stream/tests/integration_tests/project_set.rs index 79fcbfc0d48db..5cff03284f4b6 100644 --- a/src/stream/tests/integration_tests/project_set.rs +++ b/src/stream/tests/integration_tests/project_set.rs @@ -14,7 +14,7 @@ use multimap::MultiMap; use risingwave_expr::table_function::repeat; -use risingwave_stream::executor::{ExecutorInfo, ProjectSetExecutor}; +use risingwave_stream::executor::ProjectSetExecutor; use crate::prelude::*; @@ -27,30 +27,17 @@ fn create_executor() -> (MessageSender, BoxedMessageStream) { Field::unnamed(DataType::Int64), ], }; - let (tx, source) = MockSource::channel(schema, PkIndices::new()); + let (tx, source) = MockSource::channel(); + let source = source.into_executor(schema, PkIndices::new()); let test_expr = build_from_pretty("(add:int8 $0:int8 $1:int8)").into_inner(); let test_expr_watermark = build_from_pretty("(add:int8 $0:int8 1:int8)").into_inner(); let tf1 = repeat(build_from_pretty("1:int4").into_inner(), 1); let tf2 = repeat(build_from_pretty("2:int4").into_inner(), 2); - let info = ExecutorInfo { - schema: Schema { - fields: vec![ - Field::unnamed(DataType::Int64), - Field::unnamed(DataType::Int64), - Field::unnamed(DataType::Int32), - Field::unnamed(DataType::Int32), - ], - }, - pk_indices: vec![], - identity: "ProjectSetExecutor".to_string(), - }; - - let project_set = Box::new(ProjectSetExecutor::new( + let project_set = ProjectSetExecutor::new( ActorContext::for_test(123), - info, - Box::new(source), + source, vec![ test_expr.into(), test_expr_watermark.into(), @@ -60,8 +47,8 @@ fn create_executor() -> (MessageSender, BoxedMessageStream) { CHUNK_SIZE, MultiMap::from_iter(std::iter::once((0, 1))), vec![], - )); - (tx, project_set.execute()) + ); + (tx, project_set.boxed().execute()) } #[tokio::test]