From b6ef3f652cb3c16abbb5b015a5b41c9e7f6525ea Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 20 Oct 2023 19:46:35 +0800 Subject: [PATCH 01/14] first attempt Signed-off-by: Bugen Zhao --- src/frontend/src/optimizer/plan_node/mod.rs | 129 +++++++++++++++++--- 1 file changed, 113 insertions(+), 16 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index f16ebfb0c792c..0e5e8546f6bcf 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -57,6 +57,58 @@ pub trait PlanNodeMeta { fn convention(&self) -> Convention; } +pub trait ConventionMarker { + fn value() -> Convention; +} +pub trait PhysicalConventionMarker: ConventionMarker {} + +pub struct Logical; +impl ConventionMarker for Logical { + fn value() -> Convention { + Convention::Logical + } +} + +pub struct Batch; +impl ConventionMarker for Batch { + fn value() -> Convention { + Convention::Batch + } +} +impl PhysicalConventionMarker for Batch {} + +pub struct Stream; +impl ConventionMarker for Stream { + fn value() -> Convention { + Convention::Stream + } +} +impl PhysicalConventionMarker for Stream {} + +pub trait StaticPlanNodeMeta { + type Convention: ConventionMarker; + + fn node_type(&self) -> PlanNodeType; + fn plan_base(&self) -> &PlanBase; +} + +impl

PlanNodeMeta for P +where + P: StaticPlanNodeMeta, +{ + fn node_type(&self) -> PlanNodeType { + P::node_type(self) + } + + fn plan_base(&self) -> &PlanBase { + P::plan_base(self) + } + + fn convention(&self) -> Convention { + P::Convention::value() + } +} + /// The common trait over all plan nodes. Used by optimizer framework which will treat all node as /// `dyn PlanNode` /// @@ -194,7 +246,7 @@ pub trait VisitPlan: Visit { } } -#[derive(Debug, PartialEq)] +#[derive(Clone, Copy, Debug, PartialEq)] pub enum Convention { Logical, Batch, @@ -436,7 +488,7 @@ impl PlanNodeMeta for PlanRef { /// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. impl

GenericPlanRef for P where - P: PlanNodeMeta + Eq + Hash, + P: StaticPlanNodeMeta + Eq + Hash, { fn id(&self) -> PlanNodeId { self.plan_base().id() @@ -459,22 +511,49 @@ where } } -/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. -// TODO: further constrain the convention to be `Stream` or `Batch`. +impl GenericPlanRef for PlanRef { + fn id(&self) -> PlanNodeId { + self.plan_base().id() + } + + fn schema(&self) -> &Schema { + self.plan_base().schema() + } + + fn stream_key(&self) -> Option<&[usize]> { + self.plan_base().stream_key() + } + + fn ctx(&self) -> OptimizerContextRef { + self.plan_base().ctx() + } + + fn functional_dependency(&self) -> &FunctionalDependencySet { + self.plan_base().functional_dependency() + } +} + impl

PhysicalPlanRef for P where - P: PlanNodeMeta + Eq + Hash, + P: Eq + Hash, + P: StaticPlanNodeMeta, + P::Convention: PhysicalConventionMarker, { fn distribution(&self) -> &Distribution { self.plan_base().distribution() } } -/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. -// TODO: further constrain the convention to be `Stream`. +impl PhysicalPlanRef for PlanRef { + fn distribution(&self) -> &Distribution { + self.plan_base().distribution() + } +} + impl

StreamPlanRef for P where - P: PlanNodeMeta + Eq + Hash, + P: Eq + Hash, + P: StaticPlanNodeMeta, { fn append_only(&self) -> bool { self.plan_base().append_only() @@ -489,17 +568,36 @@ where } } -/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. -// TODO: further constrain the convention to be `Batch`. +impl StreamPlanRef for PlanRef { + fn append_only(&self) -> bool { + self.plan_base().append_only() + } + + fn emit_on_window_close(&self) -> bool { + self.plan_base().emit_on_window_close() + } + + fn watermark_columns(&self) -> &FixedBitSet { + self.plan_base().watermark_columns() + } +} + impl

BatchPlanRef for P where - P: PlanNodeMeta + Eq + Hash, + P: Eq + Hash, + P: StaticPlanNodeMeta, { fn order(&self) -> &Order { self.plan_base().order() } } +impl BatchPlanRef for PlanRef { + fn order(&self) -> &Order { + self.plan_base().order() + } +} + /// In order to let expression display id started from 1 for explaining, hidden column names and /// other places. We will reset expression display id to 0 and clone the whole plan to reset the /// schema. @@ -1078,16 +1176,15 @@ macro_rules! impl_plan_node_meta { $( [<$convention $name>] ),* } - $(impl PlanNodeMeta for [<$convention $name>] { - fn node_type(&self) -> PlanNodeType{ + $(impl StaticPlanNodeMeta for [<$convention $name>] { + type Convention = $convention; + + fn node_type(&self) -> PlanNodeType { PlanNodeType::[<$convention $name>] } fn plan_base(&self) -> &PlanBase { &self.base } - fn convention(&self) -> Convention { - Convention::$convention - } })* } } From 19c46123ccc97283a604eab930e2bd2ebcfd3947 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Mon, 23 Oct 2023 13:53:54 +0800 Subject: [PATCH 02/14] make plan base over generic Signed-off-by: Bugen Zhao --- src/frontend/src/optimizer/plan_node/batch.rs | 8 +- .../src/optimizer/plan_node/generic/mod.rs | 4 +- src/frontend/src/optimizer/plan_node/mod.rs | 18 +- .../src/optimizer/plan_node/plan_base.rs | 184 ++++++++++-------- .../src/optimizer/plan_node/stream.rs | 13 +- 5 files changed, 131 insertions(+), 96 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/batch.rs b/src/frontend/src/optimizer/plan_node/batch.rs index 2cb2360b3e51d..f3e38f0e081c3 100644 --- a/src/frontend/src/optimizer/plan_node/batch.rs +++ b/src/frontend/src/optimizer/plan_node/batch.rs @@ -15,6 +15,10 @@ use super::generic::PhysicalPlanRef; use crate::optimizer::property::Order; +pub trait BatchSpecific { + fn order(&self) -> &Order; +} + /// A subtrait of [`PhysicalPlanRef`] for batch plans. /// /// Due to the lack of refactoring, all plan nodes currently implement this trait @@ -23,6 +27,4 @@ use crate::optimizer::property::Order; /// [`PhysicalPlanRef`]. /// /// [`GenericPlanRef`]: super::generic::GenericPlanRef -pub trait BatchPlanRef: PhysicalPlanRef { - fn order(&self) -> &Order; -} +pub trait BatchPlanRef = PhysicalPlanRef + BatchSpecific; diff --git a/src/frontend/src/optimizer/plan_node/generic/mod.rs b/src/frontend/src/optimizer/plan_node/generic/mod.rs index aec59c90bcc4e..2ad91f80361b5 100644 --- a/src/frontend/src/optimizer/plan_node/generic/mod.rs +++ b/src/frontend/src/optimizer/plan_node/generic/mod.rs @@ -92,10 +92,12 @@ pub trait GenericPlanRef: Eq + Hash { fn ctx(&self) -> OptimizerContextRef; } -pub trait PhysicalPlanRef: GenericPlanRef { +pub trait PhysicalSpecific { fn distribution(&self) -> &Distribution; } +pub trait PhysicalPlanRef = GenericPlanRef + PhysicalSpecific; + pub trait GenericPlanNode { fn functional_dependency(&self) -> FunctionalDependencySet; fn schema(&self) -> Schema; diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 0e5e8546f6bcf..a637315c21f93 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -57,13 +57,17 @@ pub trait PlanNodeMeta { fn convention(&self) -> Convention; } -pub trait ConventionMarker { +pub trait ConventionMarker: 'static + Sized { + type Extra: 'static + Clone + Debug; + fn value() -> Convention; } pub trait PhysicalConventionMarker: ConventionMarker {} pub struct Logical; impl ConventionMarker for Logical { + type Extra = plan_base::NoExtra; + fn value() -> Convention { Convention::Logical } @@ -71,6 +75,8 @@ impl ConventionMarker for Logical { pub struct Batch; impl ConventionMarker for Batch { + type Extra = plan_base::BatchExtra; + fn value() -> Convention { Convention::Batch } @@ -79,6 +85,8 @@ impl PhysicalConventionMarker for Batch {} pub struct Stream; impl ConventionMarker for Stream { + type Extra = plan_base::StreamExtra; + fn value() -> Convention { Convention::Stream } @@ -87,8 +95,8 @@ impl PhysicalConventionMarker for Stream {} pub trait StaticPlanNodeMeta { type Convention: ConventionMarker; + const NODE_TYPE: PlanNodeType; - fn node_type(&self) -> PlanNodeType; fn plan_base(&self) -> &PlanBase; } @@ -97,7 +105,7 @@ where P: StaticPlanNodeMeta, { fn node_type(&self) -> PlanNodeType { - P::node_type(self) + P::NODE_TYPE } fn plan_base(&self) -> &PlanBase { @@ -1178,10 +1186,8 @@ macro_rules! impl_plan_node_meta { $(impl StaticPlanNodeMeta for [<$convention $name>] { type Convention = $convention; + const NODE_TYPE: PlanNodeType = PlanNodeType::[<$convention $name>]; - fn node_type(&self) -> PlanNodeType { - PlanNodeType::[<$convention $name>] - } fn plan_base(&self) -> &PlanBase { &self.base } diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs index 51b1aa5f41141..c9b4d9cf244f6 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -21,6 +21,9 @@ use super::*; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::property::{Distribution, FunctionalDependencySet, Order}; +#[derive(Clone, Debug, PartialEq, Eq, Hash)] +pub struct NoExtra; + /// Common extra fields for physical plan nodes. #[derive(Clone, Debug, PartialEq, Eq, Hash)] struct PhysicalCommonExtra { @@ -31,7 +34,7 @@ struct PhysicalCommonExtra { /// Extra fields for stream plan nodes. #[derive(Clone, Debug, PartialEq, Eq, Hash)] -struct StreamExtra { +pub struct StreamExtra { /// Common fields for physical plan nodes. physical: PhysicalCommonExtra, @@ -45,9 +48,29 @@ struct StreamExtra { watermark_columns: FixedBitSet, } +impl generic::PhysicalSpecific for StreamExtra { + fn distribution(&self) -> &Distribution { + &self.physical.dist + } +} + +impl stream::StreamSpecific for StreamExtra { + fn append_only(&self) -> bool { + self.append_only + } + + fn emit_on_window_close(&self) -> bool { + self.emit_on_window_close + } + + fn watermark_columns(&self) -> &FixedBitSet { + &self.watermark_columns + } +} + /// Extra fields for batch plan nodes. #[derive(Clone, Debug, PartialEq, Eq, Hash)] -struct BatchExtra { +pub struct BatchExtra { /// Common fields for physical plan nodes. physical: PhysicalCommonExtra, @@ -56,40 +79,15 @@ struct BatchExtra { order: Order, } -/// Extra fields for physical plan nodes. -#[derive(Clone, Debug, PartialEq, Eq, Hash)] -enum PhysicalExtra { - Stream(StreamExtra), - Batch(BatchExtra), -} - -impl PhysicalExtra { - fn common(&self) -> &PhysicalCommonExtra { - match self { - PhysicalExtra::Stream(stream) => &stream.physical, - PhysicalExtra::Batch(batch) => &batch.physical, - } - } - - fn common_mut(&mut self) -> &mut PhysicalCommonExtra { - match self { - PhysicalExtra::Stream(stream) => &mut stream.physical, - PhysicalExtra::Batch(batch) => &mut batch.physical, - } - } - - fn stream(&self) -> &StreamExtra { - match self { - PhysicalExtra::Stream(extra) => extra, - _ => panic!("access stream properties from batch plan node"), - } +impl generic::PhysicalSpecific for BatchExtra { + fn distribution(&self) -> &Distribution { + &self.physical.dist } +} - fn batch(&self) -> &BatchExtra { - match self { - PhysicalExtra::Batch(extra) => extra, - _ => panic!("access batch properties from stream plan node"), - } +impl batch::BatchSpecific for BatchExtra { + fn order(&self) -> &Order { + &self.order } } @@ -102,9 +100,9 @@ impl PhysicalExtra { /// - To access them, use traits including [`GenericPlanRef`], /// [`PhysicalPlanRef`], [`StreamPlanRef`] and [`BatchPlanRef`]. /// - To mutate them, use methods like `new_*` or `clone_with_*`. -#[derive(Clone, Debug, Educe)] -#[educe(PartialEq, Eq, Hash)] -pub struct PlanBase { +#[derive(Educe)] +#[educe(PartialEq, Eq, Hash, Clone, Debug)] +pub struct PlanBase { // -- common fields -- #[educe(PartialEq(ignore), Hash(ignore))] id: PlanNodeId, @@ -117,25 +115,24 @@ pub struct PlanBase { stream_key: Option>, functional_dependency: FunctionalDependencySet, - /// Extra fields if the plan node is physical. - physical_extra: Option, -} - -impl PlanBase { - fn physical_extra(&self) -> &PhysicalExtra { - self.physical_extra - .as_ref() - .expect("access physical properties from logical plan node") - } - - fn physical_extra_mut(&mut self) -> &mut PhysicalExtra { - self.physical_extra - .as_mut() - .expect("access physical properties from logical plan node") - } + extra: C::Extra, } -impl generic::GenericPlanRef for PlanBase { +// impl PlanBase { +// fn physical_extra(&self) -> &PhysicalExtra { +// self.physical_extra +// .as_ref() +// .expect("access physical properties from logical plan node") +// } + +// fn physical_extra_mut(&mut self) -> &mut PhysicalExtra { +// self.physical_extra +// .as_mut() +// .expect("access physical properties from logical plan node") +// } +// } + +impl generic::GenericPlanRef for PlanBase { fn id(&self) -> PlanNodeId { self.id } @@ -157,33 +154,50 @@ impl generic::GenericPlanRef for PlanBase { } } -impl generic::PhysicalPlanRef for PlanBase { +impl generic::PhysicalSpecific for PlanBase +where + C::Extra: generic::PhysicalSpecific, +{ fn distribution(&self) -> &Distribution { - &self.physical_extra().common().dist + &self.extra.distribution() } } -impl stream::StreamPlanRef for PlanBase { +impl stream::StreamSpecific for PlanBase +where + C::Extra: stream::StreamSpecific, +{ fn append_only(&self) -> bool { - self.physical_extra().stream().append_only + self.extra.append_only() } fn emit_on_window_close(&self) -> bool { - self.physical_extra().stream().emit_on_window_close + self.extra.emit_on_window_close() } fn watermark_columns(&self) -> &FixedBitSet { - &self.physical_extra().stream().watermark_columns + &self.extra.watermark_columns() } } -impl batch::BatchPlanRef for PlanBase { +impl batch::BatchSpecific for PlanBase +where + C::Extra: batch::BatchSpecific, +{ fn order(&self) -> &Order { - &self.physical_extra().batch().order + &self.extra.order() } } -impl PlanBase { +impl PlanBase { + pub fn clone_with_new_plan_id(&self) -> Self { + let mut new = self.clone(); + new.id = self.ctx().next_plan_node_id(); + new + } +} + +impl PlanBase { pub fn new_logical( ctx: OptimizerContextRef, schema: Schema, @@ -197,7 +211,7 @@ impl PlanBase { schema, stream_key, functional_dependency, - physical_extra: None, + extra: NoExtra, } } @@ -209,7 +223,9 @@ impl PlanBase { core.functional_dependency(), ) } +} +impl PlanBase { pub fn new_stream( ctx: OptimizerContextRef, schema: Schema, @@ -228,14 +244,12 @@ impl PlanBase { schema, stream_key, functional_dependency, - physical_extra: Some(PhysicalExtra::Stream({ - StreamExtra { - physical: PhysicalCommonExtra { dist }, - append_only, - emit_on_window_close, - watermark_columns, - } - })), + extra: StreamExtra { + physical: PhysicalCommonExtra { dist }, + append_only, + emit_on_window_close, + watermark_columns, + }, } } @@ -257,7 +271,9 @@ impl PlanBase { watermark_columns, ) } +} +impl PlanBase { pub fn new_batch( ctx: OptimizerContextRef, schema: Schema, @@ -272,12 +288,10 @@ impl PlanBase { schema, stream_key: None, functional_dependency, - physical_extra: Some(PhysicalExtra::Batch({ - BatchExtra { - physical: PhysicalCommonExtra { dist }, - order, - } - })), + extra: BatchExtra { + physical: PhysicalCommonExtra { dist }, + order, + }, } } @@ -288,19 +302,27 @@ impl PlanBase { ) -> Self { Self::new_batch(core.ctx(), core.schema(), dist, order) } +} - pub fn clone_with_new_plan_id(&self) -> Self { +impl PlanBase { + /// Clone the plan node with a new distribution. + /// + /// Panics if the plan node is not physical. + pub fn clone_with_new_distribution(&self, dist: Distribution) -> Self { let mut new = self.clone(); - new.id = self.ctx().next_plan_node_id(); + new.extra.physical.dist = dist; new } +} +// TODO: unify the impls for `PlanBase` and `PlanBase`. +impl PlanBase { /// Clone the plan node with a new distribution. /// /// Panics if the plan node is not physical. pub fn clone_with_new_distribution(&self, dist: Distribution) -> Self { let mut new = self.clone(); - new.physical_extra_mut().common_mut().dist = dist; + new.extra.physical.dist = dist; new } } diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 866c62c2413a5..bd7103cfc82c5 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -16,6 +16,13 @@ use fixedbitset::FixedBitSet; use super::generic::PhysicalPlanRef; +pub trait StreamSpecific { + fn append_only(&self) -> bool; + fn emit_on_window_close(&self) -> bool; + fn watermark_columns(&self) -> &FixedBitSet; +} + + /// A subtrait of [`PhysicalPlanRef`] for stream plans. /// /// Due to the lack of refactoring, all plan nodes currently implement this trait @@ -24,8 +31,4 @@ use super::generic::PhysicalPlanRef; /// [`PhysicalPlanRef`]. /// /// [`GenericPlanRef`]: super::generic::GenericPlanRef -pub trait StreamPlanRef: PhysicalPlanRef { - fn append_only(&self) -> bool; - fn emit_on_window_close(&self) -> bool; - fn watermark_columns(&self) -> &FixedBitSet; -} +pub trait StreamPlanRef = PhysicalPlanRef + StreamSpecific; From 30da6e21ca1248866eae3d88ec82a4176ea7b0a5 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Mon, 23 Oct 2023 13:58:08 +0800 Subject: [PATCH 03/14] specify generic for plan nodes Signed-off-by: Bugen Zhao --- .../src/optimizer/plan_node/batch_delete.rs | 4 ++-- .../src/optimizer/plan_node/batch_exchange.rs | 2 +- .../src/optimizer/plan_node/batch_expand.rs | 2 +- .../src/optimizer/plan_node/batch_filter.rs | 2 +- .../optimizer/plan_node/batch_group_topn.rs | 2 +- .../src/optimizer/plan_node/batch_hash_agg.rs | 2 +- .../optimizer/plan_node/batch_hash_join.rs | 2 +- .../optimizer/plan_node/batch_hop_window.rs | 2 +- .../src/optimizer/plan_node/batch_insert.rs | 4 ++-- .../src/optimizer/plan_node/batch_limit.rs | 2 +- .../optimizer/plan_node/batch_lookup_join.rs | 2 +- .../plan_node/batch_nested_loop_join.rs | 2 +- .../optimizer/plan_node/batch_over_window.rs | 2 +- .../src/optimizer/plan_node/batch_project.rs | 2 +- .../optimizer/plan_node/batch_project_set.rs | 2 +- .../src/optimizer/plan_node/batch_seq_scan.rs | 2 +- .../optimizer/plan_node/batch_simple_agg.rs | 2 +- .../src/optimizer/plan_node/batch_sort.rs | 2 +- .../src/optimizer/plan_node/batch_sort_agg.rs | 2 +- .../src/optimizer/plan_node/batch_source.rs | 2 +- .../plan_node/batch_table_function.rs | 2 +- .../src/optimizer/plan_node/batch_topn.rs | 2 +- .../src/optimizer/plan_node/batch_union.rs | 2 +- .../src/optimizer/plan_node/batch_update.rs | 2 +- .../src/optimizer/plan_node/batch_values.rs | 2 +- .../src/optimizer/plan_node/logical_agg.rs | 2 +- .../src/optimizer/plan_node/logical_apply.rs | 2 +- .../src/optimizer/plan_node/logical_dedup.rs | 2 +- .../src/optimizer/plan_node/logical_delete.rs | 2 +- .../src/optimizer/plan_node/logical_except.rs | 2 +- .../src/optimizer/plan_node/logical_expand.rs | 2 +- .../src/optimizer/plan_node/logical_filter.rs | 2 +- .../optimizer/plan_node/logical_hop_window.rs | 2 +- .../src/optimizer/plan_node/logical_insert.rs | 2 +- .../optimizer/plan_node/logical_intersect.rs | 2 +- .../src/optimizer/plan_node/logical_join.rs | 2 +- .../src/optimizer/plan_node/logical_limit.rs | 2 +- .../optimizer/plan_node/logical_multi_join.rs | 2 +- .../src/optimizer/plan_node/logical_now.rs | 2 +- .../plan_node/logical_over_window.rs | 2 +- .../optimizer/plan_node/logical_project.rs | 2 +- .../plan_node/logical_project_set.rs | 2 +- .../src/optimizer/plan_node/logical_scan.rs | 2 +- .../src/optimizer/plan_node/logical_share.rs | 12 +++++++---- .../src/optimizer/plan_node/logical_source.rs | 2 +- .../plan_node/logical_table_function.rs | 2 +- .../src/optimizer/plan_node/logical_topn.rs | 2 +- .../src/optimizer/plan_node/logical_union.rs | 2 +- .../src/optimizer/plan_node/logical_update.rs | 2 +- .../src/optimizer/plan_node/logical_values.rs | 2 +- src/frontend/src/optimizer/plan_node/mod.rs | 20 +++++++++---------- .../src/optimizer/plan_node/stream_dedup.rs | 2 +- .../optimizer/plan_node/stream_delta_join.rs | 2 +- .../src/optimizer/plan_node/stream_dml.rs | 2 +- .../plan_node/stream_dynamic_filter.rs | 2 +- .../plan_node/stream_eowc_over_window.rs | 2 +- .../optimizer/plan_node/stream_exchange.rs | 2 +- .../src/optimizer/plan_node/stream_expand.rs | 2 +- .../src/optimizer/plan_node/stream_filter.rs | 2 +- .../optimizer/plan_node/stream_fs_fetch.rs | 2 +- .../optimizer/plan_node/stream_group_topn.rs | 2 +- .../optimizer/plan_node/stream_hash_agg.rs | 2 +- .../optimizer/plan_node/stream_hash_join.rs | 2 +- .../optimizer/plan_node/stream_hop_window.rs | 2 +- .../optimizer/plan_node/stream_materialize.rs | 2 +- .../src/optimizer/plan_node/stream_now.rs | 2 +- .../optimizer/plan_node/stream_over_window.rs | 2 +- .../src/optimizer/plan_node/stream_project.rs | 2 +- .../optimizer/plan_node/stream_project_set.rs | 2 +- .../optimizer/plan_node/stream_row_id_gen.rs | 2 +- .../src/optimizer/plan_node/stream_share.rs | 2 +- .../optimizer/plan_node/stream_simple_agg.rs | 2 +- .../src/optimizer/plan_node/stream_sink.rs | 2 +- .../src/optimizer/plan_node/stream_sort.rs | 2 +- .../src/optimizer/plan_node/stream_source.rs | 2 +- .../plan_node/stream_stateless_simple_agg.rs | 2 +- .../optimizer/plan_node/stream_table_scan.rs | 2 +- .../plan_node/stream_temporal_join.rs | 2 +- .../src/optimizer/plan_node/stream_topn.rs | 2 +- .../src/optimizer/plan_node/stream_union.rs | 2 +- .../src/optimizer/plan_node/stream_values.rs | 2 +- .../plan_node/stream_watermark_filter.rs | 4 ++-- 82 files changed, 101 insertions(+), 97 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/batch_delete.rs b/src/frontend/src/optimizer/plan_node/batch_delete.rs index 85d22a46b450e..f250b1c2e1c15 100644 --- a/src/frontend/src/optimizer/plan_node/batch_delete.rs +++ b/src/frontend/src/optimizer/plan_node/batch_delete.rs @@ -26,14 +26,14 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchDelete` implements [`super::LogicalDelete`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchDelete { - pub base: PlanBase, + pub base: PlanBase, pub core: generic::Delete, } impl BatchDelete { pub fn new(core: generic::Delete) -> Self { assert_eq!(core.input.distribution(), &Distribution::Single); - let base: PlanBase = + let base: PlanBase = PlanBase::new_batch_with_core(&core, core.input.distribution().clone(), Order::any()); Self { base, core } } diff --git a/src/frontend/src/optimizer/plan_node/batch_exchange.rs b/src/frontend/src/optimizer/plan_node/batch_exchange.rs index 6477c7ec213e2..b42902f185743 100644 --- a/src/frontend/src/optimizer/plan_node/batch_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/batch_exchange.rs @@ -28,7 +28,7 @@ use crate::optimizer::property::{Distribution, DistributionDisplay, Order, Order /// without changing its content. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchExchange { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, } diff --git a/src/frontend/src/optimizer/plan_node/batch_expand.rs b/src/frontend/src/optimizer/plan_node/batch_expand.rs index af4413b9e5152..43ca3d9b33b4a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_expand.rs +++ b/src/frontend/src/optimizer/plan_node/batch_expand.rs @@ -28,7 +28,7 @@ use crate::optimizer::PlanRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchExpand { - pub base: PlanBase, + pub base: PlanBase, core: generic::Expand, } diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index 4bff7cbfee3c0..e035e2fb538fb 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -25,7 +25,7 @@ use crate::utils::Condition; /// `BatchFilter` implements [`super::LogicalFilter`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchFilter { - pub base: PlanBase, + pub base: PlanBase, core: generic::Filter, } diff --git a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs index 70ee8328623f5..fe9ad907796d5 100644 --- a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs @@ -26,7 +26,7 @@ use crate::optimizer::property::{Order, RequiredDist}; /// `BatchGroupTopN` implements [`super::LogicalTopN`] to find the top N elements with a heap #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchGroupTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, } diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs index b4ab3341ace29..ca2246729e576 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -30,7 +30,7 @@ use crate::utils::{ColIndexMappingRewriteExt, IndexSet}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchHashAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, } diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs index bad586d4af1e4..1b95f50c9bf43 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -35,7 +35,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// get output rows. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchHashJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, /// The join condition must be equivalent to `logical.on`, but separated into equal and diff --git a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs index 2a4a27f9a0583..07480b0fc6a34 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs @@ -29,7 +29,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// input rows #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchHopWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::HopWindow, window_start_exprs: Vec, window_end_exprs: Vec, diff --git a/src/frontend/src/optimizer/plan_node/batch_insert.rs b/src/frontend/src/optimizer/plan_node/batch_insert.rs index aec05eee145b8..cdea2c3cd2b75 100644 --- a/src/frontend/src/optimizer/plan_node/batch_insert.rs +++ b/src/frontend/src/optimizer/plan_node/batch_insert.rs @@ -28,14 +28,14 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchInsert` implements [`super::LogicalInsert`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchInsert { - pub base: PlanBase, + pub base: PlanBase, pub core: generic::Insert, } impl BatchInsert { pub fn new(core: generic::Insert) -> Self { assert_eq!(core.input.distribution(), &Distribution::Single); - let base: PlanBase = + let base: PlanBase = PlanBase::new_batch_with_core(&core, core.input.distribution().clone(), Order::any()); BatchInsert { base, core } diff --git a/src/frontend/src/optimizer/plan_node/batch_limit.rs b/src/frontend/src/optimizer/plan_node/batch_limit.rs index 93b14d0198979..9e8f01ac3afbc 100644 --- a/src/frontend/src/optimizer/plan_node/batch_limit.rs +++ b/src/frontend/src/optimizer/plan_node/batch_limit.rs @@ -27,7 +27,7 @@ use crate::optimizer::property::{Order, RequiredDist}; /// `BatchLimit` implements [`super::LogicalLimit`] to fetch specified rows from input #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchLimit { - pub base: PlanBase, + pub base: PlanBase, core: generic::Limit, } diff --git a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs index 48f99668c3af7..4d10ae67017bb 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -33,7 +33,7 @@ use crate::utils::ColIndexMappingRewriteExt; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchLookupJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, /// The join condition must be equivalent to `logical.on`, but separated into equal and diff --git a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs index 8980ad2f23f6d..9a1c06cd5c663 100644 --- a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs @@ -30,7 +30,7 @@ use crate::utils::ConditionDisplay; /// against all pairs of rows from inner & outer side within 2 layers of loops. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchNestedLoopJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, } diff --git a/src/frontend/src/optimizer/plan_node/batch_over_window.rs b/src/frontend/src/optimizer/plan_node/batch_over_window.rs index fb455758f331a..77d50a3b2663c 100644 --- a/src/frontend/src/optimizer/plan_node/batch_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_over_window.rs @@ -28,7 +28,7 @@ use crate::optimizer::property::{Order, RequiredDist}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchOverWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::OverWindow, } diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index 642683967c5c3..0b85ab6cc556a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -31,7 +31,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// rows #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchProject { - pub base: PlanBase, + pub base: PlanBase, core: generic::Project, } diff --git a/src/frontend/src/optimizer/plan_node/batch_project_set.rs b/src/frontend/src/optimizer/plan_node/batch_project_set.rs index 5888df9d15889..f8890b4e346ae 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project_set.rs @@ -28,7 +28,7 @@ use crate::utils::ColIndexMappingRewriteExt; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchProjectSet { - pub base: PlanBase, + pub base: PlanBase, core: generic::ProjectSet, } diff --git a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs index 6834ed29353b9..c2c63136482c3 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -36,7 +36,7 @@ use crate::optimizer::property::{Distribution, DistributionDisplay, Order}; /// `BatchSeqScan` implements [`super::LogicalScan`] to scan from a row-oriented table #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSeqScan { - pub base: PlanBase, + pub base: PlanBase, core: generic::Scan, scan_ranges: Vec, } diff --git a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs index bae8d70c2eedf..390bdc53df5be 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -25,7 +25,7 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSimpleAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, } diff --git a/src/frontend/src/optimizer/plan_node/batch_sort.rs b/src/frontend/src/optimizer/plan_node/batch_sort.rs index e7bff6d51d85b..ac76445e08702 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort.rs @@ -27,7 +27,7 @@ use crate::optimizer::property::{Order, OrderDisplay}; /// collation required by user or parent plan node. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSort { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, } diff --git a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs index 2252d4c0c0ee0..af2f68b06deca 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -27,7 +27,7 @@ use crate::utils::{ColIndexMappingRewriteExt, IndexSet}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSortAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, input_order: Order, } diff --git a/src/frontend/src/optimizer/plan_node/batch_source.rs b/src/frontend/src/optimizer/plan_node/batch_source.rs index 9e2cd6006db0b..2789c2211a76b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_source.rs +++ b/src/frontend/src/optimizer/plan_node/batch_source.rs @@ -30,7 +30,7 @@ use crate::optimizer::property::{Distribution, Order}; /// [`BatchSource`] represents a table/connector source at the very beginning of the graph. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSource { - pub base: PlanBase, + pub base: PlanBase, core: generic::Source, } diff --git a/src/frontend/src/optimizer/plan_node/batch_table_function.rs b/src/frontend/src/optimizer/plan_node/batch_table_function.rs index 0b9887cd4aaba..333863039ea96 100644 --- a/src/frontend/src/optimizer/plan_node/batch_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/batch_table_function.rs @@ -27,7 +27,7 @@ use crate::optimizer::property::{Distribution, Order}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchTableFunction { - pub base: PlanBase, + pub base: PlanBase, logical: LogicalTableFunction, } diff --git a/src/frontend/src/optimizer/plan_node/batch_topn.rs b/src/frontend/src/optimizer/plan_node/batch_topn.rs index b2eda24046d28..dcd3c1699a068 100644 --- a/src/frontend/src/optimizer/plan_node/batch_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_topn.rs @@ -28,7 +28,7 @@ use crate::optimizer::property::{Order, RequiredDist}; /// `BatchTopN` implements [`super::LogicalTopN`] to find the top N elements with a heap #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, } diff --git a/src/frontend/src/optimizer/plan_node/batch_union.rs b/src/frontend/src/optimizer/plan_node/batch_union.rs index c7c71111174c6..af8cf4c9da104 100644 --- a/src/frontend/src/optimizer/plan_node/batch_union.rs +++ b/src/frontend/src/optimizer/plan_node/batch_union.rs @@ -24,7 +24,7 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchUnion` implements [`super::LogicalUnion`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchUnion { - pub base: PlanBase, + pub base: PlanBase, core: generic::Union, } diff --git a/src/frontend/src/optimizer/plan_node/batch_update.rs b/src/frontend/src/optimizer/plan_node/batch_update.rs index 20e4b8b6b966c..ee90c0ee4be9b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_update.rs +++ b/src/frontend/src/optimizer/plan_node/batch_update.rs @@ -30,7 +30,7 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchUpdate` implements [`super::LogicalUpdate`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchUpdate { - pub base: PlanBase, + pub base: PlanBase, pub core: generic::Update, } diff --git a/src/frontend/src/optimizer/plan_node/batch_values.rs b/src/frontend/src/optimizer/plan_node/batch_values.rs index 9348cddba7422..330d8d65735df 100644 --- a/src/frontend/src/optimizer/plan_node/batch_values.rs +++ b/src/frontend/src/optimizer/plan_node/batch_values.rs @@ -30,7 +30,7 @@ use crate::optimizer::property::{Distribution, Order}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchValues { - pub base: PlanBase, + pub base: PlanBase, logical: LogicalValues, } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 0ad9b828ead4b..ea0e45460a5cc 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -48,7 +48,7 @@ use crate::utils::{ /// The output schema will first include the group key and then the aggregation calls. #[derive(Clone, Debug, PartialEq, Eq, Hash)] pub struct LogicalAgg { - pub base: PlanBase, + pub base: PlanBase, core: Agg, } diff --git a/src/frontend/src/optimizer/plan_node/logical_apply.rs b/src/frontend/src/optimizer/plan_node/logical_apply.rs index b398ce7494f61..efe5a84054746 100644 --- a/src/frontend/src/optimizer/plan_node/logical_apply.rs +++ b/src/frontend/src/optimizer/plan_node/logical_apply.rs @@ -38,7 +38,7 @@ use crate::utils::{ColIndexMapping, Condition, ConditionDisplay}; /// left side. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalApply { - pub base: PlanBase, + pub base: PlanBase, left: PlanRef, right: PlanRef, on: Condition, diff --git a/src/frontend/src/optimizer/plan_node/logical_dedup.rs b/src/frontend/src/optimizer/plan_node/logical_dedup.rs index dd46f9af9be1d..de1678211f536 100644 --- a/src/frontend/src/optimizer/plan_node/logical_dedup.rs +++ b/src/frontend/src/optimizer/plan_node/logical_dedup.rs @@ -32,7 +32,7 @@ use crate::utils::Condition; /// an `ORDER BY`. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalDedup { - pub base: PlanBase, + pub base: PlanBase, core: generic::Dedup, } diff --git a/src/frontend/src/optimizer/plan_node/logical_delete.rs b/src/frontend/src/optimizer/plan_node/logical_delete.rs index 2cda2b782af7e..119bad8f9e090 100644 --- a/src/frontend/src/optimizer/plan_node/logical_delete.rs +++ b/src/frontend/src/optimizer/plan_node/logical_delete.rs @@ -31,7 +31,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// It corresponds to the `DELETE` statements in SQL. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalDelete { - pub base: PlanBase, + pub base: PlanBase, core: generic::Delete, } diff --git a/src/frontend/src/optimizer/plan_node/logical_except.rs b/src/frontend/src/optimizer/plan_node/logical_except.rs index 11ff8b0210ee6..983794530cf0d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_except.rs +++ b/src/frontend/src/optimizer/plan_node/logical_except.rs @@ -29,7 +29,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// matching rows from its other inputs. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalExcept { - pub base: PlanBase, + pub base: PlanBase, core: generic::Except, } diff --git a/src/frontend/src/optimizer/plan_node/logical_expand.rs b/src/frontend/src/optimizer/plan_node/logical_expand.rs index d1f3b666feef5..7f326c47fad93 100644 --- a/src/frontend/src/optimizer/plan_node/logical_expand.rs +++ b/src/frontend/src/optimizer/plan_node/logical_expand.rs @@ -37,7 +37,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// is used to distinguish between different `subset`s in `column_subsets`. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalExpand { - pub base: PlanBase, + pub base: PlanBase, core: generic::Expand, } diff --git a/src/frontend/src/optimizer/plan_node/logical_filter.rs b/src/frontend/src/optimizer/plan_node/logical_filter.rs index a62b91aac5277..e6edd619a156b 100644 --- a/src/frontend/src/optimizer/plan_node/logical_filter.rs +++ b/src/frontend/src/optimizer/plan_node/logical_filter.rs @@ -37,7 +37,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// If the condition allows nulls, then a null value is treated the same as false. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalFilter { - pub base: PlanBase, + pub base: PlanBase, core: generic::Filter, } diff --git a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs index da2ec2138c3d1..20712418a98f0 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -32,7 +32,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition}; /// `LogicalHopWindow` implements Hop Table Function. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalHopWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::HopWindow, } diff --git a/src/frontend/src/optimizer/plan_node/logical_insert.rs b/src/frontend/src/optimizer/plan_node/logical_insert.rs index e93b77d79c1f2..1574859dffa85 100644 --- a/src/frontend/src/optimizer/plan_node/logical_insert.rs +++ b/src/frontend/src/optimizer/plan_node/logical_insert.rs @@ -35,7 +35,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// statements, the input relation would be [`super::LogicalValues`]. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalInsert { - pub base: PlanBase, + pub base: PlanBase, core: generic::Insert, } diff --git a/src/frontend/src/optimizer/plan_node/logical_intersect.rs b/src/frontend/src/optimizer/plan_node/logical_intersect.rs index 2da89e01d8319..ecaf1921a0757 100644 --- a/src/frontend/src/optimizer/plan_node/logical_intersect.rs +++ b/src/frontend/src/optimizer/plan_node/logical_intersect.rs @@ -28,7 +28,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// If `all` is false, it needs to eliminate duplicates. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalIntersect { - pub base: PlanBase, + pub base: PlanBase, core: generic::Intersect, } diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index a586af2f0bf42..96a9ba631241f 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -49,7 +49,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition, Condit /// right columns, dependent on the output indices provided. A repeat output index is illegal. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, } diff --git a/src/frontend/src/optimizer/plan_node/logical_limit.rs b/src/frontend/src/optimizer/plan_node/logical_limit.rs index ea53c4a2d546c..5c441d6bcb219 100644 --- a/src/frontend/src/optimizer/plan_node/logical_limit.rs +++ b/src/frontend/src/optimizer/plan_node/logical_limit.rs @@ -30,7 +30,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// `LogicalLimit` fetches up to `limit` rows from `offset` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalLimit { - pub base: PlanBase, + pub base: PlanBase, pub(super) core: generic::Limit, } diff --git a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs index 9b740abd7718e..fce2526ccfbdd 100644 --- a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs @@ -46,7 +46,7 @@ use crate::utils::{ /// expressed as 2-way `LogicalJoin`s. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalMultiJoin { - pub base: PlanBase, + pub base: PlanBase, inputs: Vec, on: Condition, output_indices: Vec, diff --git a/src/frontend/src/optimizer/plan_node/logical_now.rs b/src/frontend/src/optimizer/plan_node/logical_now.rs index 1d720db15b71a..0b817e75d6b20 100644 --- a/src/frontend/src/optimizer/plan_node/logical_now.rs +++ b/src/frontend/src/optimizer/plan_node/logical_now.rs @@ -31,7 +31,7 @@ use crate::OptimizerContextRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalNow { - pub base: PlanBase, + pub base: PlanBase, } impl LogicalNow { diff --git a/src/frontend/src/optimizer/plan_node/logical_over_window.rs b/src/frontend/src/optimizer/plan_node/logical_over_window.rs index b1796ddc62752..1f82c34b0ee9a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_over_window.rs @@ -358,7 +358,7 @@ impl<'a> ExprVisitor for OverWindowProjectBuilder<'a> { /// The output schema is the input schema plus the window functions. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalOverWindow { - pub base: PlanBase, + pub base: PlanBase, core: OverWindow, } diff --git a/src/frontend/src/optimizer/plan_node/logical_project.rs b/src/frontend/src/optimizer/plan_node/logical_project.rs index a96de7d91ecd5..51ccfcd1a05d4 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project.rs @@ -33,7 +33,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition, Substi /// `LogicalProject` computes a set of expressions from its input relation. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalProject { - pub base: PlanBase, + pub base: PlanBase, core: generic::Project, } diff --git a/src/frontend/src/optimizer/plan_node/logical_project_set.rs b/src/frontend/src/optimizer/plan_node/logical_project_set.rs index 4bf6b18cdabe3..bf6d7c912c300 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project_set.rs @@ -41,7 +41,7 @@ use crate::utils::{ColIndexMapping, Condition, Substitute}; /// column. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalProjectSet { - pub base: PlanBase, + pub base: PlanBase, core: generic::ProjectSet, } diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 07d2a6c7653e7..4322b80b103a9 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -42,7 +42,7 @@ use crate::utils::{ColIndexMapping, Condition, ConditionDisplay}; /// `LogicalScan` returns contents of a table or other equivalent object #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalScan { - pub base: PlanBase, + pub base: PlanBase, core: generic::Scan, } diff --git a/src/frontend/src/optimizer/plan_node/logical_share.rs b/src/frontend/src/optimizer/plan_node/logical_share.rs index d6b5711740a98..cf50572411425 100644 --- a/src/frontend/src/optimizer/plan_node/logical_share.rs +++ b/src/frontend/src/optimizer/plan_node/logical_share.rs @@ -20,8 +20,8 @@ use risingwave_common::error::Result; use super::utils::{childless_record, Distill}; use super::{ - generic, ColPrunable, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, - ToBatch, ToStream, + generic, ColPrunable, ConventionMarker, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, + PredicatePushdown, ToBatch, ToStream, }; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ @@ -49,7 +49,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// ``` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalShare { - pub base: PlanBase, + pub base: PlanBase, core: generic::Share, } @@ -68,7 +68,11 @@ impl LogicalShare { LogicalShare::new(input).into() } - pub(super) fn pretty_fields<'a>(base: &PlanBase, name: &'a str) -> XmlNode<'a> { + // TODO + pub(super) fn pretty_fields<'a>( + base: &PlanBase, + name: &'a str, + ) -> XmlNode<'a> { childless_record(name, vec![("id", Pretty::debug(&base.id().0))]) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 45a5fbcb2240f..e069f6b271a02 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -51,7 +51,7 @@ use crate::utils::{ColIndexMapping, Condition, IndexRewriter}; /// `LogicalSource` returns contents of a table or other equivalent object #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalSource { - pub base: PlanBase, + pub base: PlanBase, pub core: generic::Source, /// Expressions to output. This field presents and will be turned to a `Project` when diff --git a/src/frontend/src/optimizer/plan_node/logical_table_function.rs b/src/frontend/src/optimizer/plan_node/logical_table_function.rs index 15d510cc1c6fd..c2599412357f9 100644 --- a/src/frontend/src/optimizer/plan_node/logical_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/logical_table_function.rs @@ -35,7 +35,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// If the function returns a struct, it will be flattened into multiple columns. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalTableFunction { - pub base: PlanBase, + pub base: PlanBase, pub table_function: TableFunction, pub with_ordinality: bool, } diff --git a/src/frontend/src/optimizer/plan_node/logical_topn.rs b/src/frontend/src/optimizer/plan_node/logical_topn.rs index 39d97a56fe3a6..58a66c2fec47d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_topn.rs +++ b/src/frontend/src/optimizer/plan_node/logical_topn.rs @@ -36,7 +36,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition}; /// `LogicalTopN` sorts the input data and fetches up to `limit` rows from `offset` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, } diff --git a/src/frontend/src/optimizer/plan_node/logical_union.rs b/src/frontend/src/optimizer/plan_node/logical_union.rs index 1f02b026c0020..c1e75a948cf76 100644 --- a/src/frontend/src/optimizer/plan_node/logical_union.rs +++ b/src/frontend/src/optimizer/plan_node/logical_union.rs @@ -37,7 +37,7 @@ use crate::Explain; /// If `all` is false, it needs to eliminate duplicates. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalUnion { - pub base: PlanBase, + pub base: PlanBase, core: generic::Union, } diff --git a/src/frontend/src/optimizer/plan_node/logical_update.rs b/src/frontend/src/optimizer/plan_node/logical_update.rs index 1dbe1d3d3c5c9..dd841c5e6fb55 100644 --- a/src/frontend/src/optimizer/plan_node/logical_update.rs +++ b/src/frontend/src/optimizer/plan_node/logical_update.rs @@ -34,7 +34,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// It corresponds to the `UPDATE` statements in SQL. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalUpdate { - pub base: PlanBase, + pub base: PlanBase, core: generic::Update, } diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index e62c6400f2015..4bf59d8cc8db6 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -38,7 +38,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// `LogicalValues` builds rows according to a list of expressions #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalValues { - pub base: PlanBase, + pub base: PlanBase, rows: Arc<[Vec]>, } diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index a637315c21f93..ef9461d898941 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -45,9 +45,9 @@ use risingwave_pb::stream_plan::StreamNode as StreamPlanPb; use serde::Serialize; use smallvec::SmallVec; -use self::batch::BatchPlanRef; -use self::generic::{GenericPlanRef, PhysicalPlanRef}; -use self::stream::StreamPlanRef; +use self::batch::{BatchPlanRef, BatchSpecific}; +use self::generic::{GenericPlanRef, PhysicalPlanRef, PhysicalSpecific}; +use self::stream::{StreamPlanRef, StreamSpecific}; use self::utils::Distill; use super::property::{Distribution, FunctionalDependencySet, Order}; @@ -541,24 +541,24 @@ impl GenericPlanRef for PlanRef { } } -impl

PhysicalPlanRef for P +impl

PhysicalSpecific for P where P: Eq + Hash, P: StaticPlanNodeMeta, - P::Convention: PhysicalConventionMarker, + ::Extra: PhysicalSpecific, { fn distribution(&self) -> &Distribution { self.plan_base().distribution() } } -impl PhysicalPlanRef for PlanRef { +impl PhysicalSpecific for PlanRef { fn distribution(&self) -> &Distribution { self.plan_base().distribution() } } -impl

StreamPlanRef for P +impl

StreamSpecific for P where P: Eq + Hash, P: StaticPlanNodeMeta, @@ -576,7 +576,7 @@ where } } -impl StreamPlanRef for PlanRef { +impl StreamSpecific for PlanRef { fn append_only(&self) -> bool { self.plan_base().append_only() } @@ -590,7 +590,7 @@ impl StreamPlanRef for PlanRef { } } -impl

BatchPlanRef for P +impl

BatchSpecific for P where P: Eq + Hash, P: StaticPlanNodeMeta, @@ -600,7 +600,7 @@ where } } -impl BatchPlanRef for PlanRef { +impl BatchSpecific for PlanRef { fn order(&self) -> &Order { self.plan_base().order() } diff --git a/src/frontend/src/optimizer/plan_node/stream_dedup.rs b/src/frontend/src/optimizer/plan_node/stream_dedup.rs index 51b5e589e886e..b3c42e1cf82f0 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dedup.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dedup.rs @@ -27,7 +27,7 @@ use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamDedup { - pub base: PlanBase, + pub base: PlanBase, core: generic::Dedup, } diff --git a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs index bb18f9cffdf0f..e93755e6531c9 100644 --- a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs @@ -34,7 +34,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// inputs to be indexes. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamDeltaJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, /// The join condition must be equivalent to `logical.on`, but separated into equal and diff --git a/src/frontend/src/optimizer/plan_node/stream_dml.rs b/src/frontend/src/optimizer/plan_node/stream_dml.rs index 9b000974786e4..5fd0e84056462 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dml.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dml.rs @@ -24,7 +24,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamDml { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, column_descs: Vec, } diff --git a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs index a4b74f37208e7..ff643fb2d4797 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs @@ -28,7 +28,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamDynamicFilter { - pub base: PlanBase, + pub base: PlanBase, core: generic::DynamicFilter, cleaned_by_watermark: bool, } diff --git a/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs b/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs index d8c5a9635ce59..773127d22f7ce 100644 --- a/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs @@ -26,7 +26,7 @@ use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamEowcOverWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::OverWindow, } diff --git a/src/frontend/src/optimizer/plan_node/stream_exchange.rs b/src/frontend/src/optimizer/plan_node/stream_exchange.rs index 99e6c3c5161a1..b0385cb4e5a39 100644 --- a/src/frontend/src/optimizer/plan_node/stream_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/stream_exchange.rs @@ -27,7 +27,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// without changing its content. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamExchange { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, no_shuffle: bool, } diff --git a/src/frontend/src/optimizer/plan_node/stream_expand.rs b/src/frontend/src/optimizer/plan_node/stream_expand.rs index 5959b8d6be4d2..6cd9c543fd86d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_expand.rs +++ b/src/frontend/src/optimizer/plan_node/stream_expand.rs @@ -24,7 +24,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamExpand { - pub base: PlanBase, + pub base: PlanBase, core: generic::Expand, } diff --git a/src/frontend/src/optimizer/plan_node/stream_filter.rs b/src/frontend/src/optimizer/plan_node/stream_filter.rs index 0f000e6b8c0db..7ff5dc1dc2c12 100644 --- a/src/frontend/src/optimizer/plan_node/stream_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_filter.rs @@ -25,7 +25,7 @@ use crate::utils::Condition; /// `StreamFilter` implements [`super::LogicalFilter`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamFilter { - pub base: PlanBase, + pub base: PlanBase, core: generic::Filter, } diff --git a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs index 95fd72e9f6aa0..35ef51d0656b2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs +++ b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs @@ -30,7 +30,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamFsFetch { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, core: generic::Source, } diff --git a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs index 3e8f3c00206c4..d32fe485af695 100644 --- a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs @@ -27,7 +27,7 @@ use crate::PlanRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamGroupTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, /// an optional column index which is the vnode of each row computed by the input's consistent /// hash distribution diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index 55ab6b5906e59..9882b54069bf3 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -28,7 +28,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, IndexSet}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamHashAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, /// An optional column index which is the vnode of each row computed by the input's consistent diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs index 9d9c41425c4b1..f0ceb1cec0d81 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -38,7 +38,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// get output rows. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamHashJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, /// The join condition must be equivalent to `logical.on`, but separated into equal and diff --git a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs index e177be6942360..4806e3ef4ac5e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -28,7 +28,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// [`StreamHopWindow`] represents a hop window table function. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamHopWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::HopWindow, window_start_exprs: Vec, window_end_exprs: Vec, diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index d8972436d5c78..401adde487986 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -38,7 +38,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// Materializes a stream. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamMaterialize { - pub base: PlanBase, + pub base: PlanBase, /// Child of Materialize plan input: PlanRef, table: TableCatalog, diff --git a/src/frontend/src/optimizer/plan_node/stream_now.rs b/src/frontend/src/optimizer/plan_node/stream_now.rs index 91ebc344fa51d..7828dd32fa0d1 100644 --- a/src/frontend/src/optimizer/plan_node/stream_now.rs +++ b/src/frontend/src/optimizer/plan_node/stream_now.rs @@ -29,7 +29,7 @@ use crate::OptimizerContextRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamNow { - pub base: PlanBase, + pub base: PlanBase, } impl StreamNow { diff --git a/src/frontend/src/optimizer/plan_node/stream_over_window.rs b/src/frontend/src/optimizer/plan_node/stream_over_window.rs index 5a2f9d98f1340..ddec67db989df 100644 --- a/src/frontend/src/optimizer/plan_node/stream_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_over_window.rs @@ -27,7 +27,7 @@ use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamOverWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::OverWindow, } diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index c0ff0d1cf2f43..62ebc88a1e245 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -29,7 +29,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// rows. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamProject { - pub base: PlanBase, + pub base: PlanBase, core: generic::Project, /// All the watermark derivations, (input_column_index, output_column_index). And the /// derivation expression is the project's expression itself. diff --git a/src/frontend/src/optimizer/plan_node/stream_project_set.rs b/src/frontend/src/optimizer/plan_node/stream_project_set.rs index ba09d79c96c60..04857aeb843df 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project_set.rs @@ -25,7 +25,7 @@ use crate::utils::ColIndexMappingRewriteExt; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamProjectSet { - pub base: PlanBase, + pub base: PlanBase, core: generic::ProjectSet, /// All the watermark derivations, (input_column_idx, expr_idx). And the /// derivation expression is the project_set's expression itself. diff --git a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs index 083cb877cd4d6..b60016281972b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs +++ b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs @@ -22,7 +22,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamRowIdGen { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, row_id_index: usize, } diff --git a/src/frontend/src/optimizer/plan_node/stream_share.rs b/src/frontend/src/optimizer/plan_node/stream_share.rs index 3acf0b132805e..8b6da0472fd46 100644 --- a/src/frontend/src/optimizer/plan_node/stream_share.rs +++ b/src/frontend/src/optimizer/plan_node/stream_share.rs @@ -27,7 +27,7 @@ use crate::Explain; /// `StreamShare` will be translated into an `ExchangeNode` based on its distribution finally. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamShare { - pub base: PlanBase, + pub base: PlanBase, core: generic::Share, } diff --git a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs index 92d96fdf21b08..2fb30cba19eb2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs @@ -28,7 +28,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamSimpleAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, /// The index of `count(*)` in `agg_calls`. diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 32e9fb487910c..3e51a2b3c2338 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -50,7 +50,7 @@ const DOWNSTREAM_PK_KEY: &str = "primary_key"; /// [`StreamSink`] represents a table/connector sink at the very end of the graph. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamSink { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, sink_desc: SinkDesc, } diff --git a/src/frontend/src/optimizer/plan_node/stream_sort.rs b/src/frontend/src/optimizer/plan_node/stream_sort.rs index 41a56a0fd5df2..0059bc9c9559e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sort.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sort.rs @@ -29,7 +29,7 @@ use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamEowcSort { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, sort_column_index: usize, diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index ae66cf568118b..ab057fcb5f7cf 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -31,7 +31,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// [`StreamSource`] represents a table/connector source at the very beginning of the graph. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamSource { - pub base: PlanBase, + pub base: PlanBase, pub(crate) core: generic::Source, } diff --git a/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs index 474582ec877c7..06b8b793ba320 100644 --- a/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs @@ -32,7 +32,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// by `StreamSimpleAgg`. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamStatelessSimpleAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, } diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs index 965ca217a3369..2555724a88224 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -41,7 +41,7 @@ use crate::{Explain, TableCatalog}; /// creation request. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamTableScan { - pub base: PlanBase, + pub base: PlanBase, core: generic::Scan, batch_plan_id: PlanNodeId, chain_type: ChainType, diff --git a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs index 675dbeb9ab381..ad29cdd3cc4a5 100644 --- a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs @@ -33,7 +33,7 @@ use crate::utils::ColIndexMappingRewriteExt; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamTemporalJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, eq_join_predicate: EqJoinPredicate, } diff --git a/src/frontend/src/optimizer/plan_node/stream_topn.rs b/src/frontend/src/optimizer/plan_node/stream_topn.rs index 87890625f6be7..54687de42dcfd 100644 --- a/src/frontend/src/optimizer/plan_node/stream_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_topn.rs @@ -25,7 +25,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamTopN` implements [`super::LogicalTopN`] to find the top N elements with a heap #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, } diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs index 6d6dca2d8dd02..13fed2af618e2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_union.rs +++ b/src/frontend/src/optimizer/plan_node/stream_union.rs @@ -30,7 +30,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamUnion` implements [`super::LogicalUnion`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamUnion { - pub base: PlanBase, + pub base: PlanBase, core: generic::Union, } diff --git a/src/frontend/src/optimizer/plan_node/stream_values.rs b/src/frontend/src/optimizer/plan_node/stream_values.rs index f8cc5db851159..55b1b53c7202e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_values.rs +++ b/src/frontend/src/optimizer/plan_node/stream_values.rs @@ -28,7 +28,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamValues` implements `LogicalValues.to_stream()` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamValues { - pub base: PlanBase, + pub base: PlanBase, logical: LogicalValues, } diff --git a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs index 066bc9a234ca5..49c15c2ba8731 100644 --- a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs @@ -30,7 +30,7 @@ use crate::{TableCatalog, WithOptions}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamWatermarkFilter { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, watermark_descs: Vec, } @@ -54,7 +54,7 @@ impl StreamWatermarkFilter { Self::with_base(base, input, watermark_descs) } - fn with_base(base: PlanBase, input: PlanRef, watermark_descs: Vec) -> Self { + fn with_base(base: PlanBase, input: PlanRef, watermark_descs: Vec) -> Self { Self { base, input, From a74fe50045b20802e63c7347a32d0418eebdb8c2 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Mon, 23 Oct 2023 14:41:50 +0800 Subject: [PATCH 04/14] type safe refactor Signed-off-by: Bugen Zhao --- Cargo.lock | 1 + src/frontend/Cargo.toml | 1 + src/frontend/src/handler/explain.rs | 1 + src/frontend/src/optimizer/plan_node/batch.rs | 1 + .../plan_node/generic/dynamic_filter.rs | 2 +- .../src/optimizer/plan_node/generic/join.rs | 4 +- .../src/optimizer/plan_node/generic/mod.rs | 2 + src/frontend/src/optimizer/plan_node/mod.rs | 220 ++++++++++-------- .../src/optimizer/plan_node/plan_base.rs | 82 +++++-- .../src/optimizer/plan_node/stream.rs | 1 + .../src/optimizer/plan_node/stream_sink.rs | 6 +- 11 files changed, 202 insertions(+), 119 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b7e9b6c45ec06..6b02412d0ab75 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7474,6 +7474,7 @@ dependencies = [ "async-recursion", "async-trait", "auto_enums", + "auto_impl", "bk-tree", "bytes", "clap", diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 37f9f6326faea..bae582dd06e24 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -21,6 +21,7 @@ arrow-schema = { workspace = true } async-recursion = "1.0.5" async-trait = "0.1" auto_enums = { version = "0.8", features = ["futures03"] } +auto_impl = "1" bk-tree = "0.5.0" bytes = "1" clap = { version = "4", features = ["derive"] } diff --git a/src/frontend/src/handler/explain.rs b/src/frontend/src/handler/explain.rs index 5de7ec95b38bd..fe798f3fa2857 100644 --- a/src/frontend/src/handler/explain.rs +++ b/src/frontend/src/handler/explain.rs @@ -30,6 +30,7 @@ use super::create_table::{ use super::query::gen_batch_plan_by_statement; use super::RwPgResponse; use crate::handler::HandlerArgs; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{Convention, Explain}; use crate::optimizer::OptimizerContext; use crate::scheduler::worker_node_manager::WorkerNodeSelector; diff --git a/src/frontend/src/optimizer/plan_node/batch.rs b/src/frontend/src/optimizer/plan_node/batch.rs index f3e38f0e081c3..52f29a2dd1ecb 100644 --- a/src/frontend/src/optimizer/plan_node/batch.rs +++ b/src/frontend/src/optimizer/plan_node/batch.rs @@ -15,6 +15,7 @@ use super::generic::PhysicalPlanRef; use crate::optimizer::property::Order; +#[auto_impl::auto_impl(&)] pub trait BatchSpecific { fn order(&self) -> &Order; } diff --git a/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs b/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs index 85ffd922c43e7..a0b24fdda6039 100644 --- a/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs @@ -171,7 +171,7 @@ pub fn infer_left_internal_table_catalog( internal_table_catalog_builder.build(dist_keys, read_prefix_len_hint) } -pub fn infer_right_internal_table_catalog(input: &impl stream::StreamPlanRef) -> TableCatalog { +pub fn infer_right_internal_table_catalog(input: impl stream::StreamPlanRef) -> TableCatalog { let schema = input.schema(); // We require that the right table has distribution `Single` diff --git a/src/frontend/src/optimizer/plan_node/generic/join.rs b/src/frontend/src/optimizer/plan_node/generic/join.rs index 87c03cc14c8c9..2536cee984558 100644 --- a/src/frontend/src/optimizer/plan_node/generic/join.rs +++ b/src/frontend/src/optimizer/plan_node/generic/join.rs @@ -70,10 +70,10 @@ impl Join { } } -impl Join { +impl Join { /// Return stream hash join internal table catalog and degree table catalog. pub fn infer_internal_and_degree_table_catalog( - input: &PlanRef, + input: I, join_key_indices: Vec, dk_indices_in_jk: Vec, ) -> (TableCatalog, TableCatalog, Vec) { diff --git a/src/frontend/src/optimizer/plan_node/generic/mod.rs b/src/frontend/src/optimizer/plan_node/generic/mod.rs index 2ad91f80361b5..ed2a5725d634e 100644 --- a/src/frontend/src/optimizer/plan_node/generic/mod.rs +++ b/src/frontend/src/optimizer/plan_node/generic/mod.rs @@ -84,6 +84,7 @@ macro_rules! impl_distill_unit_from_fields { } pub(super) use impl_distill_unit_from_fields; +#[auto_impl::auto_impl(&)] pub trait GenericPlanRef: Eq + Hash { fn id(&self) -> PlanNodeId; fn schema(&self) -> &Schema; @@ -92,6 +93,7 @@ pub trait GenericPlanRef: Eq + Hash { fn ctx(&self) -> OptimizerContextRef; } +#[auto_impl::auto_impl(&)] pub trait PhysicalSpecific { fn distribution(&self) -> &Distribution; } diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index ef9461d898941..2b2736dc442d0 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -53,12 +53,12 @@ use super::property::{Distribution, FunctionalDependencySet, Order}; pub trait PlanNodeMeta { fn node_type(&self) -> PlanNodeType; - fn plan_base(&self) -> &PlanBase; + fn plan_base(&self) -> PlanBaseRef<'_>; fn convention(&self) -> Convention; } pub trait ConventionMarker: 'static + Sized { - type Extra: 'static + Clone + Debug; + type Extra: 'static + Eq + Hash + Clone + Debug; fn value() -> Convention; } @@ -97,19 +97,21 @@ pub trait StaticPlanNodeMeta { type Convention: ConventionMarker; const NODE_TYPE: PlanNodeType; - fn plan_base(&self) -> &PlanBase; + fn plan_base(&self) -> &PlanBase; + + fn plan_base_ref(&self) -> PlanBaseRef<'_>; } impl

PlanNodeMeta for P where - P: StaticPlanNodeMeta, + P: StaticPlanNodeMeta + 'static, { fn node_type(&self) -> PlanNodeType { P::NODE_TYPE } - fn plan_base(&self) -> &PlanBase { - P::plan_base(self) + fn plan_base(&self) -> PlanBaseRef<'_> { + StaticPlanNodeMeta::plan_base_ref(self) } fn convention(&self) -> Convention { @@ -484,7 +486,7 @@ impl PlanNodeMeta for PlanRef { self.0.node_type() } - fn plan_base(&self) -> &PlanBase { + fn plan_base(&self) -> PlanBaseRef<'_> { self.0.plan_base() } @@ -493,31 +495,31 @@ impl PlanNodeMeta for PlanRef { } } -/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. -impl

GenericPlanRef for P -where - P: StaticPlanNodeMeta + Eq + Hash, -{ - fn id(&self) -> PlanNodeId { - self.plan_base().id() - } +// /// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. +// impl

GenericPlanRef for P +// where +// P: StaticPlanNodeMeta + Eq + Hash, +// { +// fn id(&self) -> PlanNodeId { +// self.plan_base().id() +// } - fn schema(&self) -> &Schema { - self.plan_base().schema() - } +// fn schema(&self) -> &Schema { +// self.plan_base().schema() +// } - fn stream_key(&self) -> Option<&[usize]> { - self.plan_base().stream_key() - } +// fn stream_key(&self) -> Option<&[usize]> { +// self.plan_base().stream_key() +// } - fn ctx(&self) -> OptimizerContextRef { - self.plan_base().ctx() - } +// fn ctx(&self) -> OptimizerContextRef { +// self.plan_base().ctx() +// } - fn functional_dependency(&self) -> &FunctionalDependencySet { - self.plan_base().functional_dependency() - } -} +// fn functional_dependency(&self) -> &FunctionalDependencySet { +// self.plan_base().functional_dependency() +// } +// } impl GenericPlanRef for PlanRef { fn id(&self) -> PlanNodeId { @@ -525,11 +527,13 @@ impl GenericPlanRef for PlanRef { } fn schema(&self) -> &Schema { - self.plan_base().schema() + // self.plan_base().schema() + todo!() } fn stream_key(&self) -> Option<&[usize]> { - self.plan_base().stream_key() + // self.plan_base().stream_key() + todo!() } fn ctx(&self) -> OptimizerContextRef { @@ -537,44 +541,46 @@ impl GenericPlanRef for PlanRef { } fn functional_dependency(&self) -> &FunctionalDependencySet { - self.plan_base().functional_dependency() + // self.plan_base().functional_dependency() + todo!() } } -impl

PhysicalSpecific for P -where - P: Eq + Hash, - P: StaticPlanNodeMeta, - ::Extra: PhysicalSpecific, -{ - fn distribution(&self) -> &Distribution { - self.plan_base().distribution() - } -} +// impl

PhysicalSpecific for P +// where +// P: Eq + Hash, +// P: StaticPlanNodeMeta, +// ::Extra: PhysicalSpecific, +// { +// fn distribution(&self) -> &Distribution { +// self.plan_base().distribution() +// } +// } impl PhysicalSpecific for PlanRef { fn distribution(&self) -> &Distribution { - self.plan_base().distribution() + // self.plan_base().distribution() + todo!() } } -impl

StreamSpecific for P -where - P: Eq + Hash, - P: StaticPlanNodeMeta, -{ - fn append_only(&self) -> bool { - self.plan_base().append_only() - } +// impl

StreamSpecific for P +// where +// P: Eq + Hash, +// P: StaticPlanNodeMeta, +// { +// fn append_only(&self) -> bool { +// self.plan_base().append_only() +// } - fn emit_on_window_close(&self) -> bool { - self.plan_base().emit_on_window_close() - } +// fn emit_on_window_close(&self) -> bool { +// self.plan_base().emit_on_window_close() +// } - fn watermark_columns(&self) -> &FixedBitSet { - self.plan_base().watermark_columns() - } -} +// fn watermark_columns(&self) -> &FixedBitSet { +// self.plan_base().watermark_columns() +// } +// } impl StreamSpecific for PlanRef { fn append_only(&self) -> bool { @@ -586,23 +592,25 @@ impl StreamSpecific for PlanRef { } fn watermark_columns(&self) -> &FixedBitSet { - self.plan_base().watermark_columns() + // self.plan_base().watermark_columns() + todo!() } } -impl

BatchSpecific for P -where - P: Eq + Hash, - P: StaticPlanNodeMeta, -{ - fn order(&self) -> &Order { - self.plan_base().order() - } -} +// impl

BatchSpecific for P +// where +// P: Eq + Hash, +// P: StaticPlanNodeMeta, +// { +// fn order(&self) -> &Order { +// self.plan_base().order() +// } +// } impl BatchSpecific for PlanRef { fn order(&self) -> &Order { - self.plan_base().order() + // self.plan_base().order() + todo!() } } @@ -660,46 +668,46 @@ pub(crate) fn pretty_config() -> PrettyConfig { } impl dyn PlanNode { - pub fn id(&self) -> PlanNodeId { - self.plan_base().id() - } + // pub fn id(&self) -> PlanNodeId { + // self.plan_base().id() + // } - pub fn ctx(&self) -> OptimizerContextRef { - self.plan_base().ctx().clone() - } + // pub fn ctx(&self) -> OptimizerContextRef { + // self.plan_base().ctx().clone() + // } - pub fn schema(&self) -> &Schema { - self.plan_base().schema() - } + // pub fn schema(&self) -> &Schema { + // self.plan_base().schema() + // } - pub fn stream_key(&self) -> Option<&[usize]> { - self.plan_base().stream_key() - } + // pub fn stream_key(&self) -> Option<&[usize]> { + // self.plan_base().stream_key() + // } - pub fn order(&self) -> &Order { - self.plan_base().order() - } + // pub fn order(&self) -> &Order { + // self.plan_base().order() + // } - // TODO: avoid no manual delegation - pub fn distribution(&self) -> &Distribution { - self.plan_base().distribution() - } + // // TODO: avoid no manual delegation + // pub fn distribution(&self) -> &Distribution { + // self.plan_base().distribution() + // } - pub fn append_only(&self) -> bool { - self.plan_base().append_only() - } + // pub fn append_only(&self) -> bool { + // self.plan_base().append_only() + // } - pub fn emit_on_window_close(&self) -> bool { - self.plan_base().emit_on_window_close() - } + // pub fn emit_on_window_close(&self) -> bool { + // self.plan_base().emit_on_window_close() + // } - pub fn functional_dependency(&self) -> &FunctionalDependencySet { - self.plan_base().functional_dependency() - } + // pub fn functional_dependency(&self) -> &FunctionalDependencySet { + // self.plan_base().functional_dependency() + // } - pub fn watermark_columns(&self) -> &FixedBitSet { - self.plan_base().watermark_columns() - } + // pub fn watermark_columns(&self) -> &FixedBitSet { + // self.plan_base().watermark_columns() + // } /// Serialize the plan node and its children to a stream plan proto. /// @@ -1188,7 +1196,19 @@ macro_rules! impl_plan_node_meta { type Convention = $convention; const NODE_TYPE: PlanNodeType = PlanNodeType::[<$convention $name>]; - fn plan_base(&self) -> &PlanBase { + fn plan_base(&self) -> &PlanBase<$convention> { + &self.base + } + + fn plan_base_ref(&self) -> PlanBaseRef<'_> { + PlanBaseRef::$convention(&self.base) + } + } + + impl Deref for [<$convention $name>] { + type Target = PlanBase<$convention>; + + fn deref(&self) -> &Self::Target { &self.base } })* diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs index c9b4d9cf244f6..d776b4ef57687 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -118,20 +118,6 @@ pub struct PlanBase { extra: C::Extra, } -// impl PlanBase { -// fn physical_extra(&self) -> &PhysicalExtra { -// self.physical_extra -// .as_ref() -// .expect("access physical properties from logical plan node") -// } - -// fn physical_extra_mut(&mut self) -> &mut PhysicalExtra { -// self.physical_extra -// .as_mut() -// .expect("access physical properties from logical plan node") -// } -// } - impl generic::GenericPlanRef for PlanBase { fn id(&self) -> PlanNodeId { self.id @@ -329,8 +315,74 @@ impl PlanBase { // Mutators for testing only. #[cfg(test)] -impl PlanBase { +impl PlanBase { pub fn functional_dependency_mut(&mut self) -> &mut FunctionalDependencySet { &mut self.functional_dependency } } + +#[derive(PartialEq, Eq, Hash, Clone, Copy, Debug, enum_as_inner::EnumAsInner)] +pub enum PlanBaseRef<'a> { + Logical(&'a PlanBase), + Stream(&'a PlanBase), + Batch(&'a PlanBase), +} + +macro_rules! dispatch_plan_base { + ($self:ident, [$($convention:ident),+ $(,)?], $method:expr) => { + match *($self) { + $( + PlanBaseRef::$convention(plan) => $method(plan), + )+ + _ => panic!() // TODO + } + } +} + +impl GenericPlanRef for PlanBaseRef<'_> { + fn id(&self) -> PlanNodeId { + todo!() + } + + fn schema(&self) -> &Schema { + dispatch_plan_base!(self, [Logical, Stream, Batch], GenericPlanRef::schema) + } + + fn stream_key(&self) -> Option<&[usize]> { + todo!() + } + + fn functional_dependency(&self) -> &FunctionalDependencySet { + todo!() + } + + fn ctx(&self) -> OptimizerContextRef { + todo!() + } +} + +impl PhysicalSpecific for PlanBaseRef<'_> { + fn distribution(&self) -> &Distribution { + dispatch_plan_base!(self, [Stream, Batch], PhysicalSpecific::distribution) + } +} + +impl StreamSpecific for PlanBaseRef<'_> { + fn append_only(&self) -> bool { + dispatch_plan_base!(self, [Stream], StreamSpecific::append_only) + } + + fn emit_on_window_close(&self) -> bool { + dispatch_plan_base!(self, [Stream], StreamSpecific::emit_on_window_close) + } + + fn watermark_columns(&self) -> &FixedBitSet { + dispatch_plan_base!(self, [Stream], StreamSpecific::watermark_columns) + } +} + +impl BatchSpecific for PlanBaseRef<'_> { + fn order(&self) -> &Order { + dispatch_plan_base!(self, [Batch], BatchSpecific::order) + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index bd7103cfc82c5..a6e3a4d19a235 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -16,6 +16,7 @@ use fixedbitset::FixedBitSet; use super::generic::PhysicalPlanRef; +#[auto_impl::auto_impl(&)] pub trait StreamSpecific { fn append_only(&self) -> bool; fn emit_on_window_close(&self) -> bool; diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 3e51a2b3c2338..c70b3a6bab835 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -58,7 +58,11 @@ pub struct StreamSink { impl StreamSink { #[must_use] pub fn new(input: PlanRef, sink_desc: SinkDesc) -> Self { - let base = input.plan_base().clone_with_new_plan_id(); + let base = input + .plan_base() + .into_stream() + .expect("input should be stream plan") + .clone_with_new_plan_id(); Self { base, input, From 95f4dc556ab4ab9b482ba59bb6c9f5fe99b947e5 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Mon, 23 Oct 2023 15:03:55 +0800 Subject: [PATCH 05/14] remove specific traits Signed-off-by: Bugen Zhao --- src/frontend/src/optimizer/plan_node/batch.rs | 15 ++-- .../src/optimizer/plan_node/batch_delete.rs | 1 + .../src/optimizer/plan_node/batch_project.rs | 2 +- .../src/optimizer/plan_node/generic/mod.rs | 4 +- src/frontend/src/optimizer/plan_node/mod.rs | 12 +-- .../src/optimizer/plan_node/plan_base.rs | 77 ++++++++----------- .../src/optimizer/plan_node/stream.rs | 15 ++-- 7 files changed, 56 insertions(+), 70 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/batch.rs b/src/frontend/src/optimizer/plan_node/batch.rs index 52f29a2dd1ecb..1ec8315c85fc2 100644 --- a/src/frontend/src/optimizer/plan_node/batch.rs +++ b/src/frontend/src/optimizer/plan_node/batch.rs @@ -15,11 +15,6 @@ use super::generic::PhysicalPlanRef; use crate::optimizer::property::Order; -#[auto_impl::auto_impl(&)] -pub trait BatchSpecific { - fn order(&self) -> &Order; -} - /// A subtrait of [`PhysicalPlanRef`] for batch plans. /// /// Due to the lack of refactoring, all plan nodes currently implement this trait @@ -28,4 +23,12 @@ pub trait BatchSpecific { /// [`PhysicalPlanRef`]. /// /// [`GenericPlanRef`]: super::generic::GenericPlanRef -pub trait BatchPlanRef = PhysicalPlanRef + BatchSpecific; +#[auto_impl::auto_impl(&)] +pub trait BatchPlanRef: PhysicalPlanRef { + fn order(&self) -> &Order; +} + +pub mod prelude { + pub use super::super::generic::{GenericPlanRef, PhysicalPlanRef}; + pub use super::BatchPlanRef; +} diff --git a/src/frontend/src/optimizer/plan_node/batch_delete.rs b/src/frontend/src/optimizer/plan_node/batch_delete.rs index f250b1c2e1c15..1b575eabc2d1d 100644 --- a/src/frontend/src/optimizer/plan_node/batch_delete.rs +++ b/src/frontend/src/optimizer/plan_node/batch_delete.rs @@ -21,6 +21,7 @@ use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; use crate::optimizer::plan_node::ToLocalBatch; +use crate::optimizer::plan_node::generic::PhysicalPlanRef; use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchDelete` implements [`super::LogicalDelete`] diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index 0b85ab6cc556a..859062e425123 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -18,7 +18,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ProjectNode; use risingwave_pb::expr::ExprNode; -use super::generic::GenericPlanRef; +use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, diff --git a/src/frontend/src/optimizer/plan_node/generic/mod.rs b/src/frontend/src/optimizer/plan_node/generic/mod.rs index ed2a5725d634e..fe85e0919804e 100644 --- a/src/frontend/src/optimizer/plan_node/generic/mod.rs +++ b/src/frontend/src/optimizer/plan_node/generic/mod.rs @@ -94,12 +94,10 @@ pub trait GenericPlanRef: Eq + Hash { } #[auto_impl::auto_impl(&)] -pub trait PhysicalSpecific { +pub trait PhysicalPlanRef: GenericPlanRef { fn distribution(&self) -> &Distribution; } -pub trait PhysicalPlanRef = GenericPlanRef + PhysicalSpecific; - pub trait GenericPlanNode { fn functional_dependency(&self) -> FunctionalDependencySet; fn schema(&self) -> Schema; diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 2b2736dc442d0..e1e3c3cfabbf4 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -45,9 +45,9 @@ use risingwave_pb::stream_plan::StreamNode as StreamPlanPb; use serde::Serialize; use smallvec::SmallVec; -use self::batch::{BatchPlanRef, BatchSpecific}; -use self::generic::{GenericPlanRef, PhysicalPlanRef, PhysicalSpecific}; -use self::stream::{StreamPlanRef, StreamSpecific}; +use self::batch::BatchPlanRef; +use self::generic::{GenericPlanRef, PhysicalPlanRef}; +use self::stream::StreamPlanRef; use self::utils::Distill; use super::property::{Distribution, FunctionalDependencySet, Order}; @@ -557,7 +557,7 @@ impl GenericPlanRef for PlanRef { // } // } -impl PhysicalSpecific for PlanRef { +impl PhysicalPlanRef for PlanRef { fn distribution(&self) -> &Distribution { // self.plan_base().distribution() todo!() @@ -582,7 +582,7 @@ impl PhysicalSpecific for PlanRef { // } // } -impl StreamSpecific for PlanRef { +impl StreamPlanRef for PlanRef { fn append_only(&self) -> bool { self.plan_base().append_only() } @@ -607,7 +607,7 @@ impl StreamSpecific for PlanRef { // } // } -impl BatchSpecific for PlanRef { +impl BatchPlanRef for PlanRef { fn order(&self) -> &Order { // self.plan_base().order() todo!() diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs index d776b4ef57687..1db611984860a 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -32,6 +32,11 @@ struct PhysicalCommonExtra { dist: Distribution, } +trait GetPhysicalCommon { + fn physical(&self) -> &PhysicalCommonExtra; + fn physical_mut(&mut self) -> &mut PhysicalCommonExtra; +} + /// Extra fields for stream plan nodes. #[derive(Clone, Debug, PartialEq, Eq, Hash)] pub struct StreamExtra { @@ -48,23 +53,13 @@ pub struct StreamExtra { watermark_columns: FixedBitSet, } -impl generic::PhysicalSpecific for StreamExtra { - fn distribution(&self) -> &Distribution { - &self.physical.dist - } -} - -impl stream::StreamSpecific for StreamExtra { - fn append_only(&self) -> bool { - self.append_only - } - - fn emit_on_window_close(&self) -> bool { - self.emit_on_window_close +impl GetPhysicalCommon for StreamExtra { + fn physical(&self) -> &PhysicalCommonExtra { + &self.physical } - fn watermark_columns(&self) -> &FixedBitSet { - &self.watermark_columns + fn physical_mut(&mut self) -> &mut PhysicalCommonExtra { + &mut self.physical } } @@ -79,15 +74,13 @@ pub struct BatchExtra { order: Order, } -impl generic::PhysicalSpecific for BatchExtra { - fn distribution(&self) -> &Distribution { - &self.physical.dist +impl GetPhysicalCommon for BatchExtra { + fn physical(&self) -> &PhysicalCommonExtra { + &self.physical } -} -impl batch::BatchSpecific for BatchExtra { - fn order(&self) -> &Order { - &self.order + fn physical_mut(&mut self) -> &mut PhysicalCommonExtra { + &mut self.physical } } @@ -140,38 +133,32 @@ impl generic::GenericPlanRef for PlanBase { } } -impl generic::PhysicalSpecific for PlanBase +impl generic::PhysicalPlanRef for PlanBase where - C::Extra: generic::PhysicalSpecific, + C::Extra: GetPhysicalCommon, { fn distribution(&self) -> &Distribution { - &self.extra.distribution() + &self.extra.physical().dist } } -impl stream::StreamSpecific for PlanBase -where - C::Extra: stream::StreamSpecific, -{ +impl stream::StreamPlanRef for PlanBase { fn append_only(&self) -> bool { - self.extra.append_only() + self.extra.append_only } fn emit_on_window_close(&self) -> bool { - self.extra.emit_on_window_close() + self.extra.emit_on_window_close } fn watermark_columns(&self) -> &FixedBitSet { - &self.extra.watermark_columns() + &self.extra.watermark_columns } } -impl batch::BatchSpecific for PlanBase -where - C::Extra: batch::BatchSpecific, -{ +impl batch::BatchPlanRef for PlanBase { fn order(&self) -> &Order { - &self.extra.order() + &self.extra.order } } @@ -361,28 +348,28 @@ impl GenericPlanRef for PlanBaseRef<'_> { } } -impl PhysicalSpecific for PlanBaseRef<'_> { +impl PhysicalPlanRef for PlanBaseRef<'_> { fn distribution(&self) -> &Distribution { - dispatch_plan_base!(self, [Stream, Batch], PhysicalSpecific::distribution) + dispatch_plan_base!(self, [Stream, Batch], PhysicalPlanRef::distribution) } } -impl StreamSpecific for PlanBaseRef<'_> { +impl StreamPlanRef for PlanBaseRef<'_> { fn append_only(&self) -> bool { - dispatch_plan_base!(self, [Stream], StreamSpecific::append_only) + dispatch_plan_base!(self, [Stream], StreamPlanRef::append_only) } fn emit_on_window_close(&self) -> bool { - dispatch_plan_base!(self, [Stream], StreamSpecific::emit_on_window_close) + dispatch_plan_base!(self, [Stream], StreamPlanRef::emit_on_window_close) } fn watermark_columns(&self) -> &FixedBitSet { - dispatch_plan_base!(self, [Stream], StreamSpecific::watermark_columns) + dispatch_plan_base!(self, [Stream], StreamPlanRef::watermark_columns) } } -impl BatchSpecific for PlanBaseRef<'_> { +impl BatchPlanRef for PlanBaseRef<'_> { fn order(&self) -> &Order { - dispatch_plan_base!(self, [Batch], BatchSpecific::order) + dispatch_plan_base!(self, [Batch], BatchPlanRef::order) } } diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index a6e3a4d19a235..19f0da550d4a1 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -16,14 +16,6 @@ use fixedbitset::FixedBitSet; use super::generic::PhysicalPlanRef; -#[auto_impl::auto_impl(&)] -pub trait StreamSpecific { - fn append_only(&self) -> bool; - fn emit_on_window_close(&self) -> bool; - fn watermark_columns(&self) -> &FixedBitSet; -} - - /// A subtrait of [`PhysicalPlanRef`] for stream plans. /// /// Due to the lack of refactoring, all plan nodes currently implement this trait @@ -32,4 +24,9 @@ pub trait StreamSpecific { /// [`PhysicalPlanRef`]. /// /// [`GenericPlanRef`]: super::generic::GenericPlanRef -pub trait StreamPlanRef = PhysicalPlanRef + StreamSpecific; +#[auto_impl::auto_impl(&)] +pub trait StreamPlanRef: PhysicalPlanRef { + fn append_only(&self) -> bool; + fn emit_on_window_close(&self) -> bool; + fn watermark_columns(&self) -> &FixedBitSet; +} From c619cdf6a2b5081932698530b98b73eee6e603fa Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Mon, 23 Oct 2023 15:47:54 +0800 Subject: [PATCH 06/14] import prelude Signed-off-by: Bugen Zhao --- src/frontend/src/optimizer/mod.rs | 8 +++--- .../src/optimizer/plan_node/batch_delete.rs | 2 +- .../src/optimizer/plan_node/batch_expand.rs | 1 + .../src/optimizer/plan_node/batch_filter.rs | 1 + .../optimizer/plan_node/batch_group_topn.rs | 1 + .../src/optimizer/plan_node/batch_hash_agg.rs | 1 + .../optimizer/plan_node/batch_hash_join.rs | 1 + .../optimizer/plan_node/batch_hop_window.rs | 1 + .../src/optimizer/plan_node/batch_insert.rs | 1 + .../src/optimizer/plan_node/batch_limit.rs | 1 + .../optimizer/plan_node/batch_lookup_join.rs | 1 + .../plan_node/batch_nested_loop_join.rs | 1 + .../optimizer/plan_node/batch_over_window.rs | 1 + .../optimizer/plan_node/batch_project_set.rs | 1 + .../optimizer/plan_node/batch_simple_agg.rs | 1 + .../src/optimizer/plan_node/batch_sort.rs | 1 + .../src/optimizer/plan_node/batch_sort_agg.rs | 1 + .../src/optimizer/plan_node/batch_topn.rs | 1 + .../src/optimizer/plan_node/batch_union.rs | 1 + .../src/optimizer/plan_node/batch_update.rs | 1 + .../src/optimizer/plan_node/logical_agg.rs | 4 +++ .../src/optimizer/plan_node/logical_dedup.rs | 2 ++ .../src/optimizer/plan_node/logical_join.rs | 6 +++++ .../plan_node/logical_over_window.rs | 2 ++ .../src/optimizer/plan_node/logical_topn.rs | 2 ++ .../src/optimizer/plan_node/stream.rs | 5 ++++ .../optimizer/plan_node/stream_delta_join.rs | 1 + .../src/optimizer/plan_node/stream_dml.rs | 1 + .../plan_node/stream_dynamic_filter.rs | 1 + .../plan_node/stream_eowc_over_window.rs | 1 + .../src/optimizer/plan_node/stream_expand.rs | 1 + .../src/optimizer/plan_node/stream_filter.rs | 1 + .../optimizer/plan_node/stream_group_topn.rs | 1 + .../optimizer/plan_node/stream_hash_agg.rs | 1 + .../optimizer/plan_node/stream_hash_join.rs | 1 + .../optimizer/plan_node/stream_hop_window.rs | 1 + .../optimizer/plan_node/stream_materialize.rs | 1 + .../optimizer/plan_node/stream_over_window.rs | 1 + .../src/optimizer/plan_node/stream_project.rs | 1 + .../optimizer/plan_node/stream_project_set.rs | 1 + .../optimizer/plan_node/stream_row_id_gen.rs | 1 + .../src/optimizer/plan_node/stream_share.rs | 25 +++++++++++-------- .../src/optimizer/plan_node/stream_sink.rs | 1 + .../plan_node/stream_stateless_simple_agg.rs | 1 + .../plan_node/stream_temporal_join.rs | 1 + .../src/optimizer/plan_node/stream_topn.rs | 1 + .../src/optimizer/plan_node/stream_union.rs | 1 + .../plan_node/stream_watermark_filter.rs | 7 +++++- .../src/optimizer/property/distribution.rs | 1 + src/frontend/src/optimizer/property/order.rs | 2 ++ src/frontend/src/scheduler/plan_fragmenter.rs | 2 +- 51 files changed, 89 insertions(+), 17 deletions(-) diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 0df387b0a53d5..4004748a2f4f9 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -17,6 +17,7 @@ use std::ops::DerefMut; pub mod plan_node; pub use plan_node::{Explain, PlanRef}; + pub mod property; mod delta_join_solver; @@ -46,10 +47,11 @@ use risingwave_connector::sink::catalog::SinkFormatDesc; use risingwave_pb::catalog::WatermarkDesc; use self::heuristic_optimizer::ApplyOrder; +use self::plan_node::generic::{self, PhysicalPlanRef}; use self::plan_node::{ - generic, stream_enforce_eowc_requirement, BatchProject, Convention, LogicalProject, - LogicalSource, StreamDml, StreamMaterialize, StreamProject, StreamRowIdGen, StreamSink, - StreamWatermarkFilter, ToStreamContext, + stream_enforce_eowc_requirement, BatchProject, Convention, LogicalProject, LogicalSource, + StreamDml, StreamMaterialize, StreamProject, StreamRowIdGen, StreamSink, StreamWatermarkFilter, + ToStreamContext, }; #[cfg(debug_assertions)] use self::plan_visitor::InputRefValidator; diff --git a/src/frontend/src/optimizer/plan_node/batch_delete.rs b/src/frontend/src/optimizer/plan_node/batch_delete.rs index 1b575eabc2d1d..0d476229d582a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_delete.rs +++ b/src/frontend/src/optimizer/plan_node/batch_delete.rs @@ -20,8 +20,8 @@ use super::utils::impl_distill_by_unit; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; -use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::plan_node::generic::PhysicalPlanRef; +use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchDelete` implements [`super::LogicalDelete`] diff --git a/src/frontend/src/optimizer/plan_node/batch_expand.rs b/src/frontend/src/optimizer/plan_node/batch_expand.rs index 43ca3d9b33b4a..6dc46b8ee5b67 100644 --- a/src/frontend/src/optimizer/plan_node/batch_expand.rs +++ b/src/frontend/src/optimizer/plan_node/batch_expand.rs @@ -18,6 +18,7 @@ use risingwave_pb::batch_plan::expand_node::Subset; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ExpandNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable}; use crate::optimizer::plan_node::{ diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index e035e2fb538fb..0594a1f33dbce 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::FilterNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; use crate::expr::{Expr, ExprImpl, ExprRewriter}; diff --git a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs index fe9ad907796d5..b65cc73bf1b3b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::GroupTopNNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs index ca2246729e576..bc84a9daff42b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -17,6 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashAggNode; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef, PlanAggCall}; use super::utils::impl_distill_by_unit; use super::{ diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs index 1b95f50c9bf43..1e9a3b1fc5d10 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -18,6 +18,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashJoinNode; use risingwave_pb::plan_common::JoinType; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::{ diff --git a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs index 07480b0fc6a34..1eb1c4dd22d88 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HopWindowNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, diff --git a/src/frontend/src/optimizer/plan_node/batch_insert.rs b/src/frontend/src/optimizer/plan_node/batch_insert.rs index cdea2c3cd2b75..3f5c2c82dd6eb 100644 --- a/src/frontend/src/optimizer/plan_node/batch_insert.rs +++ b/src/frontend/src/optimizer/plan_node/batch_insert.rs @@ -18,6 +18,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::InsertNode; use risingwave_pb::plan_common::{DefaultColumns, IndexAndExpr}; +use super::batch::prelude::*; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; diff --git a/src/frontend/src/optimizer/plan_node/batch_limit.rs b/src/frontend/src/optimizer/plan_node/batch_limit.rs index 9e8f01ac3afbc..f6923cc59a012 100644 --- a/src/frontend/src/optimizer/plan_node/batch_limit.rs +++ b/src/frontend/src/optimizer/plan_node/batch_limit.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::LimitNode; +use super::batch::prelude::*; use super::generic::PhysicalPlanRef; use super::utils::impl_distill_by_unit; use super::{ diff --git a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs index 4d10ae67017bb..2ee7e438809c9 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -18,6 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{DistributedLookupJoinNode, LocalLookupJoinNode}; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::ExprRewritable; diff --git a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs index 9a1c06cd5c663..6e82f3b1b6bb5 100644 --- a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs @@ -17,6 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::NestedLoopJoinNode; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchPb, ToDistributedBatch}; diff --git a/src/frontend/src/optimizer/plan_node/batch_over_window.rs b/src/frontend/src/optimizer/plan_node/batch_over_window.rs index 77d50a3b2663c..362f687903a4f 100644 --- a/src/frontend/src/optimizer/plan_node/batch_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_over_window.rs @@ -17,6 +17,7 @@ use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortOverWindowNode; +use super::batch::prelude::*; use super::batch::BatchPlanRef; use super::generic::PlanWindowFunction; use super::utils::impl_distill_by_unit; diff --git a/src/frontend/src/optimizer/plan_node/batch_project_set.rs b/src/frontend/src/optimizer/plan_node/batch_project_set.rs index f8890b4e346ae..fad56834c26ab 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project_set.rs @@ -17,6 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ProjectSetNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable}; use crate::expr::ExprRewriter; diff --git a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs index 390bdc53df5be..cb133e0f2c606 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortAggNode; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef, PlanAggCall}; use super::utils::impl_distill_by_unit; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; diff --git a/src/frontend/src/optimizer/plan_node/batch_sort.rs b/src/frontend/src/optimizer/plan_node/batch_sort.rs index ac76445e08702..35ca522fe3613 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort.rs @@ -17,6 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortNode; +use super::batch::prelude::*; use super::batch::BatchPlanRef; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; diff --git a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs index af2f68b06deca..0c2d697047ae2 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -17,6 +17,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortAggNode; use risingwave_pb::expr::ExprNode; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef, PlanAggCall}; use super::utils::impl_distill_by_unit; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; diff --git a/src/frontend/src/optimizer/plan_node/batch_topn.rs b/src/frontend/src/optimizer/plan_node/batch_topn.rs index dcd3c1699a068..aab0476330931 100644 --- a/src/frontend/src/optimizer/plan_node/batch_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_topn.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::TopNNode; +use super::batch::prelude::*; use super::generic::TopNLimit; use super::utils::impl_distill_by_unit; use super::{ diff --git a/src/frontend/src/optimizer/plan_node/batch_union.rs b/src/frontend/src/optimizer/plan_node/batch_union.rs index af8cf4c9da104..5a0653caee045 100644 --- a/src/frontend/src/optimizer/plan_node/batch_union.rs +++ b/src/frontend/src/optimizer/plan_node/batch_union.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::UnionNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanRef, ToBatchPb, ToDistributedBatch}; use crate::optimizer::plan_node::{PlanBase, PlanTreeNode, ToLocalBatch}; diff --git a/src/frontend/src/optimizer/plan_node/batch_update.rs b/src/frontend/src/optimizer/plan_node/batch_update.rs index ee90c0ee4be9b..2169d7a3ebec0 100644 --- a/src/frontend/src/optimizer/plan_node/batch_update.rs +++ b/src/frontend/src/optimizer/plan_node/batch_update.rs @@ -18,6 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::UpdateNode; +use super::batch::prelude::*; use super::generic::GenericPlanRef; use super::utils::impl_distill_by_unit; use super::{ diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index ea0e45460a5cc..e6809a921ef72 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -182,6 +182,8 @@ impl LogicalAgg { /// Generates distributed stream plan. fn gen_dist_stream_agg_plan(&self, stream_input: PlanRef) -> Result { + use super::stream::prelude::*; + let input_dist = stream_input.distribution(); debug_assert!(*input_dist != Distribution::Broadcast); @@ -1137,6 +1139,8 @@ fn new_stream_hash_agg(core: Agg, vnode_col_idx: Option) -> Stre impl ToStream for LogicalAgg { fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { + use super::stream::prelude::*; + for agg_call in self.agg_calls() { if matches!(agg_call.agg_kind, agg_kinds::unimplemented_in_stream!()) { return Err(ErrorCode::NotImplemented( diff --git a/src/frontend/src/optimizer/plan_node/logical_dedup.rs b/src/frontend/src/optimizer/plan_node/logical_dedup.rs index de1678211f536..a3bddd612a832 100644 --- a/src/frontend/src/optimizer/plan_node/logical_dedup.rs +++ b/src/frontend/src/optimizer/plan_node/logical_dedup.rs @@ -99,6 +99,8 @@ impl ToStream for LogicalDedup { } fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { + use super::stream::prelude::*; + let input = self.input().to_stream(ctx)?; let input = RequiredDist::hash_shard(self.dedup_cols()) .enforce_if_not_satisfies(input, &Order::any())?; diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index 96a9ba631241f..c595ecb45493d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -866,6 +866,8 @@ impl LogicalJoin { predicate: EqJoinPredicate, ctx: &mut ToStreamContext, ) -> Result { + use super::stream::prelude::*; + assert!(predicate.has_eq()); let mut right = self.right().to_stream_with_dist_required( &RequiredDist::shard_by_key(self.right().schema().len(), &predicate.right_eq_indexes()), @@ -1009,6 +1011,8 @@ impl LogicalJoin { predicate: EqJoinPredicate, ctx: &mut ToStreamContext, ) -> Result { + use super::stream::prelude::*; + assert!(predicate.has_eq()); let right = self.right(); @@ -1179,6 +1183,8 @@ impl LogicalJoin { predicate: Condition, ctx: &mut ToStreamContext, ) -> Result> { + use super::stream::prelude::*; + // If there is exactly one predicate, it is a comparison (<, <=, >, >=), and the // join is a `Inner` or `LeftSemi` join, we can convert the scalar subquery into a // `StreamDynamicFilter` diff --git a/src/frontend/src/optimizer/plan_node/logical_over_window.rs b/src/frontend/src/optimizer/plan_node/logical_over_window.rs index 1f82c34b0ee9a..8f0d68924a3ed 100644 --- a/src/frontend/src/optimizer/plan_node/logical_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_over_window.rs @@ -772,6 +772,8 @@ impl ToBatch for LogicalOverWindow { impl ToStream for LogicalOverWindow { fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { + use super::stream::prelude::*; + if self .window_functions() .iter() diff --git a/src/frontend/src/optimizer/plan_node/logical_topn.rs b/src/frontend/src/optimizer/plan_node/logical_topn.rs index 58a66c2fec47d..25d02b65f1f1f 100644 --- a/src/frontend/src/optimizer/plan_node/logical_topn.rs +++ b/src/frontend/src/optimizer/plan_node/logical_topn.rs @@ -107,6 +107,8 @@ impl LogicalTopN { } fn gen_dist_stream_top_n_plan(&self, stream_input: PlanRef) -> Result { + use super::stream::prelude::*; + let input_dist = stream_input.distribution().clone(); // if it is append only, for now we don't generate 2-phase rules diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 19f0da550d4a1..33e5eb5fda300 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -30,3 +30,8 @@ pub trait StreamPlanRef: PhysicalPlanRef { fn emit_on_window_close(&self) -> bool; fn watermark_columns(&self) -> &FixedBitSet; } + +pub mod prelude { + pub use super::super::generic::{GenericPlanRef, PhysicalPlanRef}; + pub use super::StreamPlanRef; +} diff --git a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs index e93755e6531c9..19a7ee02caeaf 100644 --- a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs @@ -21,6 +21,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ArrangementInfo, DeltaIndexJoinNode}; use super::generic::{self, GenericPlanRef}; +use super::stream::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, StreamNode}; use crate::expr::{Expr, ExprRewriter}; diff --git a/src/frontend/src/optimizer/plan_node/stream_dml.rs b/src/frontend/src/optimizer/plan_node/stream_dml.rs index 5fd0e84056462..1b7b72e90358b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dml.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dml.rs @@ -17,6 +17,7 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{ColumnDesc, INITIAL_TABLE_VERSION_ID}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; diff --git a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs index ff643fb2d4797..f648e856432ed 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs @@ -18,6 +18,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::DynamicFilterNode; use super::generic::{DynamicFilter, GenericPlanRef}; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, column_names_pretty, watermark_pretty, Distill}; use super::{generic, ExprRewritable}; diff --git a/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs b/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs index 773127d22f7ce..278f6062e56cd 100644 --- a/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs @@ -19,6 +19,7 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{self, GenericPlanRef, PlanWindowFunction}; +use super::stream::prelude::*; use super::utils::{impl_distill_by_unit, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::stream_fragmenter::BuildFragmentGraphState; diff --git a/src/frontend/src/optimizer/plan_node/stream_expand.rs b/src/frontend/src/optimizer/plan_node/stream_expand.rs index 6cd9c543fd86d..70b2b145820e8 100644 --- a/src/frontend/src/optimizer/plan_node/stream_expand.rs +++ b/src/frontend/src/optimizer/plan_node/stream_expand.rs @@ -17,6 +17,7 @@ use risingwave_pb::stream_plan::expand_node::Subset; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::ExpandNode; +use super::stream::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::Distribution; diff --git a/src/frontend/src/optimizer/plan_node/stream_filter.rs b/src/frontend/src/optimizer/plan_node/stream_filter.rs index 7ff5dc1dc2c12..956b6593bac0d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_filter.rs @@ -15,6 +15,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::FilterNode; +use super::stream::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::{Expr, ExprImpl, ExprRewriter}; diff --git a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs index d32fe485af695..4bd3eb92663da 100644 --- a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs @@ -17,6 +17,7 @@ use pretty_xmlish::XmlNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{DistillUnit, GenericPlanRef, TopNLimit}; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{plan_node_name, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanTreeNodeUnary, StreamNode}; diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index 9882b54069bf3..8038109e2352a 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -19,6 +19,7 @@ use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{self, GenericPlanRef, PlanAggCall}; +use super::stream::prelude::*; use super::utils::{childless_record, plan_node_name, watermark_pretty, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs index f0ceb1cec0d81..3c41ab9245af7 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -21,6 +21,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{DeltaExpression, HashJoinNode, PbInequalityPair}; use super::generic::{GenericPlanRef, Join}; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, plan_node_name, watermark_pretty, Distill}; use super::{ diff --git a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs index 4806e3ef4ac5e..615e4e23ed763 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -18,6 +18,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::HopWindowNode; use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 401adde487986..0bca63f849426 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -24,6 +24,7 @@ use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::derive::derive_columns; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, Distill}; use super::{reorganize_elements_id, ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamNode}; diff --git a/src/frontend/src/optimizer/plan_node/stream_over_window.rs b/src/frontend/src/optimizer/plan_node/stream_over_window.rs index ddec67db989df..5cffe6e0c0fd8 100644 --- a/src/frontend/src/optimizer/plan_node/stream_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_over_window.rs @@ -19,6 +19,7 @@ use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{GenericPlanNode, PlanWindowFunction}; +use super::stream::prelude::*; use super::utils::{impl_distill_by_unit, TableCatalogBuilder}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::plan_node::generic::GenericPlanRef; diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index 62ebc88a1e245..21d8e5f033900 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -18,6 +18,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::ProjectNode; use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; diff --git a/src/frontend/src/optimizer/plan_node/stream_project_set.rs b/src/frontend/src/optimizer/plan_node/stream_project_set.rs index 04857aeb843df..ba9fb16acca33 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project_set.rs @@ -17,6 +17,7 @@ use itertools::Itertools; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::ProjectSetNode; +use super::stream::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::{try_derive_watermark, ExprRewriter, WatermarkDerivation}; diff --git a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs index b60016281972b..d3f8eaebdf3ab 100644 --- a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs +++ b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs @@ -15,6 +15,7 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; +use super::stream::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::Distribution; diff --git a/src/frontend/src/optimizer/plan_node/stream_share.rs b/src/frontend/src/optimizer/plan_node/stream_share.rs index 8b6da0472fd46..ae68c6fc6651c 100644 --- a/src/frontend/src/optimizer/plan_node/stream_share.rs +++ b/src/frontend/src/optimizer/plan_node/stream_share.rs @@ -17,7 +17,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::PbStreamNode; use super::generic::GenericPlanRef; -use super::stream::StreamPlanRef; +use super::stream::prelude::*; use super::utils::Distill; use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamExchange, StreamNode}; use crate::optimizer::plan_node::{LogicalShare, PlanBase, PlanTreeNode}; @@ -33,16 +33,19 @@ pub struct StreamShare { impl StreamShare { pub fn new(core: generic::Share) -> Self { - let input = core.input.borrow().0.clone(); - let dist = input.distribution().clone(); - // Filter executor won't change the append-only behavior of the stream. - let base = PlanBase::new_stream_with_core( - &core, - dist, - input.append_only(), - input.emit_on_window_close(), - input.watermark_columns().clone(), - ); + let base = { + let input = core.input.borrow(); + let dist = input.distribution().clone(); + // Filter executor won't change the append-only behavior of the stream. + PlanBase::new_stream_with_core( + &core, + dist, + input.append_only(), + input.emit_on_window_close(), + input.watermark_columns().clone(), + ) + }; + StreamShare { base, core } } } diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index c70b3a6bab835..3f2f927cf0114 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -38,6 +38,7 @@ use tracing::info; use super::derive::{derive_columns, derive_pk}; use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::utils::{childless_record, Distill, IndicesDisplay, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, StreamNode}; use crate::optimizer::plan_node::PlanTreeNodeUnary; diff --git a/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs index 06b8b793ba320..510d76138877f 100644 --- a/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs @@ -17,6 +17,7 @@ use itertools::Itertools; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{self, PlanAggCall}; +use super::stream::prelude::*; use super::utils::impl_distill_by_unit; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; diff --git a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs index ad29cdd3cc4a5..bf74344e8ef12 100644 --- a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs @@ -19,6 +19,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::TemporalJoinNode; use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, StreamNode}; diff --git a/src/frontend/src/optimizer/plan_node/stream_topn.rs b/src/frontend/src/optimizer/plan_node/stream_topn.rs index 54687de42dcfd..37acf93a729a0 100644 --- a/src/frontend/src/optimizer/plan_node/stream_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_topn.rs @@ -17,6 +17,7 @@ use pretty_xmlish::XmlNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{DistillUnit, TopNLimit}; +use super::stream::prelude::*; use super::utils::{plan_node_name, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::{Distribution, Order}; diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs index 13fed2af618e2..50de3c35ba8fb 100644 --- a/src/frontend/src/optimizer/plan_node/stream_union.rs +++ b/src/frontend/src/optimizer/plan_node/stream_union.rs @@ -20,6 +20,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::UnionNode; use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanRef}; diff --git a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs index 49c15c2ba8731..84d830a898f83 100644 --- a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs @@ -21,6 +21,7 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_pb::catalog::WatermarkDesc; use risingwave_pb::stream_plan::stream_node::PbNodeBody; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -54,7 +55,11 @@ impl StreamWatermarkFilter { Self::with_base(base, input, watermark_descs) } - fn with_base(base: PlanBase, input: PlanRef, watermark_descs: Vec) -> Self { + fn with_base( + base: PlanBase, + input: PlanRef, + watermark_descs: Vec, + ) -> Self { Self { base, input, diff --git a/src/frontend/src/optimizer/property/distribution.rs b/src/frontend/src/optimizer/property/distribution.rs index b6e7715dd155f..7b958b047d196 100644 --- a/src/frontend/src/optimizer/property/distribution.rs +++ b/src/frontend/src/optimizer/property/distribution.rs @@ -47,6 +47,7 @@ use std::fmt; use std::fmt::Debug; use fixedbitset::FixedBitSet; +use generic::PhysicalPlanRef; use itertools::Itertools; use risingwave_common::catalog::{FieldDisplay, Schema, TableId}; use risingwave_common::error::Result; diff --git a/src/frontend/src/optimizer/property/order.rs b/src/frontend/src/optimizer/property/order.rs index 19ad7586e1c11..bf373e8629d7b 100644 --- a/src/frontend/src/optimizer/property/order.rs +++ b/src/frontend/src/optimizer/property/order.rs @@ -85,6 +85,8 @@ const ANY_ORDER: Order = Order { impl Order { pub fn enforce_if_not_satisfies(&self, plan: PlanRef) -> Result { + use crate::optimizer::plan_node::batch::prelude::*; + if !plan.order().satisfies(self) { Ok(self.enforce(plan)) } else { diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index cb20103b3e76f..cc23256f43687 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -42,7 +42,7 @@ use uuid::Uuid; use super::SchedulerError; use crate::catalog::catalog_service::CatalogReader; use crate::catalog::TableId; -use crate::optimizer::plan_node::generic::GenericPlanRef; +use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef}; use crate::optimizer::plan_node::{PlanNodeId, PlanNodeType}; use crate::optimizer::property::Distribution; use crate::optimizer::PlanRef; From dd75d1ec417ea77e8c07d0c88f48eeaa5149b93b Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Mon, 23 Oct 2023 15:49:06 +0800 Subject: [PATCH 07/14] fix lifetime Signed-off-by: Bugen Zhao --- src/frontend/src/optimizer/plan_node/mod.rs | 144 ++++-------------- .../src/optimizer/plan_node/plan_base.rs | 49 ++++-- 2 files changed, 68 insertions(+), 125 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index e1e3c3cfabbf4..fd3262b39a835 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -161,7 +161,6 @@ impl_downcast!(PlanNode); // Using a new type wrapper allows direct function implementation on `PlanRef`, // and we currently need a manual implementation of `PartialEq` for `PlanRef`. -#[allow(clippy::derived_hash_with_manual_eq)] #[derive(Clone, Debug, Eq, Hash)] pub struct PlanRef(Rc); @@ -495,45 +494,17 @@ impl PlanNodeMeta for PlanRef { } } -// /// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. -// impl

GenericPlanRef for P -// where -// P: StaticPlanNodeMeta + Eq + Hash, -// { -// fn id(&self) -> PlanNodeId { -// self.plan_base().id() -// } - -// fn schema(&self) -> &Schema { -// self.plan_base().schema() -// } - -// fn stream_key(&self) -> Option<&[usize]> { -// self.plan_base().stream_key() -// } - -// fn ctx(&self) -> OptimizerContextRef { -// self.plan_base().ctx() -// } - -// fn functional_dependency(&self) -> &FunctionalDependencySet { -// self.plan_base().functional_dependency() -// } -// } - impl GenericPlanRef for PlanRef { fn id(&self) -> PlanNodeId { self.plan_base().id() } fn schema(&self) -> &Schema { - // self.plan_base().schema() - todo!() + self.plan_base().schema() } - fn stream_key(&self) -> Option<&[usize]> { - // self.plan_base().stream_key() - todo!() + fn stream_key<'a>(&'a self) -> Option<&'a [usize]> { + self.plan_base().stream_key() } fn ctx(&self) -> OptimizerContextRef { @@ -541,47 +512,16 @@ impl GenericPlanRef for PlanRef { } fn functional_dependency(&self) -> &FunctionalDependencySet { - // self.plan_base().functional_dependency() - todo!() + self.plan_base().functional_dependency() } } -// impl

PhysicalSpecific for P -// where -// P: Eq + Hash, -// P: StaticPlanNodeMeta, -// ::Extra: PhysicalSpecific, -// { -// fn distribution(&self) -> &Distribution { -// self.plan_base().distribution() -// } -// } - impl PhysicalPlanRef for PlanRef { fn distribution(&self) -> &Distribution { - // self.plan_base().distribution() - todo!() + self.plan_base().distribution() } } -// impl

StreamSpecific for P -// where -// P: Eq + Hash, -// P: StaticPlanNodeMeta, -// { -// fn append_only(&self) -> bool { -// self.plan_base().append_only() -// } - -// fn emit_on_window_close(&self) -> bool { -// self.plan_base().emit_on_window_close() -// } - -// fn watermark_columns(&self) -> &FixedBitSet { -// self.plan_base().watermark_columns() -// } -// } - impl StreamPlanRef for PlanRef { fn append_only(&self) -> bool { self.plan_base().append_only() @@ -592,25 +532,13 @@ impl StreamPlanRef for PlanRef { } fn watermark_columns(&self) -> &FixedBitSet { - // self.plan_base().watermark_columns() - todo!() + self.plan_base().watermark_columns() } } -// impl

BatchSpecific for P -// where -// P: Eq + Hash, -// P: StaticPlanNodeMeta, -// { -// fn order(&self) -> &Order { -// self.plan_base().order() -// } -// } - impl BatchPlanRef for PlanRef { fn order(&self) -> &Order { - // self.plan_base().order() - todo!() + self.plan_base().order() } } @@ -667,53 +595,37 @@ pub(crate) fn pretty_config() -> PrettyConfig { } } +// TODO: remove this direct implementation always require `GenericPlanRef`. impl dyn PlanNode { - // pub fn id(&self) -> PlanNodeId { - // self.plan_base().id() - // } - - // pub fn ctx(&self) -> OptimizerContextRef { - // self.plan_base().ctx().clone() - // } - - // pub fn schema(&self) -> &Schema { - // self.plan_base().schema() - // } - - // pub fn stream_key(&self) -> Option<&[usize]> { - // self.plan_base().stream_key() - // } - - // pub fn order(&self) -> &Order { - // self.plan_base().order() - // } - - // // TODO: avoid no manual delegation - // pub fn distribution(&self) -> &Distribution { - // self.plan_base().distribution() - // } + pub fn id(&self) -> PlanNodeId { + self.plan_base().id() + } - // pub fn append_only(&self) -> bool { - // self.plan_base().append_only() - // } + pub fn ctx(&self) -> OptimizerContextRef { + self.plan_base().ctx().clone() + } - // pub fn emit_on_window_close(&self) -> bool { - // self.plan_base().emit_on_window_close() - // } + pub fn schema(&self) -> &Schema { + self.plan_base().schema() + } - // pub fn functional_dependency(&self) -> &FunctionalDependencySet { - // self.plan_base().functional_dependency() - // } + pub fn stream_key(&self) -> Option<&[usize]> { + self.plan_base().stream_key() + } - // pub fn watermark_columns(&self) -> &FixedBitSet { - // self.plan_base().watermark_columns() - // } + pub fn functional_dependency(&self) -> &FunctionalDependencySet { + self.plan_base().functional_dependency() + } +} +impl dyn PlanNode { /// Serialize the plan node and its children to a stream plan proto. /// /// Note that [`StreamTableScan`] has its own implementation of `to_stream_prost`. We have a /// hook inside to do some ad-hoc thing for [`StreamTableScan`]. pub fn to_stream_prost(&self, state: &mut BuildFragmentGraphState) -> StreamPlanPb { + use stream::prelude::*; + if let Some(stream_table_scan) = self.as_stream_table_scan() { return stream_table_scan.adhoc_to_stream_prost(state); } @@ -740,7 +652,7 @@ impl dyn PlanNode { .map(|x| *x as u32) .collect(), fields: self.schema().to_prost(), - append_only: self.append_only(), + append_only: self.plan_base().append_only(), } } diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs index 1db611984860a..e73ef59f63de8 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -317,40 +317,71 @@ pub enum PlanBaseRef<'a> { macro_rules! dispatch_plan_base { ($self:ident, [$($convention:ident),+ $(,)?], $method:expr) => { - match *($self) { + match $self { $( PlanBaseRef::$convention(plan) => $method(plan), )+ + #[allow(unreachable_patterns)] _ => panic!() // TODO } } } +impl<'a> PlanBaseRef<'a> { + pub(super) fn schema(self) -> &'a Schema { + dispatch_plan_base!(self, [Logical, Stream, Batch], GenericPlanRef::schema) + } + + pub(super) fn stream_key(self) -> Option<&'a [usize]> { + dispatch_plan_base!(self, [Logical, Stream, Batch], GenericPlanRef::stream_key) + } + + pub(super) fn functional_dependency(self) -> &'a FunctionalDependencySet { + dispatch_plan_base!( + self, + [Logical, Stream, Batch], + GenericPlanRef::functional_dependency + ) + } + + pub(super) fn distribution(self) -> &'a Distribution { + dispatch_plan_base!(self, [Stream, Batch], PhysicalPlanRef::distribution) + } + + pub(super) fn watermark_columns(self) -> &'a FixedBitSet { + dispatch_plan_base!(self, [Stream], StreamPlanRef::watermark_columns) + } + + pub(super) fn order(self) -> &'a Order { + dispatch_plan_base!(self, [Batch], BatchPlanRef::order) + } +} + impl GenericPlanRef for PlanBaseRef<'_> { fn id(&self) -> PlanNodeId { - todo!() + dispatch_plan_base!(self, [Logical, Stream, Batch], GenericPlanRef::id) } fn schema(&self) -> &Schema { - dispatch_plan_base!(self, [Logical, Stream, Batch], GenericPlanRef::schema) + (*self).schema() } fn stream_key(&self) -> Option<&[usize]> { - todo!() + (*self).stream_key() } fn functional_dependency(&self) -> &FunctionalDependencySet { - todo!() + (*self).functional_dependency() } fn ctx(&self) -> OptimizerContextRef { - todo!() + dispatch_plan_base!(self, [Logical, Stream, Batch], GenericPlanRef::ctx) } } impl PhysicalPlanRef for PlanBaseRef<'_> { fn distribution(&self) -> &Distribution { - dispatch_plan_base!(self, [Stream, Batch], PhysicalPlanRef::distribution) + (*self).distribution() } } @@ -364,12 +395,12 @@ impl StreamPlanRef for PlanBaseRef<'_> { } fn watermark_columns(&self) -> &FixedBitSet { - dispatch_plan_base!(self, [Stream], StreamPlanRef::watermark_columns) + (*self).watermark_columns() } } impl BatchPlanRef for PlanBaseRef<'_> { fn order(&self) -> &Order { - dispatch_plan_base!(self, [Batch], BatchPlanRef::order) + (*self).order() } } From 43fd89b6d9c271c7550674085a2d56ced2502d17 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Mon, 23 Oct 2023 16:09:49 +0800 Subject: [PATCH 08/14] refine docs Signed-off-by: Bugen Zhao --- src/frontend/src/optimizer/plan_node/mod.rs | 5 +- .../src/optimizer/plan_node/plan_base.rs | 69 ++++++++++++------- 2 files changed, 45 insertions(+), 29 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index fd3262b39a835..fd72155cec3b3 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -62,7 +62,6 @@ pub trait ConventionMarker: 'static + Sized { fn value() -> Convention; } -pub trait PhysicalConventionMarker: ConventionMarker {} pub struct Logical; impl ConventionMarker for Logical { @@ -81,7 +80,6 @@ impl ConventionMarker for Batch { Convention::Batch } } -impl PhysicalConventionMarker for Batch {} pub struct Stream; impl ConventionMarker for Stream { @@ -91,7 +89,6 @@ impl ConventionMarker for Stream { Convention::Stream } } -impl PhysicalConventionMarker for Stream {} pub trait StaticPlanNodeMeta { type Convention: ConventionMarker; @@ -104,7 +101,7 @@ pub trait StaticPlanNodeMeta { impl

PlanNodeMeta for P where - P: StaticPlanNodeMeta + 'static, + P: StaticPlanNodeMeta, { fn node_type(&self) -> PlanNodeType { P::NODE_TYPE diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs index e73ef59f63de8..d97f2bb988c9f 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -24,19 +24,27 @@ use crate::optimizer::property::{Distribution, FunctionalDependencySet, Order}; #[derive(Clone, Debug, PartialEq, Eq, Hash)] pub struct NoExtra; -/// Common extra fields for physical plan nodes. -#[derive(Clone, Debug, PartialEq, Eq, Hash)] -struct PhysicalCommonExtra { - /// The distribution property of the PlanNode's output, store an `Distribution::any()` here - /// will not affect correctness, but insert unnecessary exchange in plan - dist: Distribution, -} +// Make them public types in a private module to allow using them as public trait bounds, +// while still keeping them private to the super module. +mod physical_common { + use super::*; + + /// Common extra fields for physical plan nodes. + #[derive(Clone, Debug, PartialEq, Eq, Hash)] + pub struct PhysicalCommonExtra { + /// The distribution property of the PlanNode's output, store an `Distribution::any()` here + /// will not affect correctness, but insert unnecessary exchange in plan + pub dist: Distribution, + } -trait GetPhysicalCommon { - fn physical(&self) -> &PhysicalCommonExtra; - fn physical_mut(&mut self) -> &mut PhysicalCommonExtra; + pub trait GetPhysicalCommon { + fn physical(&self) -> &PhysicalCommonExtra; + fn physical_mut(&mut self) -> &mut PhysicalCommonExtra; + } } +use physical_common::*; + /// Extra fields for stream plan nodes. #[derive(Clone, Debug, PartialEq, Eq, Hash)] pub struct StreamExtra { @@ -277,25 +285,16 @@ impl PlanBase { } } -impl PlanBase { - /// Clone the plan node with a new distribution. - /// - /// Panics if the plan node is not physical. - pub fn clone_with_new_distribution(&self, dist: Distribution) -> Self { - let mut new = self.clone(); - new.extra.physical.dist = dist; - new - } -} - -// TODO: unify the impls for `PlanBase` and `PlanBase`. -impl PlanBase { +impl PlanBase +where + C::Extra: GetPhysicalCommon, +{ /// Clone the plan node with a new distribution. /// /// Panics if the plan node is not physical. pub fn clone_with_new_distribution(&self, dist: Distribution) -> Self { let mut new = self.clone(); - new.extra.physical.dist = dist; + new.extra.physical_mut().dist = dist; new } } @@ -315,18 +314,38 @@ pub enum PlanBaseRef<'a> { Batch(&'a PlanBase), } +impl PlanBaseRef<'_> { + pub fn convention(self) -> Convention { + match self { + PlanBaseRef::Logical(_) => Convention::Logical, + PlanBaseRef::Stream(_) => Convention::Stream, + PlanBaseRef::Batch(_) => Convention::Batch, + } + } +} + +/// Dispatch a method call to the corresponding plan base type. macro_rules! dispatch_plan_base { ($self:ident, [$($convention:ident),+ $(,)?], $method:expr) => { match $self { $( PlanBaseRef::$convention(plan) => $method(plan), )+ + #[allow(unreachable_patterns)] - _ => panic!() // TODO + _ => unreachable!("calling `{}` on a plan node of `{:?}`", stringify!($method), $self.convention()), } } } +/// Workaround for getters returning references. +/// +/// For example, callers writing `GenericPlanRef::schema(&foo.plan_base())` will lead to a +/// borrow checker error, as it borrows [`PlanBaseRef`] again, which is already a reference. +/// +/// As a workaround, we directly let the getters below take the ownership of [`PlanBaseRef`], when +/// callers write `foo.plan_base().schema()`, the compiler will prefer these ones over the ones +/// defined in traits like [`GenericPlanRef`]. impl<'a> PlanBaseRef<'a> { pub(super) fn schema(self) -> &'a Schema { dispatch_plan_base!(self, [Logical, Stream, Batch], GenericPlanRef::schema) From 0650f9b43e3c37b4e5e83a09e17bf8870663d5e1 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 24 Oct 2023 14:15:19 +0800 Subject: [PATCH 09/14] minor refactor on share Signed-off-by: Bugen Zhao --- src/frontend/src/optimizer/plan_node/logical_share.rs | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_share.rs b/src/frontend/src/optimizer/plan_node/logical_share.rs index cf50572411425..d0cec9108a555 100644 --- a/src/frontend/src/optimizer/plan_node/logical_share.rs +++ b/src/frontend/src/optimizer/plan_node/logical_share.rs @@ -20,8 +20,8 @@ use risingwave_common::error::Result; use super::utils::{childless_record, Distill}; use super::{ - generic, ColPrunable, ConventionMarker, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, ToBatch, ToStream, + generic, ColPrunable, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, + ToBatch, ToStream, }; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ @@ -68,11 +68,7 @@ impl LogicalShare { LogicalShare::new(input).into() } - // TODO - pub(super) fn pretty_fields<'a>( - base: &PlanBase, - name: &'a str, - ) -> XmlNode<'a> { + pub(super) fn pretty_fields<'a>(base: impl GenericPlanRef, name: &'a str) -> XmlNode<'a> { childless_record(name, vec![("id", Pretty::debug(&base.id().0))]) } } From 7edc247383f348275198115b100bac0dfdebe146 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 24 Oct 2023 14:26:18 +0800 Subject: [PATCH 10/14] minor refactors Signed-off-by: Bugen Zhao --- .../src/optimizer/plan_node/generic/agg.rs | 14 +++++++------- .../optimizer/plan_node/generic/dynamic_filter.rs | 2 +- src/frontend/src/optimizer/plan_node/plan_base.rs | 4 +++- 3 files changed, 11 insertions(+), 9 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 2fb251ca89aa6..31a3db76d6590 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -265,7 +265,7 @@ pub struct MaterializedInputState { impl Agg { pub fn infer_tables( &self, - me: &impl stream::StreamPlanRef, + me: impl stream::StreamPlanRef, vnode_col_idx: Option, window_col_idx: Option, ) -> ( @@ -274,9 +274,9 @@ impl Agg { HashMap, ) { ( - self.infer_intermediate_state_table(me, vnode_col_idx, window_col_idx), - self.infer_stream_agg_state(me, vnode_col_idx, window_col_idx), - self.infer_distinct_dedup_tables(me, vnode_col_idx, window_col_idx), + self.infer_intermediate_state_table(&me, vnode_col_idx, window_col_idx), + self.infer_stream_agg_state(&me, vnode_col_idx, window_col_idx), + self.infer_distinct_dedup_tables(&me, vnode_col_idx, window_col_idx), ) } @@ -338,7 +338,7 @@ impl Agg { /// Infer `AggCallState`s for streaming agg. pub fn infer_stream_agg_state( &self, - me: &impl stream::StreamPlanRef, + me: impl stream::StreamPlanRef, vnode_col_idx: Option, window_col_idx: Option, ) -> Vec { @@ -487,7 +487,7 @@ impl Agg { /// group key | state for AGG1 | state for AGG2 | ... pub fn infer_intermediate_state_table( &self, - me: &impl GenericPlanRef, + me: impl GenericPlanRef, vnode_col_idx: Option, window_col_idx: Option, ) -> TableCatalog { @@ -553,7 +553,7 @@ impl Agg { /// group key | distinct key | count for AGG1(distinct x) | count for AGG2(distinct x) | ... pub fn infer_distinct_dedup_tables( &self, - me: &impl GenericPlanRef, + me: impl GenericPlanRef, vnode_col_idx: Option, window_col_idx: Option, ) -> HashMap { diff --git a/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs b/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs index a0b24fdda6039..edadd8b07ae3f 100644 --- a/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs @@ -140,7 +140,7 @@ impl DynamicFilter { } pub fn infer_left_internal_table_catalog( - me: &impl stream::StreamPlanRef, + me: impl stream::StreamPlanRef, left_key_index: usize, ) -> TableCatalog { let schema = me.schema(); diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs index d97f2bb988c9f..241226417405f 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -112,7 +112,9 @@ pub struct PlanBase { schema: Schema, /// the pk indices of the PlanNode's output, a empty stream key vec means there is no stream key - // TODO: this is actually a logical and stream only property + // TODO: this is actually a logical and stream only property. + // - For logical nodes, this is `None` in most time expect for the phase after `logical_rewrite_for_stream`. + // - For stream nodes, this is always `Some`. stream_key: Option>, functional_dependency: FunctionalDependencySet, From e6c55fef1685ef98fa47a614ada17594c9484811 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 24 Oct 2023 14:32:39 +0800 Subject: [PATCH 11/14] remove super prefix for plan base convention marker Signed-off-by: Bugen Zhao --- src/frontend/src/optimizer/plan_node/batch.rs | 1 + src/frontend/src/optimizer/plan_node/batch_delete.rs | 5 +++-- src/frontend/src/optimizer/plan_node/batch_exchange.rs | 5 ++--- src/frontend/src/optimizer/plan_node/batch_expand.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_filter.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_group_topn.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_hash_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_hash_join.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_hop_window.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_insert.rs | 4 ++-- src/frontend/src/optimizer/plan_node/batch_limit.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_lookup_join.rs | 2 +- .../src/optimizer/plan_node/batch_nested_loop_join.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_over_window.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_project.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_project_set.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_seq_scan.rs | 5 ++--- src/frontend/src/optimizer/plan_node/batch_simple_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_sort.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_sort_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_source.rs | 4 ++-- .../src/optimizer/plan_node/batch_table_function.rs | 4 ++-- src/frontend/src/optimizer/plan_node/batch_topn.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_union.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_update.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_values.rs | 4 ++-- src/frontend/src/optimizer/plan_node/stream.rs | 1 + src/frontend/src/optimizer/plan_node/stream_dedup.rs | 8 ++++---- src/frontend/src/optimizer/plan_node/stream_delta_join.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_dml.rs | 2 +- .../src/optimizer/plan_node/stream_dynamic_filter.rs | 2 +- .../src/optimizer/plan_node/stream_eowc_over_window.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_exchange.rs | 5 ++--- src/frontend/src/optimizer/plan_node/stream_expand.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_filter.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs | 4 ++-- src/frontend/src/optimizer/plan_node/stream_group_topn.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_hash_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_hash_join.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_hop_window.rs | 2 +- .../src/optimizer/plan_node/stream_materialize.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_now.rs | 4 ++-- .../src/optimizer/plan_node/stream_over_window.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_project.rs | 2 +- .../src/optimizer/plan_node/stream_project_set.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_share.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_simple_agg.rs | 5 ++--- src/frontend/src/optimizer/plan_node/stream_sink.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_sort.rs | 5 ++--- src/frontend/src/optimizer/plan_node/stream_source.rs | 4 ++-- .../optimizer/plan_node/stream_stateless_simple_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_table_scan.rs | 6 ++---- .../src/optimizer/plan_node/stream_temporal_join.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_topn.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_union.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_values.rs | 4 ++-- .../src/optimizer/plan_node/stream_watermark_filter.rs | 4 ++-- 58 files changed, 78 insertions(+), 82 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/batch.rs b/src/frontend/src/optimizer/plan_node/batch.rs index 1ec8315c85fc2..bb3cbb96d6c5d 100644 --- a/src/frontend/src/optimizer/plan_node/batch.rs +++ b/src/frontend/src/optimizer/plan_node/batch.rs @@ -30,5 +30,6 @@ pub trait BatchPlanRef: PhysicalPlanRef { pub mod prelude { pub use super::super::generic::{GenericPlanRef, PhysicalPlanRef}; + pub use super::super::Batch; pub use super::BatchPlanRef; } diff --git a/src/frontend/src/optimizer/plan_node/batch_delete.rs b/src/frontend/src/optimizer/plan_node/batch_delete.rs index 0d476229d582a..96ca967a2f745 100644 --- a/src/frontend/src/optimizer/plan_node/batch_delete.rs +++ b/src/frontend/src/optimizer/plan_node/batch_delete.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::DeleteNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, @@ -27,14 +28,14 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchDelete` implements [`super::LogicalDelete`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchDelete { - pub base: PlanBase, + pub base: PlanBase, pub core: generic::Delete, } impl BatchDelete { pub fn new(core: generic::Delete) -> Self { assert_eq!(core.input.distribution(), &Distribution::Single); - let base: PlanBase = + let base = PlanBase::new_batch_with_core(&core, core.input.distribution().clone(), Order::any()); Self { base, core } } diff --git a/src/frontend/src/optimizer/plan_node/batch_exchange.rs b/src/frontend/src/optimizer/plan_node/batch_exchange.rs index b42902f185743..ec3a195e050bb 100644 --- a/src/frontend/src/optimizer/plan_node/batch_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/batch_exchange.rs @@ -17,8 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{ExchangeNode, MergeSortExchangeNode}; -use super::batch::BatchPlanRef; -use super::generic::{GenericPlanRef, PhysicalPlanRef}; +use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; use crate::optimizer::plan_node::ToLocalBatch; @@ -28,7 +27,7 @@ use crate::optimizer::property::{Distribution, DistributionDisplay, Order, Order /// without changing its content. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchExchange { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, } diff --git a/src/frontend/src/optimizer/plan_node/batch_expand.rs b/src/frontend/src/optimizer/plan_node/batch_expand.rs index 6dc46b8ee5b67..7b31d59fd13e5 100644 --- a/src/frontend/src/optimizer/plan_node/batch_expand.rs +++ b/src/frontend/src/optimizer/plan_node/batch_expand.rs @@ -29,7 +29,7 @@ use crate::optimizer::PlanRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchExpand { - pub base: PlanBase, + pub base: PlanBase, core: generic::Expand, } diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index 0594a1f33dbce..c5c1430772750 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -26,7 +26,7 @@ use crate::utils::Condition; /// `BatchFilter` implements [`super::LogicalFilter`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchFilter { - pub base: PlanBase, + pub base: PlanBase, core: generic::Filter, } diff --git a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs index b65cc73bf1b3b..1817c90442761 100644 --- a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs @@ -27,7 +27,7 @@ use crate::optimizer::property::{Order, RequiredDist}; /// `BatchGroupTopN` implements [`super::LogicalTopN`] to find the top N elements with a heap #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchGroupTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, } diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs index bc84a9daff42b..58512a42f0240 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -31,7 +31,7 @@ use crate::utils::{ColIndexMappingRewriteExt, IndexSet}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchHashAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, } diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs index 1e9a3b1fc5d10..d2135bb47b520 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -36,7 +36,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// get output rows. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchHashJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, /// The join condition must be equivalent to `logical.on`, but separated into equal and diff --git a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs index 1eb1c4dd22d88..3cbfbb38372f4 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs @@ -30,7 +30,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// input rows #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchHopWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::HopWindow, window_start_exprs: Vec, window_end_exprs: Vec, diff --git a/src/frontend/src/optimizer/plan_node/batch_insert.rs b/src/frontend/src/optimizer/plan_node/batch_insert.rs index 3f5c2c82dd6eb..2d89b2a14b128 100644 --- a/src/frontend/src/optimizer/plan_node/batch_insert.rs +++ b/src/frontend/src/optimizer/plan_node/batch_insert.rs @@ -29,14 +29,14 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchInsert` implements [`super::LogicalInsert`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchInsert { - pub base: PlanBase, + pub base: PlanBase, pub core: generic::Insert, } impl BatchInsert { pub fn new(core: generic::Insert) -> Self { assert_eq!(core.input.distribution(), &Distribution::Single); - let base: PlanBase = + let base: PlanBase = PlanBase::new_batch_with_core(&core, core.input.distribution().clone(), Order::any()); BatchInsert { base, core } diff --git a/src/frontend/src/optimizer/plan_node/batch_limit.rs b/src/frontend/src/optimizer/plan_node/batch_limit.rs index f6923cc59a012..f1b9739e8ac2b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_limit.rs +++ b/src/frontend/src/optimizer/plan_node/batch_limit.rs @@ -28,7 +28,7 @@ use crate::optimizer::property::{Order, RequiredDist}; /// `BatchLimit` implements [`super::LogicalLimit`] to fetch specified rows from input #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchLimit { - pub base: PlanBase, + pub base: PlanBase, core: generic::Limit, } diff --git a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs index 2ee7e438809c9..b78bf314c1276 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -34,7 +34,7 @@ use crate::utils::ColIndexMappingRewriteExt; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchLookupJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, /// The join condition must be equivalent to `logical.on`, but separated into equal and diff --git a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs index 6e82f3b1b6bb5..fe789b47937f5 100644 --- a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs @@ -31,7 +31,7 @@ use crate::utils::ConditionDisplay; /// against all pairs of rows from inner & outer side within 2 layers of loops. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchNestedLoopJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, } diff --git a/src/frontend/src/optimizer/plan_node/batch_over_window.rs b/src/frontend/src/optimizer/plan_node/batch_over_window.rs index 362f687903a4f..011de0dfb1459 100644 --- a/src/frontend/src/optimizer/plan_node/batch_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_over_window.rs @@ -29,7 +29,7 @@ use crate::optimizer::property::{Order, RequiredDist}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchOverWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::OverWindow, } diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index 859062e425123..b32498d547dbf 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -31,7 +31,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// rows #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchProject { - pub base: PlanBase, + pub base: PlanBase, core: generic::Project, } diff --git a/src/frontend/src/optimizer/plan_node/batch_project_set.rs b/src/frontend/src/optimizer/plan_node/batch_project_set.rs index fad56834c26ab..5a355c0db22db 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project_set.rs @@ -29,7 +29,7 @@ use crate::utils::ColIndexMappingRewriteExt; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchProjectSet { - pub base: PlanBase, + pub base: PlanBase, core: generic::ProjectSet, } diff --git a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs index c2c63136482c3..7a2d97c266b36 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -24,8 +24,7 @@ use risingwave_pb::batch_plan::row_seq_scan_node::ChunkSize; use risingwave_pb::batch_plan::{RowSeqScanNode, SysRowSeqScanNode}; use risingwave_pb::plan_common::PbColumnDesc; -use super::batch::BatchPlanRef; -use super::generic::{GenericPlanRef, PhysicalPlanRef}; +use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch}; use crate::catalog::ColumnId; @@ -36,7 +35,7 @@ use crate::optimizer::property::{Distribution, DistributionDisplay, Order}; /// `BatchSeqScan` implements [`super::LogicalScan`] to scan from a row-oriented table #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSeqScan { - pub base: PlanBase, + pub base: PlanBase, core: generic::Scan, scan_ranges: Vec, } diff --git a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs index cb133e0f2c606..cbc8f0a95c1f8 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -26,7 +26,7 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSimpleAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, } diff --git a/src/frontend/src/optimizer/plan_node/batch_sort.rs b/src/frontend/src/optimizer/plan_node/batch_sort.rs index 35ca522fe3613..c2e713c68138a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort.rs @@ -28,7 +28,7 @@ use crate::optimizer::property::{Order, OrderDisplay}; /// collation required by user or parent plan node. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSort { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, } diff --git a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs index 0c2d697047ae2..cd8e3b43ca8ec 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -28,7 +28,7 @@ use crate::utils::{ColIndexMappingRewriteExt, IndexSet}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSortAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, input_order: Order, } diff --git a/src/frontend/src/optimizer/plan_node/batch_source.rs b/src/frontend/src/optimizer/plan_node/batch_source.rs index 2789c2211a76b..56f6399cd844e 100644 --- a/src/frontend/src/optimizer/plan_node/batch_source.rs +++ b/src/frontend/src/optimizer/plan_node/batch_source.rs @@ -19,7 +19,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SourceNode; -use super::generic::GenericPlanRef; +use super::batch::prelude::*; use super::utils::{childless_record, column_names_pretty, Distill}; use super::{ generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch, ToLocalBatch, @@ -30,7 +30,7 @@ use crate::optimizer::property::{Distribution, Order}; /// [`BatchSource`] represents a table/connector source at the very beginning of the graph. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSource { - pub base: PlanBase, + pub base: PlanBase, core: generic::Source, } diff --git a/src/frontend/src/optimizer/plan_node/batch_table_function.rs b/src/frontend/src/optimizer/plan_node/batch_table_function.rs index 333863039ea96..a249ac722cdcf 100644 --- a/src/frontend/src/optimizer/plan_node/batch_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/batch_table_function.rs @@ -17,7 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::TableFunctionNode; -use super::generic::GenericPlanRef; +use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchPb, ToDistributedBatch}; use crate::expr::ExprRewriter; @@ -27,7 +27,7 @@ use crate::optimizer::property::{Distribution, Order}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchTableFunction { - pub base: PlanBase, + pub base: PlanBase, logical: LogicalTableFunction, } diff --git a/src/frontend/src/optimizer/plan_node/batch_topn.rs b/src/frontend/src/optimizer/plan_node/batch_topn.rs index aab0476330931..d508b0da53317 100644 --- a/src/frontend/src/optimizer/plan_node/batch_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_topn.rs @@ -29,7 +29,7 @@ use crate::optimizer::property::{Order, RequiredDist}; /// `BatchTopN` implements [`super::LogicalTopN`] to find the top N elements with a heap #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, } diff --git a/src/frontend/src/optimizer/plan_node/batch_union.rs b/src/frontend/src/optimizer/plan_node/batch_union.rs index 5a0653caee045..5f9e354970a65 100644 --- a/src/frontend/src/optimizer/plan_node/batch_union.rs +++ b/src/frontend/src/optimizer/plan_node/batch_union.rs @@ -25,7 +25,7 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchUnion` implements [`super::LogicalUnion`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchUnion { - pub base: PlanBase, + pub base: PlanBase, core: generic::Union, } diff --git a/src/frontend/src/optimizer/plan_node/batch_update.rs b/src/frontend/src/optimizer/plan_node/batch_update.rs index 2169d7a3ebec0..fdcc546f873a8 100644 --- a/src/frontend/src/optimizer/plan_node/batch_update.rs +++ b/src/frontend/src/optimizer/plan_node/batch_update.rs @@ -31,7 +31,7 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchUpdate` implements [`super::LogicalUpdate`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchUpdate { - pub base: PlanBase, + pub base: PlanBase, pub core: generic::Update, } diff --git a/src/frontend/src/optimizer/plan_node/batch_values.rs b/src/frontend/src/optimizer/plan_node/batch_values.rs index 330d8d65735df..00483f37256f4 100644 --- a/src/frontend/src/optimizer/plan_node/batch_values.rs +++ b/src/frontend/src/optimizer/plan_node/batch_values.rs @@ -18,7 +18,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::values_node::ExprTuple; use risingwave_pb::batch_plan::ValuesNode; -use super::generic::GenericPlanRef; +use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{ ExprRewritable, LogicalValues, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchPb, @@ -30,7 +30,7 @@ use crate::optimizer::property::{Distribution, Order}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchValues { - pub base: PlanBase, + pub base: PlanBase, logical: LogicalValues, } diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 33e5eb5fda300..f33a15d2570b3 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -33,5 +33,6 @@ pub trait StreamPlanRef: PhysicalPlanRef { pub mod prelude { pub use super::super::generic::{GenericPlanRef, PhysicalPlanRef}; + pub use super::super::Stream; pub use super::StreamPlanRef; } diff --git a/src/frontend/src/optimizer/plan_node/stream_dedup.rs b/src/frontend/src/optimizer/plan_node/stream_dedup.rs index b3c42e1cf82f0..af1e0cec77e41 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dedup.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dedup.rs @@ -17,17 +17,17 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::DedupNode; -use super::generic::{self, GenericPlanNode, GenericPlanRef, PhysicalPlanRef}; -use super::stream::StreamPlanRef; +use super::generic::GenericPlanNode; +use super::stream::prelude::*; use super::utils::{impl_distill_by_unit, TableCatalogBuilder}; -use super::{ExprRewritable, PlanBase, PlanTreeNodeUnary, StreamNode}; +use super::{generic, ExprRewritable, PlanBase, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::plan_node::PlanRef; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamDedup { - pub base: PlanBase, + pub base: PlanBase, core: generic::Dedup, } diff --git a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs index 19a7ee02caeaf..49a2b97a97454 100644 --- a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs @@ -35,7 +35,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// inputs to be indexes. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamDeltaJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, /// The join condition must be equivalent to `logical.on`, but separated into equal and diff --git a/src/frontend/src/optimizer/plan_node/stream_dml.rs b/src/frontend/src/optimizer/plan_node/stream_dml.rs index 1b7b72e90358b..07154463392f2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dml.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dml.rs @@ -25,7 +25,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamDml { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, column_descs: Vec, } diff --git a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs index f648e856432ed..0ac631413ca3e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs @@ -29,7 +29,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamDynamicFilter { - pub base: PlanBase, + pub base: PlanBase, core: generic::DynamicFilter, cleaned_by_watermark: bool, } diff --git a/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs b/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs index 278f6062e56cd..065fcfebf74b7 100644 --- a/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs @@ -27,7 +27,7 @@ use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamEowcOverWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::OverWindow, } diff --git a/src/frontend/src/optimizer/plan_node/stream_exchange.rs b/src/frontend/src/optimizer/plan_node/stream_exchange.rs index b0385cb4e5a39..d9d28fce298ad 100644 --- a/src/frontend/src/optimizer/plan_node/stream_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/stream_exchange.rs @@ -16,8 +16,7 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{DispatchStrategy, DispatcherType, ExchangeNode}; -use super::generic::{GenericPlanRef, PhysicalPlanRef}; -use super::stream::StreamPlanRef; +use super::stream::prelude::*; use super::utils::{childless_record, plan_node_name, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::{Distribution, DistributionDisplay}; @@ -27,7 +26,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// without changing its content. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamExchange { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, no_shuffle: bool, } diff --git a/src/frontend/src/optimizer/plan_node/stream_expand.rs b/src/frontend/src/optimizer/plan_node/stream_expand.rs index 70b2b145820e8..1085d38c06a18 100644 --- a/src/frontend/src/optimizer/plan_node/stream_expand.rs +++ b/src/frontend/src/optimizer/plan_node/stream_expand.rs @@ -25,7 +25,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamExpand { - pub base: PlanBase, + pub base: PlanBase, core: generic::Expand, } diff --git a/src/frontend/src/optimizer/plan_node/stream_filter.rs b/src/frontend/src/optimizer/plan_node/stream_filter.rs index 956b6593bac0d..8bba484a90e29 100644 --- a/src/frontend/src/optimizer/plan_node/stream_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_filter.rs @@ -26,7 +26,7 @@ use crate::utils::Condition; /// `StreamFilter` implements [`super::LogicalFilter`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamFilter { - pub base: PlanBase, + pub base: PlanBase, core: generic::Filter, } diff --git a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs index 35ef51d0656b2..2fc1aba8127fd 100644 --- a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs +++ b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs @@ -20,9 +20,9 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{PbStreamFsFetch, StreamFsFetchNode}; +use super::stream::prelude::*; use super::{PlanBase, PlanRef, PlanTreeNodeUnary}; use crate::catalog::source_catalog::SourceCatalog; -use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::utils::{childless_record, Distill}; use crate::optimizer::plan_node::{generic, ExprRewritable, StreamNode}; use crate::optimizer::property::Distribution; @@ -30,7 +30,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamFsFetch { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, core: generic::Source, } diff --git a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs index 4bd3eb92663da..d0c3077f83286 100644 --- a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs @@ -28,7 +28,7 @@ use crate::PlanRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamGroupTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, /// an optional column index which is the vnode of each row computed by the input's consistent /// hash distribution diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index 8038109e2352a..6ef8849b1e142 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -29,7 +29,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, IndexSet}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamHashAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, /// An optional column index which is the vnode of each row computed by the input's consistent diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs index 3c41ab9245af7..36aff15d96055 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -39,7 +39,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// get output rows. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamHashJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, /// The join condition must be equivalent to `logical.on`, but separated into equal and diff --git a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs index 615e4e23ed763..e2a55ff121949 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -29,7 +29,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// [`StreamHopWindow`] represents a hop window table function. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamHopWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::HopWindow, window_start_exprs: Vec, window_end_exprs: Vec, diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 0bca63f849426..9741123d34536 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -39,7 +39,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// Materializes a stream. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamMaterialize { - pub base: PlanBase, + pub base: PlanBase, /// Child of Materialize plan input: PlanRef, table: TableCatalog, diff --git a/src/frontend/src/optimizer/plan_node/stream_now.rs b/src/frontend/src/optimizer/plan_node/stream_now.rs index 7828dd32fa0d1..36f9c3811516c 100644 --- a/src/frontend/src/optimizer/plan_node/stream_now.rs +++ b/src/frontend/src/optimizer/plan_node/stream_now.rs @@ -19,7 +19,7 @@ use risingwave_common::types::DataType; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::NowNode; -use super::generic::{GenericPlanRef, PhysicalPlanRef}; +use super::stream::prelude::*; use super::utils::{childless_record, Distill, TableCatalogBuilder}; use super::{ExprRewritable, LogicalNow, PlanBase, StreamNode}; use crate::optimizer::plan_node::utils::column_names_pretty; @@ -29,7 +29,7 @@ use crate::OptimizerContextRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamNow { - pub base: PlanBase, + pub base: PlanBase, } impl StreamNow { diff --git a/src/frontend/src/optimizer/plan_node/stream_over_window.rs b/src/frontend/src/optimizer/plan_node/stream_over_window.rs index 5cffe6e0c0fd8..89469f63e5740 100644 --- a/src/frontend/src/optimizer/plan_node/stream_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_over_window.rs @@ -28,7 +28,7 @@ use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamOverWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::OverWindow, } diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index 21d8e5f033900..da68260e8930b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -30,7 +30,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// rows. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamProject { - pub base: PlanBase, + pub base: PlanBase, core: generic::Project, /// All the watermark derivations, (input_column_index, output_column_index). And the /// derivation expression is the project's expression itself. diff --git a/src/frontend/src/optimizer/plan_node/stream_project_set.rs b/src/frontend/src/optimizer/plan_node/stream_project_set.rs index ba9fb16acca33..5fc922b215975 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project_set.rs @@ -26,7 +26,7 @@ use crate::utils::ColIndexMappingRewriteExt; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamProjectSet { - pub base: PlanBase, + pub base: PlanBase, core: generic::ProjectSet, /// All the watermark derivations, (input_column_idx, expr_idx). And the /// derivation expression is the project_set's expression itself. diff --git a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs index d3f8eaebdf3ab..203bf8951493f 100644 --- a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs +++ b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs @@ -23,7 +23,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamRowIdGen { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, row_id_index: usize, } diff --git a/src/frontend/src/optimizer/plan_node/stream_share.rs b/src/frontend/src/optimizer/plan_node/stream_share.rs index ae68c6fc6651c..15690c047aa60 100644 --- a/src/frontend/src/optimizer/plan_node/stream_share.rs +++ b/src/frontend/src/optimizer/plan_node/stream_share.rs @@ -27,7 +27,7 @@ use crate::Explain; /// `StreamShare` will be translated into an `ExchangeNode` based on its distribution finally. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamShare { - pub base: PlanBase, + pub base: PlanBase, core: generic::Share, } diff --git a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs index 2fb30cba19eb2..06aa227d7e316 100644 --- a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs @@ -18,17 +18,16 @@ use pretty_xmlish::XmlNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{self, PlanAggCall}; +use super::stream::prelude::*; use super::utils::{childless_record, plan_node_name, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; -use crate::optimizer::plan_node::generic::PhysicalPlanRef; -use crate::optimizer::plan_node::stream::StreamPlanRef; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamSimpleAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, /// The index of `count(*)` in `agg_calls`. diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 3f2f927cf0114..cd48fda60e06b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -51,7 +51,7 @@ const DOWNSTREAM_PK_KEY: &str = "primary_key"; /// [`StreamSink`] represents a table/connector sink at the very end of the graph. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamSink { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, sink_desc: SinkDesc, } diff --git a/src/frontend/src/optimizer/plan_node/stream_sort.rs b/src/frontend/src/optimizer/plan_node/stream_sort.rs index 0059bc9c9559e..2468fe52858e4 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sort.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sort.rs @@ -20,8 +20,7 @@ use risingwave_common::catalog::FieldDisplay; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; -use super::generic::{GenericPlanRef, PhysicalPlanRef}; -use super::stream::StreamPlanRef; +use super::stream::prelude::*; use super::utils::{childless_record, Distill, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -29,7 +28,7 @@ use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamEowcSort { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, sort_column_index: usize, diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index ab057fcb5f7cf..51a764b53fab2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -20,10 +20,10 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::{PbStreamSource, SourceNode}; +use super::stream::prelude::*; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanBase, StreamNode}; use crate::catalog::source_catalog::SourceCatalog; -use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::utils::column_names_pretty; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -31,7 +31,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// [`StreamSource`] represents a table/connector source at the very beginning of the graph. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamSource { - pub base: PlanBase, + pub base: PlanBase, pub(crate) core: generic::Source, } diff --git a/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs index 510d76138877f..ca076132f3a55 100644 --- a/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs @@ -33,7 +33,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// by `StreamSimpleAgg`. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamStatelessSimpleAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, } diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs index 2555724a88224..882410e5c9010 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -24,13 +24,11 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::{ChainType, PbStreamNode}; -use super::generic::PhysicalPlanRef; +use super::stream::prelude::*; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanNodeId, PlanRef, StreamNode}; use crate::catalog::ColumnId; use crate::expr::{ExprRewriter, FunctionCall}; -use crate::optimizer::plan_node::generic::GenericPlanRef; -use crate::optimizer::plan_node::stream::StreamPlanRef; use crate::optimizer::plan_node::utils::{IndicesDisplay, TableCatalogBuilder}; use crate::optimizer::property::{Distribution, DistributionDisplay}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -41,7 +39,7 @@ use crate::{Explain, TableCatalog}; /// creation request. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamTableScan { - pub base: PlanBase, + pub base: PlanBase, core: generic::Scan, batch_plan_id: PlanNodeId, chain_type: ChainType, diff --git a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs index bf74344e8ef12..cb524f35bd329 100644 --- a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs @@ -34,7 +34,7 @@ use crate::utils::ColIndexMappingRewriteExt; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamTemporalJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, eq_join_predicate: EqJoinPredicate, } diff --git a/src/frontend/src/optimizer/plan_node/stream_topn.rs b/src/frontend/src/optimizer/plan_node/stream_topn.rs index 37acf93a729a0..1f3618369d15e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_topn.rs @@ -26,7 +26,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamTopN` implements [`super::LogicalTopN`] to find the top N elements with a heap #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, } diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs index 50de3c35ba8fb..425cdc6914564 100644 --- a/src/frontend/src/optimizer/plan_node/stream_union.rs +++ b/src/frontend/src/optimizer/plan_node/stream_union.rs @@ -31,7 +31,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamUnion` implements [`super::LogicalUnion`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamUnion { - pub base: PlanBase, + pub base: PlanBase, core: generic::Union, } diff --git a/src/frontend/src/optimizer/plan_node/stream_values.rs b/src/frontend/src/optimizer/plan_node/stream_values.rs index 55b1b53c7202e..3ae19a3112c5e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_values.rs +++ b/src/frontend/src/optimizer/plan_node/stream_values.rs @@ -18,7 +18,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::values_node::ExprTuple; use risingwave_pb::stream_plan::ValuesNode; -use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, LogicalValues, PlanBase, StreamNode}; use crate::expr::{Expr, ExprImpl}; @@ -28,7 +28,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamValues` implements `LogicalValues.to_stream()` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamValues { - pub base: PlanBase, + pub base: PlanBase, logical: LogicalValues, } diff --git a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs index 84d830a898f83..45997be797c0f 100644 --- a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs @@ -31,7 +31,7 @@ use crate::{TableCatalog, WithOptions}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamWatermarkFilter { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, watermark_descs: Vec, } @@ -56,7 +56,7 @@ impl StreamWatermarkFilter { } fn with_base( - base: PlanBase, + base: PlanBase, input: PlanRef, watermark_descs: Vec, ) -> Self { From 3809e76e107ed2a90c88ab3bfaf255fd22d11057 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 24 Oct 2023 15:57:26 +0800 Subject: [PATCH 12/14] refine docs Signed-off-by: Bugen Zhao --- src/frontend/src/optimizer/plan_node/batch.rs | 1 + src/frontend/src/optimizer/plan_node/mod.rs | 79 +++++++++++++------ .../src/optimizer/plan_node/plan_base.rs | 25 ++++-- .../src/optimizer/plan_node/stream.rs | 1 + 4 files changed, 78 insertions(+), 28 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/batch.rs b/src/frontend/src/optimizer/plan_node/batch.rs index bb3cbb96d6c5d..5eeafab64f1cd 100644 --- a/src/frontend/src/optimizer/plan_node/batch.rs +++ b/src/frontend/src/optimizer/plan_node/batch.rs @@ -28,6 +28,7 @@ pub trait BatchPlanRef: PhysicalPlanRef { fn order(&self) -> &Order; } +/// Prelude for batch plan nodes. pub mod prelude { pub use super::super::generic::{GenericPlanRef, PhysicalPlanRef}; pub use super::super::Batch; diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index fd72155cec3b3..beaaa18726dfe 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -51,18 +51,18 @@ use self::stream::StreamPlanRef; use self::utils::Distill; use super::property::{Distribution, FunctionalDependencySet, Order}; -pub trait PlanNodeMeta { - fn node_type(&self) -> PlanNodeType; - fn plan_base(&self) -> PlanBaseRef<'_>; - fn convention(&self) -> Convention; -} - +/// A marker trait for different conventions, used for enforcing type safety. +/// +/// Implementors are [`Logical`], [`Batch`], and [`Stream`]. pub trait ConventionMarker: 'static + Sized { + /// The extra fields in the [`PlanBase`] of this convention. type Extra: 'static + Eq + Hash + Clone + Debug; + /// Get the [`Convention`] enum value. fn value() -> Convention; } +/// The marker for logical convention. pub struct Logical; impl ConventionMarker for Logical { type Extra = plan_base::NoExtra; @@ -72,6 +72,7 @@ impl ConventionMarker for Logical { } } +/// The marker for batch convention. pub struct Batch; impl ConventionMarker for Batch { type Extra = plan_base::BatchExtra; @@ -81,6 +82,7 @@ impl ConventionMarker for Batch { } } +/// The marker for stream convention. pub struct Stream; impl ConventionMarker for Stream { type Extra = plan_base::StreamExtra; @@ -90,31 +92,54 @@ impl ConventionMarker for Stream { } } -pub trait StaticPlanNodeMeta { +/// The trait for accessing the meta data and [`PlanBase`] for plan nodes. +pub trait PlanNodeMeta { type Convention: ConventionMarker; + const NODE_TYPE: PlanNodeType; + /// Get the reference to the [`PlanBase`] with corresponding convention. fn plan_base(&self) -> &PlanBase; + /// Get the reference to the [`PlanBase`] with erased convention. + /// + /// This is mainly used for implementing [`AnyPlanNodeMeta`]. Callers should prefer + /// [`PlanNodeMeta::plan_base`] instead as it is more type-safe. fn plan_base_ref(&self) -> PlanBaseRef<'_>; } -impl

PlanNodeMeta for P -where - P: StaticPlanNodeMeta, -{ - fn node_type(&self) -> PlanNodeType { - P::NODE_TYPE - } +// Intentionally made private. +mod plan_node_meta { + use super::*; - fn plan_base(&self) -> PlanBaseRef<'_> { - StaticPlanNodeMeta::plan_base_ref(self) + /// The object-safe version of [`PlanNodeMeta`], used as a super trait of [`PlanNode`]. + /// + /// Check [`PlanNodeMeta`] for more details. + pub trait AnyPlanNodeMeta { + fn node_type(&self) -> PlanNodeType; + fn plan_base(&self) -> PlanBaseRef<'_>; + fn convention(&self) -> Convention; } - fn convention(&self) -> Convention { - P::Convention::value() + /// Implement [`AnyPlanNodeMeta`] for all [`PlanNodeMeta`]. + impl

AnyPlanNodeMeta for P + where + P: PlanNodeMeta, + { + fn node_type(&self) -> PlanNodeType { + P::NODE_TYPE + } + + fn plan_base(&self) -> PlanBaseRef<'_> { + PlanNodeMeta::plan_base_ref(self) + } + + fn convention(&self) -> Convention { + P::Convention::value() + } } } +use plan_node_meta::AnyPlanNodeMeta; /// The common trait over all plan nodes. Used by optimizer framework which will treat all node as /// `dyn PlanNode` @@ -136,7 +161,7 @@ pub trait PlanNode: + ToPb + ToLocalBatch + PredicatePushdown - + PlanNodeMeta + + AnyPlanNodeMeta { } @@ -477,7 +502,8 @@ impl PlanTreeNode for PlanRef { } } -impl PlanNodeMeta for PlanRef { +/// Implement again for the `dyn` newtype wrapper. +impl AnyPlanNodeMeta for PlanRef { fn node_type(&self) -> PlanNodeType { self.0.node_type() } @@ -491,6 +517,8 @@ impl PlanNodeMeta for PlanRef { } } +/// Allow access to all fields defined in [`GenericPlanRef`] for the type-erased plan node. +// TODO: may also implement on `dyn PlanNode` directly. impl GenericPlanRef for PlanRef { fn id(&self) -> PlanNodeId { self.plan_base().id() @@ -513,12 +541,16 @@ impl GenericPlanRef for PlanRef { } } +/// Allow access to all fields defined in [`PhysicalPlanRef`] for the type-erased plan node. +// TODO: may also implement on `dyn PlanNode` directly. impl PhysicalPlanRef for PlanRef { fn distribution(&self) -> &Distribution { self.plan_base().distribution() } } +/// Allow access to all fields defined in [`StreamPlanRef`] for the type-erased plan node. +// TODO: may also implement on `dyn PlanNode` directly. impl StreamPlanRef for PlanRef { fn append_only(&self) -> bool { self.plan_base().append_only() @@ -533,6 +565,8 @@ impl StreamPlanRef for PlanRef { } } +/// Allow access to all fields defined in [`BatchPlanRef`] for the type-erased plan node. +// TODO: may also implement on `dyn PlanNode` directly. impl BatchPlanRef for PlanRef { fn order(&self) -> &Order { self.plan_base().order() @@ -592,7 +626,8 @@ pub(crate) fn pretty_config() -> PrettyConfig { } } -// TODO: remove this direct implementation always require `GenericPlanRef`. +/// Directly implement methods for [`PlanNode`] to access the fields defined in [`GenericPlanRef`]. +// TODO: always require `GenericPlanRef` to make it more consistent. impl dyn PlanNode { pub fn id(&self) -> PlanNodeId { self.plan_base().id() @@ -1101,7 +1136,7 @@ macro_rules! impl_plan_node_meta { $( [<$convention $name>] ),* } - $(impl StaticPlanNodeMeta for [<$convention $name>] { + $(impl PlanNodeMeta for [<$convention $name>] { type Convention = $convention; const NODE_TYPE: PlanNodeType = PlanNodeType::[<$convention $name>]; diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs index 241226417405f..0d2e649379112 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -21,6 +21,7 @@ use super::*; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::property::{Distribution, FunctionalDependencySet, Order}; +/// No extra fields for logical plan nodes. #[derive(Clone, Debug, PartialEq, Eq, Hash)] pub struct NoExtra; @@ -37,6 +38,8 @@ mod physical_common { pub dist: Distribution, } + /// A helper trait to reuse code for accessing the common physical fields of batch and stream + /// plan bases. pub trait GetPhysicalCommon { fn physical(&self) -> &PhysicalCommonExtra; fn physical_mut(&mut self) -> &mut PhysicalCommonExtra; @@ -92,14 +95,17 @@ impl GetPhysicalCommon for BatchExtra { } } -/// the common fields of all nodes, please make a field named `base` in -/// every planNode and correctly value it when construct the planNode. +/// The common fields of all plan nodes with different conventions. +/// +/// Please make a field named `base` in every planNode and correctly value +/// it when construct the planNode. /// /// All fields are intentionally made private and immutable, as they should /// normally be the same as the given [`GenericPlanNode`] when constructing. /// /// - To access them, use traits including [`GenericPlanRef`], -/// [`PhysicalPlanRef`], [`StreamPlanRef`] and [`BatchPlanRef`]. +/// [`PhysicalPlanRef`], [`StreamPlanRef`] and [`BatchPlanRef`] with +/// compile-time checks. /// - To mutate them, use methods like `new_*` or `clone_with_*`. #[derive(Educe)] #[educe(PartialEq, Eq, Hash, Clone, Debug)] @@ -118,6 +124,7 @@ pub struct PlanBase { stream_key: Option>, functional_dependency: FunctionalDependencySet, + /// Extra fields for different conventions. extra: C::Extra, } @@ -309,6 +316,11 @@ impl PlanBase { } } +/// Reference to [`PlanBase`] with erased conventions. +/// +/// Used for accessing fields on a type-erased plan node. All traits of [`GenericPlanRef`], +/// [`PhysicalPlanRef`], [`StreamPlanRef`] and [`BatchPlanRef`] are implemented for this type, +/// so runtime checks are required when calling methods on it. #[derive(PartialEq, Eq, Hash, Clone, Copy, Debug, enum_as_inner::EnumAsInner)] pub enum PlanBaseRef<'a> { Logical(&'a PlanBase), @@ -317,6 +329,7 @@ pub enum PlanBaseRef<'a> { } impl PlanBaseRef<'_> { + /// Get the convention of this plan base. pub fn convention(self) -> Convention { match self { PlanBaseRef::Logical(_) => Convention::Logical, @@ -345,9 +358,9 @@ macro_rules! dispatch_plan_base { /// For example, callers writing `GenericPlanRef::schema(&foo.plan_base())` will lead to a /// borrow checker error, as it borrows [`PlanBaseRef`] again, which is already a reference. /// -/// As a workaround, we directly let the getters below take the ownership of [`PlanBaseRef`], when -/// callers write `foo.plan_base().schema()`, the compiler will prefer these ones over the ones -/// defined in traits like [`GenericPlanRef`]. +/// As a workaround, we directly let the getters below take the ownership of [`PlanBaseRef`], +/// which is `Copy`. When callers write `foo.plan_base().schema()`, the compiler will prefer +/// these ones over the ones defined in traits like [`GenericPlanRef`]. impl<'a> PlanBaseRef<'a> { pub(super) fn schema(self) -> &'a Schema { dispatch_plan_base!(self, [Logical, Stream, Batch], GenericPlanRef::schema) diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index f33a15d2570b3..394a64b656ad3 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -31,6 +31,7 @@ pub trait StreamPlanRef: PhysicalPlanRef { fn watermark_columns(&self) -> &FixedBitSet; } +/// Prelude for stream plan nodes. pub mod prelude { pub use super::super::generic::{GenericPlanRef, PhysicalPlanRef}; pub use super::super::Stream; From e53f3a57a0178aa4e8b736eac13c07f8a41fda39 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 24 Oct 2023 16:05:26 +0800 Subject: [PATCH 13/14] fix clippy Signed-off-by: Bugen Zhao --- src/frontend/src/optimizer/plan_node/logical_share.rs | 2 +- src/frontend/src/optimizer/plan_node/mod.rs | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_share.rs b/src/frontend/src/optimizer/plan_node/logical_share.rs index d0cec9108a555..5fe4347a19134 100644 --- a/src/frontend/src/optimizer/plan_node/logical_share.rs +++ b/src/frontend/src/optimizer/plan_node/logical_share.rs @@ -68,7 +68,7 @@ impl LogicalShare { LogicalShare::new(input).into() } - pub(super) fn pretty_fields<'a>(base: impl GenericPlanRef, name: &'a str) -> XmlNode<'a> { + pub(super) fn pretty_fields(base: impl GenericPlanRef, name: &str) -> XmlNode<'_> { childless_record(name, vec![("id", Pretty::debug(&base.id().0))]) } } diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index beaaa18726dfe..ec7777e42e737 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -183,6 +183,7 @@ impl_downcast!(PlanNode); // Using a new type wrapper allows direct function implementation on `PlanRef`, // and we currently need a manual implementation of `PartialEq` for `PlanRef`. +#[allow(clippy::derived_hash_with_manual_eq)] #[derive(Clone, Debug, Eq, Hash)] pub struct PlanRef(Rc); @@ -528,7 +529,7 @@ impl GenericPlanRef for PlanRef { self.plan_base().schema() } - fn stream_key<'a>(&'a self) -> Option<&'a [usize]> { + fn stream_key(&self) -> Option<&[usize]> { self.plan_base().stream_key() } From 35172ec2413870fd3f0020430963bc52b1e26b68 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 24 Oct 2023 16:09:32 +0800 Subject: [PATCH 14/14] remove super prefix for logical marker Signed-off-by: Bugen Zhao --- src/frontend/src/optimizer/plan_node/logical_agg.rs | 4 ++-- src/frontend/src/optimizer/plan_node/logical_apply.rs | 6 +++--- src/frontend/src/optimizer/plan_node/logical_dedup.rs | 8 ++++---- src/frontend/src/optimizer/plan_node/logical_delete.rs | 6 +++--- src/frontend/src/optimizer/plan_node/logical_except.rs | 6 ++++-- src/frontend/src/optimizer/plan_node/logical_expand.rs | 6 +++--- src/frontend/src/optimizer/plan_node/logical_filter.rs | 6 +++--- .../src/optimizer/plan_node/logical_hop_window.rs | 7 ++++--- src/frontend/src/optimizer/plan_node/logical_insert.rs | 6 +++--- src/frontend/src/optimizer/plan_node/logical_intersect.rs | 6 ++++-- src/frontend/src/optimizer/plan_node/logical_join.rs | 6 +++--- src/frontend/src/optimizer/plan_node/logical_limit.rs | 6 +++--- .../src/optimizer/plan_node/logical_multi_join.rs | 4 ++-- src/frontend/src/optimizer/plan_node/logical_now.rs | 4 ++-- .../src/optimizer/plan_node/logical_over_window.rs | 4 ++-- src/frontend/src/optimizer/plan_node/logical_project.rs | 6 +++--- .../src/optimizer/plan_node/logical_project_set.rs | 7 ++++--- src/frontend/src/optimizer/plan_node/logical_scan.rs | 4 ++-- src/frontend/src/optimizer/plan_node/logical_share.rs | 6 +++--- src/frontend/src/optimizer/plan_node/logical_source.rs | 8 ++++---- .../src/optimizer/plan_node/logical_table_function.rs | 4 ++-- src/frontend/src/optimizer/plan_node/logical_topn.rs | 8 ++++---- src/frontend/src/optimizer/plan_node/logical_union.rs | 6 ++++-- src/frontend/src/optimizer/plan_node/logical_update.rs | 6 +++--- src/frontend/src/optimizer/plan_node/logical_values.rs | 6 +++--- 25 files changed, 77 insertions(+), 69 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index e6809a921ef72..0aed32abec40e 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -22,7 +22,7 @@ use risingwave_expr::aggregate::{agg_kinds, AggKind}; use super::generic::{self, Agg, GenericPlanRef, PlanAggCall, ProjectBuilder}; use super::utils::impl_distill_by_unit; use super::{ - BatchHashAgg, BatchSimpleAgg, ColPrunable, ExprRewritable, PlanBase, PlanRef, + BatchHashAgg, BatchSimpleAgg, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHashAgg, StreamProject, StreamSimpleAgg, StreamStatelessSimpleAgg, ToBatch, ToStream, }; @@ -48,7 +48,7 @@ use crate::utils::{ /// The output schema will first include the group key and then the aggregation calls. #[derive(Clone, Debug, PartialEq, Eq, Hash)] pub struct LogicalAgg { - pub base: PlanBase, + pub base: PlanBase, core: Agg, } diff --git a/src/frontend/src/optimizer/plan_node/logical_apply.rs b/src/frontend/src/optimizer/plan_node/logical_apply.rs index efe5a84054746..fa5576c61710b 100644 --- a/src/frontend/src/optimizer/plan_node/logical_apply.rs +++ b/src/frontend/src/optimizer/plan_node/logical_apply.rs @@ -11,8 +11,8 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. -// +// use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::Schema; use risingwave_common::error::{ErrorCode, Result, RwError}; @@ -23,7 +23,7 @@ use super::generic::{ }; use super::utils::{childless_record, Distill}; use super::{ - ColPrunable, LogicalJoin, LogicalProject, PlanBase, PlanRef, PlanTreeNodeBinary, + ColPrunable, Logical, LogicalJoin, LogicalProject, PlanBase, PlanRef, PlanTreeNodeBinary, PredicatePushdown, ToBatch, ToStream, }; use crate::expr::{CorrelatedId, Expr, ExprImpl, ExprRewriter, InputRef}; @@ -38,7 +38,7 @@ use crate::utils::{ColIndexMapping, Condition, ConditionDisplay}; /// left side. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalApply { - pub base: PlanBase, + pub base: PlanBase, left: PlanRef, right: PlanRef, on: Condition, diff --git a/src/frontend/src/optimizer/plan_node/logical_dedup.rs b/src/frontend/src/optimizer/plan_node/logical_dedup.rs index a3bddd612a832..7be8b64ae01af 100644 --- a/src/frontend/src/optimizer/plan_node/logical_dedup.rs +++ b/src/frontend/src/optimizer/plan_node/logical_dedup.rs @@ -21,9 +21,9 @@ use super::generic::TopNLimit; use super::utils::impl_distill_by_unit; use super::{ gen_filter_and_pushdown, generic, BatchGroupTopN, ColPrunable, ColumnPruningContext, - ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, - PredicatePushdownContext, RewriteStreamContext, StreamDedup, StreamGroupTopN, ToBatch, - ToStream, ToStreamContext, + ExprRewritable, Logical, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, + PredicatePushdown, PredicatePushdownContext, RewriteStreamContext, StreamDedup, + StreamGroupTopN, ToBatch, ToStream, ToStreamContext, }; use crate::optimizer::property::{Order, RequiredDist}; use crate::utils::Condition; @@ -32,7 +32,7 @@ use crate::utils::Condition; /// an `ORDER BY`. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalDedup { - pub base: PlanBase, + pub base: PlanBase, core: generic::Dedup, } diff --git a/src/frontend/src/optimizer/plan_node/logical_delete.rs b/src/frontend/src/optimizer/plan_node/logical_delete.rs index 119bad8f9e090..63f8c81b5991d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_delete.rs +++ b/src/frontend/src/optimizer/plan_node/logical_delete.rs @@ -17,8 +17,8 @@ use risingwave_common::error::Result; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchDelete, ColPrunable, ExprRewritable, LogicalProject, - PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchDelete, ColPrunable, ExprRewritable, Logical, + LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; use crate::optimizer::plan_node::{ @@ -31,7 +31,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// It corresponds to the `DELETE` statements in SQL. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalDelete { - pub base: PlanBase, + pub base: PlanBase, core: generic::Delete, } diff --git a/src/frontend/src/optimizer/plan_node/logical_except.rs b/src/frontend/src/optimizer/plan_node/logical_except.rs index 983794530cf0d..353cfe6583d5d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_except.rs +++ b/src/frontend/src/optimizer/plan_node/logical_except.rs @@ -17,7 +17,9 @@ use risingwave_common::catalog::Schema; use risingwave_common::error::Result; use super::utils::impl_distill_by_unit; -use super::{ColPrunable, ExprRewritable, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream}; +use super::{ + ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, +}; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ generic, ColumnPruningContext, PlanTreeNode, PredicatePushdownContext, RewriteStreamContext, @@ -29,7 +31,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// matching rows from its other inputs. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalExcept { - pub base: PlanBase, + pub base: PlanBase, core: generic::Except, } diff --git a/src/frontend/src/optimizer/plan_node/logical_expand.rs b/src/frontend/src/optimizer/plan_node/logical_expand.rs index 7f326c47fad93..31209122e16a3 100644 --- a/src/frontend/src/optimizer/plan_node/logical_expand.rs +++ b/src/frontend/src/optimizer/plan_node/logical_expand.rs @@ -18,8 +18,8 @@ use risingwave_common::error::Result; use super::generic::GenericPlanRef; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchExpand, ColPrunable, ExprRewritable, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, StreamExpand, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchExpand, ColPrunable, ExprRewritable, Logical, PlanBase, + PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamExpand, ToBatch, ToStream, }; use crate::optimizer::plan_node::{ ColumnPruningContext, LogicalProject, PredicatePushdownContext, RewriteStreamContext, @@ -37,7 +37,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// is used to distinguish between different `subset`s in `column_subsets`. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalExpand { - pub base: PlanBase, + pub base: PlanBase, core: generic::Expand, } diff --git a/src/frontend/src/optimizer/plan_node/logical_filter.rs b/src/frontend/src/optimizer/plan_node/logical_filter.rs index e6edd619a156b..a9ebd26d38219 100644 --- a/src/frontend/src/optimizer/plan_node/logical_filter.rs +++ b/src/frontend/src/optimizer/plan_node/logical_filter.rs @@ -21,8 +21,8 @@ use risingwave_common::types::DataType; use super::generic::GenericPlanRef; use super::utils::impl_distill_by_unit; use super::{ - generic, ColPrunable, ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, ToBatch, ToStream, + generic, ColPrunable, ExprRewritable, Logical, LogicalProject, PlanBase, PlanRef, + PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::expr::{assert_input_ref, ExprImpl, ExprRewriter, ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::{ @@ -37,7 +37,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// If the condition allows nulls, then a null value is treated the same as false. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalFilter { - pub base: PlanBase, + pub base: PlanBase, core: generic::Filter, } diff --git a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs index 20712418a98f0..a592337f7e26d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -20,8 +20,9 @@ use risingwave_common::types::Interval; use super::generic::{GenericPlanNode, GenericPlanRef}; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchHopWindow, ColPrunable, ExprRewritable, LogicalFilter, - PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHopWindow, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchHopWindow, ColPrunable, ExprRewritable, Logical, + LogicalFilter, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHopWindow, + ToBatch, ToStream, }; use crate::expr::{ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::{ @@ -32,7 +33,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition}; /// `LogicalHopWindow` implements Hop Table Function. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalHopWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::HopWindow, } diff --git a/src/frontend/src/optimizer/plan_node/logical_insert.rs b/src/frontend/src/optimizer/plan_node/logical_insert.rs index 1574859dffa85..f801affcc93f9 100644 --- a/src/frontend/src/optimizer/plan_node/logical_insert.rs +++ b/src/frontend/src/optimizer/plan_node/logical_insert.rs @@ -19,8 +19,8 @@ use risingwave_common::error::Result; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{ - gen_filter_and_pushdown, generic, BatchInsert, ColPrunable, ExprRewritable, LogicalProject, - PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchInsert, ColPrunable, ExprRewritable, Logical, + LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; use crate::expr::{ExprImpl, ExprRewriter}; @@ -35,7 +35,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// statements, the input relation would be [`super::LogicalValues`]. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalInsert { - pub base: PlanBase, + pub base: PlanBase, core: generic::Insert, } diff --git a/src/frontend/src/optimizer/plan_node/logical_intersect.rs b/src/frontend/src/optimizer/plan_node/logical_intersect.rs index ecaf1921a0757..6e20cf87c6927 100644 --- a/src/frontend/src/optimizer/plan_node/logical_intersect.rs +++ b/src/frontend/src/optimizer/plan_node/logical_intersect.rs @@ -17,7 +17,9 @@ use risingwave_common::catalog::Schema; use risingwave_common::error::Result; use super::utils::impl_distill_by_unit; -use super::{ColPrunable, ExprRewritable, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream}; +use super::{ + ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, +}; use crate::optimizer::plan_node::{ generic, ColumnPruningContext, PlanTreeNode, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, @@ -28,7 +30,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// If `all` is false, it needs to eliminate duplicates. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalIntersect { - pub base: PlanBase, + pub base: PlanBase, core: generic::Intersect, } diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index c595ecb45493d..a928481230d3c 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -26,8 +26,8 @@ use super::generic::{ }; use super::utils::{childless_record, Distill}; use super::{ - generic, ColPrunable, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, PredicatePushdown, - StreamHashJoin, StreamProject, ToBatch, ToStream, + generic, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PlanTreeNodeBinary, + PredicatePushdown, StreamHashJoin, StreamProject, ToBatch, ToStream, }; use crate::expr::{CollectInputRef, Expr, ExprImpl, ExprRewriter, ExprType, InputRef}; use crate::optimizer::plan_node::generic::DynamicFilter; @@ -49,7 +49,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition, Condit /// right columns, dependent on the output indices provided. A repeat output index is illegal. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, } diff --git a/src/frontend/src/optimizer/plan_node/logical_limit.rs b/src/frontend/src/optimizer/plan_node/logical_limit.rs index 5c441d6bcb219..f6678faf396a1 100644 --- a/src/frontend/src/optimizer/plan_node/logical_limit.rs +++ b/src/frontend/src/optimizer/plan_node/logical_limit.rs @@ -17,8 +17,8 @@ use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchLimit, ColPrunable, ExprRewritable, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchLimit, ColPrunable, ExprRewritable, Logical, PlanBase, + PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::optimizer::plan_node::{ ColumnPruningContext, LogicalTopN, PredicatePushdownContext, RewriteStreamContext, @@ -30,7 +30,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// `LogicalLimit` fetches up to `limit` rows from `offset` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalLimit { - pub base: PlanBase, + pub base: PlanBase, pub(super) core: generic::Limit, } diff --git a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs index fce2526ccfbdd..819f84e963cfd 100644 --- a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs @@ -23,7 +23,7 @@ use risingwave_pb::plan_common::JoinType; use super::utils::{childless_record, Distill}; use super::{ - ColPrunable, ExprRewritable, LogicalFilter, LogicalJoin, LogicalProject, PlanBase, + ColPrunable, ExprRewritable, Logical, LogicalFilter, LogicalJoin, LogicalProject, PlanBase, PlanNodeType, PlanRef, PlanTreeNodeBinary, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; @@ -46,7 +46,7 @@ use crate::utils::{ /// expressed as 2-way `LogicalJoin`s. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalMultiJoin { - pub base: PlanBase, + pub base: PlanBase, inputs: Vec, on: Condition, output_indices: Vec, diff --git a/src/frontend/src/optimizer/plan_node/logical_now.rs b/src/frontend/src/optimizer/plan_node/logical_now.rs index 0b817e75d6b20..c13a0c93a6e70 100644 --- a/src/frontend/src/optimizer/plan_node/logical_now.rs +++ b/src/frontend/src/optimizer/plan_node/logical_now.rs @@ -21,7 +21,7 @@ use risingwave_common::types::DataType; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{ - ColPrunable, ColumnPruningContext, ExprRewritable, LogicalFilter, PlanBase, PlanRef, + ColPrunable, ColumnPruningContext, ExprRewritable, Logical, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, RewriteStreamContext, StreamNow, ToBatch, ToStream, ToStreamContext, }; use crate::optimizer::plan_node::utils::column_names_pretty; @@ -31,7 +31,7 @@ use crate::OptimizerContextRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalNow { - pub base: PlanBase, + pub base: PlanBase, } impl LogicalNow { diff --git a/src/frontend/src/optimizer/plan_node/logical_over_window.rs b/src/frontend/src/optimizer/plan_node/logical_over_window.rs index 8f0d68924a3ed..a78a145ab1997 100644 --- a/src/frontend/src/optimizer/plan_node/logical_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_over_window.rs @@ -23,7 +23,7 @@ use risingwave_expr::window_function::{Frame, FrameBound, WindowFuncKind}; use super::generic::{GenericPlanRef, OverWindow, PlanWindowFunction, ProjectBuilder}; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, BatchOverWindow, ColPrunable, ExprRewritable, LogicalProject, + gen_filter_and_pushdown, BatchOverWindow, ColPrunable, ExprRewritable, Logical, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamEowcOverWindow, StreamEowcSort, StreamOverWindow, ToBatch, ToStream, }; @@ -358,7 +358,7 @@ impl<'a> ExprVisitor for OverWindowProjectBuilder<'a> { /// The output schema is the input schema plus the window functions. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalOverWindow { - pub base: PlanBase, + pub base: PlanBase, core: OverWindow, } diff --git a/src/frontend/src/optimizer/plan_node/logical_project.rs b/src/frontend/src/optimizer/plan_node/logical_project.rs index 51ccfcd1a05d4..59a5509ebcd70 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project.rs @@ -19,8 +19,8 @@ use risingwave_common::error::Result; use super::utils::{childless_record, Distill}; use super::{ - gen_filter_and_pushdown, generic, BatchProject, ColPrunable, ExprRewritable, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, StreamProject, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchProject, ColPrunable, ExprRewritable, Logical, PlanBase, + PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProject, ToBatch, ToStream, }; use crate::expr::{collect_input_refs, ExprImpl, ExprRewriter, InputRef}; use crate::optimizer::plan_node::generic::GenericPlanRef; @@ -33,7 +33,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition, Substi /// `LogicalProject` computes a set of expressions from its input relation. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalProject { - pub base: PlanBase, + pub base: PlanBase, core: generic::Project, } diff --git a/src/frontend/src/optimizer/plan_node/logical_project_set.rs b/src/frontend/src/optimizer/plan_node/logical_project_set.rs index bf6d7c912c300..fc894713c2f43 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project_set.rs @@ -19,8 +19,9 @@ use risingwave_common::types::DataType; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchProjectSet, ColPrunable, ExprRewritable, LogicalProject, - PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProjectSet, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchProjectSet, ColPrunable, ExprRewritable, Logical, + LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProjectSet, + ToBatch, ToStream, }; use crate::expr::{ collect_input_refs, Expr, ExprImpl, ExprRewriter, FunctionCall, InputRef, TableFunction, @@ -41,7 +42,7 @@ use crate::utils::{ColIndexMapping, Condition, Substitute}; /// column. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalProjectSet { - pub base: PlanBase, + pub base: PlanBase, core: generic::ProjectSet, } diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 4322b80b103a9..200302ae70f97 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -25,7 +25,7 @@ use risingwave_common::util::sort_util::ColumnOrder; use super::generic::{GenericPlanNode, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::{ - generic, BatchFilter, BatchProject, ColPrunable, ExprRewritable, PlanBase, PlanRef, + generic, BatchFilter, BatchProject, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, StreamTableScan, ToBatch, ToStream, }; use crate::catalog::{ColumnId, IndexCatalog}; @@ -42,7 +42,7 @@ use crate::utils::{ColIndexMapping, Condition, ConditionDisplay}; /// `LogicalScan` returns contents of a table or other equivalent object #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalScan { - pub base: PlanBase, + pub base: PlanBase, core: generic::Scan, } diff --git a/src/frontend/src/optimizer/plan_node/logical_share.rs b/src/frontend/src/optimizer/plan_node/logical_share.rs index 5fe4347a19134..3e7193342fb67 100644 --- a/src/frontend/src/optimizer/plan_node/logical_share.rs +++ b/src/frontend/src/optimizer/plan_node/logical_share.rs @@ -20,8 +20,8 @@ use risingwave_common::error::Result; use super::utils::{childless_record, Distill}; use super::{ - generic, ColPrunable, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, - ToBatch, ToStream, + generic, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PlanTreeNodeUnary, + PredicatePushdown, ToBatch, ToStream, }; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ @@ -49,7 +49,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// ``` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalShare { - pub base: PlanBase, + pub base: PlanBase, core: generic::Share, } diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index e069f6b271a02..01166e74f1359 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -32,9 +32,9 @@ use super::generic::GenericPlanRef; use super::stream_watermark_filter::StreamWatermarkFilter; use super::utils::{childless_record, Distill}; use super::{ - generic, BatchProject, BatchSource, ColPrunable, ExprRewritable, LogicalFilter, LogicalProject, - PlanBase, PlanRef, PredicatePushdown, StreamProject, StreamRowIdGen, StreamSource, ToBatch, - ToStream, + generic, BatchProject, BatchSource, ColPrunable, ExprRewritable, Logical, LogicalFilter, + LogicalProject, PlanBase, PlanRef, PredicatePushdown, StreamProject, StreamRowIdGen, + StreamSource, ToBatch, ToStream, }; use crate::catalog::source_catalog::SourceCatalog; use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, InputRef}; @@ -51,7 +51,7 @@ use crate::utils::{ColIndexMapping, Condition, IndexRewriter}; /// `LogicalSource` returns contents of a table or other equivalent object #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalSource { - pub base: PlanBase, + pub base: PlanBase, pub core: generic::Source, /// Expressions to output. This field presents and will be turned to a `Project` when diff --git a/src/frontend/src/optimizer/plan_node/logical_table_function.rs b/src/frontend/src/optimizer/plan_node/logical_table_function.rs index c2599412357f9..4553722dca328 100644 --- a/src/frontend/src/optimizer/plan_node/logical_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/logical_table_function.rs @@ -19,7 +19,7 @@ use risingwave_common::types::DataType; use super::utils::{childless_record, Distill}; use super::{ - ColPrunable, ExprRewritable, LogicalFilter, LogicalProject, PlanBase, PlanRef, + ColPrunable, ExprRewritable, Logical, LogicalFilter, LogicalProject, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, }; use crate::expr::{Expr, ExprRewriter, TableFunction}; @@ -35,7 +35,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// If the function returns a struct, it will be flattened into multiple columns. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalTableFunction { - pub base: PlanBase, + pub base: PlanBase, pub table_function: TableFunction, pub with_ordinality: bool, } diff --git a/src/frontend/src/optimizer/plan_node/logical_topn.rs b/src/frontend/src/optimizer/plan_node/logical_topn.rs index 25d02b65f1f1f..940714d7d4abb 100644 --- a/src/frontend/src/optimizer/plan_node/logical_topn.rs +++ b/src/frontend/src/optimizer/plan_node/logical_topn.rs @@ -20,9 +20,9 @@ use risingwave_common::util::sort_util::ColumnOrder; use super::generic::TopNLimit; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchGroupTopN, ColPrunable, ExprRewritable, PlanBase, - PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamGroupTopN, StreamProject, ToBatch, - ToStream, + gen_filter_and_pushdown, generic, BatchGroupTopN, ColPrunable, ExprRewritable, Logical, + PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamGroupTopN, StreamProject, + ToBatch, ToStream, }; use crate::expr::{ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::{ @@ -36,7 +36,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition}; /// `LogicalTopN` sorts the input data and fetches up to `limit` rows from `offset` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, } diff --git a/src/frontend/src/optimizer/plan_node/logical_union.rs b/src/frontend/src/optimizer/plan_node/logical_union.rs index c1e75a948cf76..e108707e0b13a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_union.rs +++ b/src/frontend/src/optimizer/plan_node/logical_union.rs @@ -21,7 +21,9 @@ use risingwave_common::error::Result; use risingwave_common::types::{DataType, Scalar}; use super::utils::impl_distill_by_unit; -use super::{ColPrunable, ExprRewritable, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream}; +use super::{ + ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, +}; use crate::expr::{ExprImpl, InputRef, Literal}; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::stream_union::StreamUnion; @@ -37,7 +39,7 @@ use crate::Explain; /// If `all` is false, it needs to eliminate duplicates. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalUnion { - pub base: PlanBase, + pub base: PlanBase, core: generic::Union, } diff --git a/src/frontend/src/optimizer/plan_node/logical_update.rs b/src/frontend/src/optimizer/plan_node/logical_update.rs index dd841c5e6fb55..34c0939f13bc9 100644 --- a/src/frontend/src/optimizer/plan_node/logical_update.rs +++ b/src/frontend/src/optimizer/plan_node/logical_update.rs @@ -18,8 +18,8 @@ use risingwave_common::error::Result; use super::generic::GenericPlanRef; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchUpdate, ColPrunable, ExprRewritable, LogicalProject, - PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchUpdate, ColPrunable, ExprRewritable, Logical, + LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; use crate::expr::{ExprImpl, ExprRewriter}; @@ -34,7 +34,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// It corresponds to the `UPDATE` statements in SQL. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalUpdate { - pub base: PlanBase, + pub base: PlanBase, core: generic::Update, } diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index 4bf59d8cc8db6..38867b3d9c223 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -24,8 +24,8 @@ use risingwave_common::types::{DataType, ScalarImpl}; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{ - BatchValues, ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, - StreamValues, ToBatch, ToStream, + BatchValues, ColPrunable, ExprRewritable, Logical, LogicalFilter, PlanBase, PlanRef, + PredicatePushdown, StreamValues, ToBatch, ToStream, }; use crate::expr::{Expr, ExprImpl, ExprRewriter, Literal}; use crate::optimizer::optimizer_context::OptimizerContextRef; @@ -38,7 +38,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// `LogicalValues` builds rows according to a list of expressions #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalValues { - pub base: PlanBase, + pub base: PlanBase, rows: Arc<[Vec]>, }