From 8b260c975de368407bc12ce413aa7c7d507e2b5f Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 20 Oct 2023 17:35:12 +0800 Subject: [PATCH 01/11] use some getters Signed-off-by: Bugen Zhao --- Cargo.lock | 12 +++ src/frontend/Cargo.toml | 1 + .../src/optimizer/plan_node/batch_exchange.rs | 15 ++-- .../src/optimizer/plan_node/batch_seq_scan.rs | 3 +- .../src/optimizer/plan_node/batch_sort.rs | 5 +- .../src/optimizer/plan_node/batch_source.rs | 5 +- .../src/optimizer/plan_node/generic/mod.rs | 8 -- .../src/optimizer/plan_node/logical_expand.rs | 3 +- .../src/optimizer/plan_node/logical_filter.rs | 3 +- .../optimizer/plan_node/logical_hop_window.rs | 3 +- .../src/optimizer/plan_node/logical_join.rs | 6 +- .../optimizer/plan_node/logical_multi_join.rs | 9 +-- .../optimizer/plan_node/logical_project.rs | 2 +- .../plan_node/logical_project_set.rs | 7 +- .../src/optimizer/plan_node/logical_scan.rs | 2 +- .../src/optimizer/plan_node/logical_share.rs | 2 +- .../src/optimizer/plan_node/logical_source.rs | 3 +- .../src/optimizer/plan_node/logical_union.rs | 6 +- src/frontend/src/optimizer/plan_node/mod.rs | 24 +++--- .../src/optimizer/plan_node/plan_base.rs | 79 +++++++++++-------- .../optimizer/plan_node/stream_exchange.rs | 7 +- .../optimizer/plan_node/stream_hash_agg.rs | 3 +- .../optimizer/plan_node/stream_materialize.rs | 7 +- .../src/optimizer/plan_node/stream_share.rs | 2 +- .../optimizer/plan_node/stream_simple_agg.rs | 3 +- .../src/optimizer/plan_node/stream_sink.rs | 5 +- .../src/optimizer/plan_node/stream_sort.rs | 3 +- .../optimizer/plan_node/stream_table_scan.rs | 6 +- .../optimizer/rule/expand_to_project_rule.rs | 2 +- .../optimizer/rule/union_to_distinct_rule.rs | 4 +- src/frontend/src/scheduler/plan_fragmenter.rs | 2 +- 31 files changed, 131 insertions(+), 111 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 47e8d5f09c30d..57ad0bbaab302 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6625,6 +6625,17 @@ dependencies = [ "zstd-sys", ] +[[package]] +name = "readonly" +version = "0.2.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8f439da1766942fe069954da6058b2e6c1760eb878bae76f5be9fc29f56f574" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.37", +] + [[package]] name = "redis" version = "0.23.3" @@ -7490,6 +7501,7 @@ dependencies = [ "pretty_assertions", "prometheus", "rand", + "readonly", "risingwave_batch", "risingwave_common", "risingwave_common_service", diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 37f9f6326faea..8bb4ef8bcfcb6 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -48,6 +48,7 @@ pretty-xmlish = "0.1.13" pretty_assertions = "1" prometheus = { version = "0.13", features = ["process"] } rand = "0.8" +readonly = "0.2" risingwave_batch = { workspace = true } risingwave_common = { workspace = true } risingwave_common_service = { workspace = true } diff --git a/src/frontend/src/optimizer/plan_node/batch_exchange.rs b/src/frontend/src/optimizer/plan_node/batch_exchange.rs index 583838e877c5e..20816bf6e2fe0 100644 --- a/src/frontend/src/optimizer/plan_node/batch_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/batch_exchange.rs @@ -17,6 +17,9 @@ 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; +use super::stream::StreamPlanRef; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; use crate::optimizer::plan_node::ToLocalBatch; @@ -43,12 +46,12 @@ impl Distill for BatchExchange { fn distill<'a>(&self) -> XmlNode<'a> { let input_schema = self.input.schema(); let order = OrderDisplay { - order: &self.base.order, + order: &self.base.order(), input_schema, } .distill(); let dist = Pretty::display(&DistributionDisplay { - distribution: &self.base.dist, + distribution: &self.base.distribution(), input_schema, }); childless_record("BatchExchange", vec![("order", order), ("dist", dist)]) @@ -75,18 +78,18 @@ impl ToDistributedBatch for BatchExchange { /// The serialization of Batch Exchange is default cuz it will be rewritten in scheduler. impl ToBatchPb for BatchExchange { fn to_batch_prost_body(&self) -> NodeBody { - if self.base.order.is_any() { + if self.base.order().is_any() { NodeBody::Exchange(ExchangeNode { sources: vec![], - input_schema: self.base.schema.to_prost(), + input_schema: self.base.schema().to_prost(), }) } else { NodeBody::MergeSortExchange(MergeSortExchangeNode { exchange: Some(ExchangeNode { sources: vec![], - input_schema: self.base.schema.to_prost(), + input_schema: self.base.schema().to_prost(), }), - column_orders: self.base.order.to_protobuf(), + column_orders: self.base.order().to_protobuf(), }) } } 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 cfc557fe375c6..de68643b29eb8 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -24,6 +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::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch}; use crate::catalog::ColumnId; @@ -196,7 +197,7 @@ impl Distill for BatchSeqScan { if verbose { let dist = Pretty::display(&DistributionDisplay { distribution: self.distribution(), - input_schema: &self.base.schema, + input_schema: &self.base.schema(), }); vec.push(("distribution", dist)); } diff --git a/src/frontend/src/optimizer/plan_node/batch_sort.rs b/src/frontend/src/optimizer/plan_node/batch_sort.rs index 8576a18c19333..e7bff6d51d85b 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::BatchPlanRef; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; use crate::optimizer::plan_node::ToLocalBatch; @@ -56,7 +57,7 @@ impl PlanTreeNodeUnary for BatchSort { } fn clone_with_input(&self, input: PlanRef) -> Self { - Self::new(input, self.base.order.clone()) + Self::new(input, self.base.order().clone()) } } impl_plan_tree_node_for_unary! {BatchSort} @@ -70,7 +71,7 @@ impl ToDistributedBatch for BatchSort { impl ToBatchPb for BatchSort { fn to_batch_prost_body(&self) -> NodeBody { - let column_orders = self.base.order.to_protobuf(); + let column_orders = self.base.order().to_protobuf(); NodeBody::Sort(SortNode { column_orders }) } } diff --git a/src/frontend/src/optimizer/plan_node/batch_source.rs b/src/frontend/src/optimizer/plan_node/batch_source.rs index 3adfbf670343a..5557a8fba0c8c 100644 --- a/src/frontend/src/optimizer/plan_node/batch_source.rs +++ b/src/frontend/src/optimizer/plan_node/batch_source.rs @@ -58,8 +58,9 @@ impl BatchSource { } pub fn clone_with_dist(&self) -> Self { - let mut base = self.base.clone(); - base.dist = Distribution::SomeShard; + let base = self + .base + .clone_with_new_distribution(Distribution::SomeShard); Self { base, logical: self.logical.clone(), diff --git a/src/frontend/src/optimizer/plan_node/generic/mod.rs b/src/frontend/src/optimizer/plan_node/generic/mod.rs index 49038500b4301..87389ae74bd54 100644 --- a/src/frontend/src/optimizer/plan_node/generic/mod.rs +++ b/src/frontend/src/optimizer/plan_node/generic/mod.rs @@ -92,14 +92,6 @@ pub trait GenericPlanRef: Eq + Hash { } pub trait GenericPlanNode { - /// return (schema, `stream_key`, fds) - fn logical_properties(&self) -> (Schema, Option>, FunctionalDependencySet) { - ( - self.schema(), - self.stream_key(), - self.functional_dependency(), - ) - } fn functional_dependency(&self) -> FunctionalDependencySet; fn schema(&self) -> Schema; fn stream_key(&self) -> Option>; diff --git a/src/frontend/src/optimizer/plan_node/logical_expand.rs b/src/frontend/src/optimizer/plan_node/logical_expand.rs index b32374e6dc427..2159165915b8c 100644 --- a/src/frontend/src/optimizer/plan_node/logical_expand.rs +++ b/src/frontend/src/optimizer/plan_node/logical_expand.rs @@ -192,8 +192,7 @@ mod tests { let mut values = LogicalValues::new(vec![], Schema { fields }, ctx); values .base - .functional_dependency - .add_functional_dependency_by_column_indices(&[0], &[1, 2]); + .functional_dependency_mut().add_functional_dependency_by_column_indices(&[0], &[1, 2]); let column_subsets = vec![vec![0, 1], vec![2]]; let expand = LogicalExpand::create(values.into(), column_subsets); diff --git a/src/frontend/src/optimizer/plan_node/logical_filter.rs b/src/frontend/src/optimizer/plan_node/logical_filter.rs index 72ee7d246b83d..27284126ddf14 100644 --- a/src/frontend/src/optimizer/plan_node/logical_filter.rs +++ b/src/frontend/src/optimizer/plan_node/logical_filter.rs @@ -462,8 +462,7 @@ mod tests { // 3 --> 1, 2 values .base - .functional_dependency - .add_functional_dependency_by_column_indices(&[3], &[1, 2]); + .functional_dependency_mut().add_functional_dependency_by_column_indices(&[3], &[1, 2]); // v1 = 0 AND v2 = v3 let predicate = ExprImpl::FunctionCall(Box::new( FunctionCall::new( 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 e4bd65efa647c..4d36e819e71e4 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -446,8 +446,7 @@ mod test { // 0, 1 --> 2 values .base - .functional_dependency - .add_functional_dependency_by_column_indices(&[0, 1], &[2]); + .functional_dependency_mut().add_functional_dependency_by_column_indices(&[0, 1], &[2]); let hop_window: PlanRef = LogicalHopWindow::new( values.into(), InputRef::new(0, DataType::Date), diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index cfc49a1da3353..dfc01bcb3afa5 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -2000,8 +2000,7 @@ mod tests { // 0 --> 1 values .base - .functional_dependency - .add_functional_dependency_by_column_indices(&[0], &[1]); + .functional_dependency_mut().add_functional_dependency_by_column_indices(&[0], &[1]); values }; let right = { @@ -2014,8 +2013,7 @@ mod tests { // 0 --> 1, 2 values .base - .functional_dependency - .add_functional_dependency_by_column_indices(&[0], &[1, 2]); + .functional_dependency_mut().add_functional_dependency_by_column_indices(&[0], &[1, 2]); values }; // l0 = 0 AND l1 = r1 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 1c0253ab7aafd..6f5cae139e15e 100644 --- a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs @@ -883,8 +883,7 @@ mod test { // 0 --> 1 values .base - .functional_dependency - .add_functional_dependency_by_column_indices(&[0], &[1]); + .functional_dependency_mut().add_functional_dependency_by_column_indices(&[0], &[1]); values }; let t2 = { @@ -897,8 +896,7 @@ mod test { // 0 --> 1, 2 values .base - .functional_dependency - .add_functional_dependency_by_column_indices(&[0], &[1, 2]); + .functional_dependency_mut().add_functional_dependency_by_column_indices(&[0], &[1, 2]); values }; let t3 = { @@ -910,8 +908,7 @@ mod test { // {} --> 0 values .base - .functional_dependency - .add_functional_dependency_by_column_indices(&[], &[0]); + .functional_dependency_mut().add_functional_dependency_by_column_indices(&[], &[0]); values }; // On: v0 = 0 AND v1 = v3 AND v4 = v5 diff --git a/src/frontend/src/optimizer/plan_node/logical_project.rs b/src/frontend/src/optimizer/plan_node/logical_project.rs index f3bb51cc7f971..a96de7d91ecd5 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project.rs @@ -284,7 +284,7 @@ impl ToStream for LogicalProject { // But the target size of `out_col_change` should be the same as the length of the new // schema. let (map, _) = out_col_change.into_parts(); - let out_col_change = ColIndexMapping::with_target_size(map, proj.base.schema.len()); + let out_col_change = ColIndexMapping::with_target_size(map, proj.base.schema().len()); Ok((proj.into(), out_col_change)) } } 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 cba907eeeb379..4bf6b18cdabe3 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project_set.rs @@ -427,7 +427,7 @@ mod test { let mut values = LogicalValues::new(vec![], Schema { fields }, ctx); values .base - .functional_dependency + .functional_dependency_mut() .add_functional_dependency_by_column_indices(&[1], &[2]); let project_set = LogicalProjectSet::new( values.into(), @@ -449,8 +449,9 @@ mod test { ); let fd_set: HashSet = project_set .base - .functional_dependency - .into_dependencies() + .functional_dependency() + .as_dependencies() + .clone() .into_iter() .collect(); let expected_fd_set: HashSet = diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index a499a9c6ea3d3..59eafc3aac680 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -551,7 +551,7 @@ impl ToStream for LogicalScan { None.into(), ))); } - match self.base.stream_key.is_none() { + match self.base.stream_key().is_none() { true => { let mut col_ids = HashSet::new(); diff --git a/src/frontend/src/optimizer/plan_node/logical_share.rs b/src/frontend/src/optimizer/plan_node/logical_share.rs index d924ee7180168..d6b5711740a98 100644 --- a/src/frontend/src/optimizer/plan_node/logical_share.rs +++ b/src/frontend/src/optimizer/plan_node/logical_share.rs @@ -69,7 +69,7 @@ impl LogicalShare { } pub(super) fn pretty_fields<'a>(base: &PlanBase, name: &'a str) -> XmlNode<'a> { - childless_record(name, vec![("id", Pretty::debug(&base.id.0))]) + 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 caf04021755f8..4accd4ce20cf4 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -28,6 +28,7 @@ use risingwave_connector::source::{ConnectorProperties, DataType}; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; use risingwave_pb::plan_common::GeneratedColumnDesc; +use super::generic::GenericPlanRef; use super::stream_watermark_filter::StreamWatermarkFilter; use super::utils::{childless_record, Distill}; use super::{ @@ -506,7 +507,7 @@ impl PredicatePushdown for LogicalSource { let mut new_conjunctions = Vec::with_capacity(predicate.conjunctions.len()); for expr in predicate.conjunctions { - if let Some(e) = expr_to_kafka_timestamp_range(expr, &mut range, &self.base.schema) { + if let Some(e) = expr_to_kafka_timestamp_range(expr, &mut range, &self.base.schema()) { // Not recognized, so push back new_conjunctions.push(e); } diff --git a/src/frontend/src/optimizer/plan_node/logical_union.rs b/src/frontend/src/optimizer/plan_node/logical_union.rs index 51e4e620cf4ca..1f02b026c0020 100644 --- a/src/frontend/src/optimizer/plan_node/logical_union.rs +++ b/src/frontend/src/optimizer/plan_node/logical_union.rs @@ -130,7 +130,7 @@ impl ToBatch for LogicalUnion { if !self.all() { let batch_union = BatchUnion::new(new_logical).into(); Ok(BatchHashAgg::new( - generic::Agg::new(vec![], (0..self.base.schema.len()).collect(), batch_union) + generic::Agg::new(vec![], (0..self.base.schema().len()).collect(), batch_union) .with_enable_two_phase(false), ) .into()) @@ -170,7 +170,7 @@ impl ToStream for LogicalUnion { &self, ctx: &mut RewriteStreamContext, ) -> Result<(PlanRef, ColIndexMapping)> { - let original_schema = self.base.schema.clone(); + let original_schema = self.base.schema().clone(); let original_schema_len = original_schema.len(); let mut rewrites = vec![]; for input in &self.core.inputs { @@ -353,7 +353,7 @@ mod tests { // Check the result let union = plan.as_logical_union().unwrap(); - assert_eq!(union.base.schema.len(), 2); + assert_eq!(union.base.schema().len(), 2); } #[tokio::test] diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 188787c93b8c0..4b7bd65760d4f 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -421,27 +421,27 @@ impl PlanTreeNode for PlanRef { impl StreamPlanRef for PlanRef { fn distribution(&self) -> &Distribution { - &self.plan_base().dist + &self.plan_base().distribution() } fn append_only(&self) -> bool { - self.plan_base().append_only + self.plan_base().append_only() } fn emit_on_window_close(&self) -> bool { - self.plan_base().emit_on_window_close + self.plan_base().emit_on_window_close() } } impl BatchPlanRef for PlanRef { fn order(&self) -> &Order { - &self.plan_base().order + &self.plan_base().order() } } impl GenericPlanRef for PlanRef { fn schema(&self) -> &Schema { - &self.plan_base().schema + &self.plan_base().schema() } fn stream_key(&self) -> Option<&[usize]> { @@ -512,7 +512,7 @@ pub(crate) fn pretty_config() -> PrettyConfig { impl dyn PlanNode { pub fn id(&self) -> PlanNodeId { - self.plan_base().id + self.plan_base().id() } pub fn ctx(&self) -> OptimizerContextRef { @@ -520,7 +520,7 @@ impl dyn PlanNode { } pub fn schema(&self) -> &Schema { - &self.plan_base().schema + &self.plan_base().schema() } pub fn stream_key(&self) -> Option<&[usize]> { @@ -528,23 +528,23 @@ impl dyn PlanNode { } pub fn order(&self) -> &Order { - &self.plan_base().order + &self.plan_base().order() } pub fn distribution(&self) -> &Distribution { - &self.plan_base().dist + &self.plan_base().distribution() } pub fn append_only(&self) -> bool { - self.plan_base().append_only + self.plan_base().append_only() } pub fn emit_on_window_close(&self) -> bool { - self.plan_base().emit_on_window_close + self.plan_base().emit_on_window_close() } pub fn functional_dependency(&self) -> &FunctionalDependencySet { - &self.plan_base().functional_dependency + &self.plan_base().functional_dependency() } pub fn watermark_columns(&self) -> &FixedBitSet { diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs index e9a5bf26885bf..bd4254709f19e 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -27,28 +27,33 @@ use crate::optimizer::property::{Distribution, FunctionalDependencySet, Order}; /// every planNode and correctly value it when construct the planNode. #[derive(Clone, Debug, Educe)] #[educe(PartialEq, Eq, Hash)] +// #[readonly::make] pub struct PlanBase { - #[educe(PartialEq(ignore))] - #[educe(Hash(ignore))] - pub id: PlanNodeId, - #[educe(PartialEq(ignore))] - #[educe(Hash(ignore))] + // -- common fields -- + #[educe(PartialEq(ignore), Hash(ignore))] + id: PlanNodeId, + #[educe(PartialEq(ignore), Hash(ignore))] pub ctx: OptimizerContextRef, - pub schema: Schema, + + schema: Schema, /// the pk indices of the PlanNode's output, a empty stream key vec means there is no stream key - pub stream_key: Option>, + stream_key: Option>, + functional_dependency: FunctionalDependencySet, + + // -- batch-only fields -- /// The order property of the PlanNode's output, store an `&Order::any()` here will not affect /// correctness, but insert unnecessary sort in plan - pub order: Order, + order: Order, + + // -- stream-only fields -- /// 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, + dist: Distribution, /// The append-only property of the PlanNode's output is a stream-only property. Append-only /// means the stream contains only insert operation. - pub append_only: bool, + append_only: bool, /// Whether the output is emitted on window close. - pub emit_on_window_close: bool, - pub functional_dependency: FunctionalDependencySet, + emit_on_window_close: bool, /// The watermark column indices of the PlanNode's output. There could be watermark output from /// this stream operator. pub watermark_columns: FixedBitSet, @@ -92,6 +97,17 @@ impl batch::BatchPlanRef for PlanBase { } } +impl PlanBase { + #[cfg(test)] + pub fn functional_dependency_mut(&mut self) -> &mut FunctionalDependencySet { + &mut self.functional_dependency + } + + pub fn id(&self) -> PlanNodeId { + self.id + } +} + impl PlanBase { pub fn new_logical( ctx: OptimizerContextRef, @@ -104,8 +120,8 @@ impl PlanBase { Self { id, ctx, - schema, - stream_key, + schema: schema, + stream_key: stream_key, dist: Distribution::Single, order: Order::any(), // Logical plan node won't touch `append_only` field @@ -159,12 +175,12 @@ impl PlanBase { Self { id, ctx, - schema, - dist, + schema: schema, + dist: dist, order: Order::any(), - stream_key, - append_only, - emit_on_window_close, + stream_key: stream_key, + append_only: append_only, + emit_on_window_close: emit_on_window_close, functional_dependency, watermark_columns, } @@ -190,9 +206,9 @@ impl PlanBase { Self { id, ctx, - schema, - dist, - order, + schema: schema, + dist: dist, + order: order, stream_key: None, // Batch plan node won't touch `append_only` field append_only: true, @@ -202,24 +218,17 @@ impl PlanBase { } } - pub fn derive_stream_plan_base(plan_node: &PlanRef) -> Self { - PlanBase::new_stream( - plan_node.ctx(), - plan_node.schema().clone(), - plan_node.stream_key().map(|v| v.to_vec()), - plan_node.functional_dependency().clone(), - plan_node.distribution().clone(), - plan_node.append_only(), - plan_node.emit_on_window_close(), - plan_node.watermark_columns().clone(), - ) - } - pub fn clone_with_new_plan_id(&self) -> Self { let mut new = self.clone(); new.id = self.ctx.next_plan_node_id(); new } + + pub fn clone_with_new_distribution(&self, dist: Distribution) -> Self { + let mut new = self.clone(); + new.dist = dist; + new + } } macro_rules! impl_base_delegate { diff --git a/src/frontend/src/optimizer/plan_node/stream_exchange.rs b/src/frontend/src/optimizer/plan_node/stream_exchange.rs index 0fa1713bf4488..99d4322530e88 100644 --- a/src/frontend/src/optimizer/plan_node/stream_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/stream_exchange.rs @@ -16,6 +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::stream::StreamPlanRef; use super::utils::{childless_record, plan_node_name, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::{Distribution, DistributionDisplay}; @@ -78,7 +79,7 @@ impl StreamExchange { impl Distill for StreamExchange { fn distill<'a>(&self) -> XmlNode<'a> { let distribution_display = DistributionDisplay { - distribution: &self.base.dist, + distribution: &self.base.distribution(), input_schema: self.input.schema(), }; childless_record( @@ -117,13 +118,13 @@ impl StreamNode for StreamExchange { }) } else { Some(DispatchStrategy { - r#type: match &self.base.dist { + r#type: match &self.base.distribution() { Distribution::HashShard(_) => DispatcherType::Hash, Distribution::Single => DispatcherType::Simple, Distribution::Broadcast => DispatcherType::Broadcast, _ => panic!("Do not allow Any or AnyShard in serialization process"), } as i32, - dist_key_indices: match &self.base.dist { + dist_key_indices: match &self.base.distribution() { Distribution::HashShard(keys) => { keys.iter().map(|num| *num as u32).collect() } 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 5f0cfc16a4171..1098fa8ab6d42 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -22,6 +22,7 @@ use super::generic::{self, PlanAggCall}; use super::utils::{childless_record, plan_node_name, watermark_pretty, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; +use crate::optimizer::plan_node::stream::StreamPlanRef; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, IndexSet}; @@ -214,7 +215,7 @@ impl StreamNode for StreamHashAgg { }) .collect(), row_count_index: self.row_count_idx as u32, - emit_on_window_close: self.base.emit_on_window_close, + emit_on_window_close: self.base.emit_on_window_close(), }) } } diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 128f1eeb24ed9..33750a2ff4eac 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -29,6 +29,7 @@ use super::{reorganize_elements_id, ExprRewritable, PlanRef, PlanTreeNodeUnary, use crate::catalog::table_catalog::{TableCatalog, TableType, TableVersion}; use crate::catalog::FragmentId; use crate::optimizer::plan_node::derive::derive_pk; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{PlanBase, PlanNodeMeta}; use crate::optimizer::property::{Cardinality, Distribution, Order, RequiredDist}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -293,16 +294,16 @@ impl PlanTreeNodeUnary for StreamMaterialize { fn clone_with_input(&self, input: PlanRef) -> Self { let new = Self::new(input, self.table().clone()); new.base - .schema + .schema() .fields .iter() - .zip_eq_fast(self.base.schema.fields.iter()) + .zip_eq_fast(self.base.schema().fields.iter()) .for_each(|(a, b)| { assert_eq!(a.data_type, b.data_type); assert_eq!(a.type_name, b.type_name); assert_eq!(a.sub_fields, b.sub_fields); }); - assert_eq!(new.plan_base().stream_key, self.plan_base().stream_key); + assert_eq!(new.plan_base().stream_key(), self.plan_base().stream_key()); new } } diff --git a/src/frontend/src/optimizer/plan_node/stream_share.rs b/src/frontend/src/optimizer/plan_node/stream_share.rs index 969527fa69702..32273f6994b87 100644 --- a/src/frontend/src/optimizer/plan_node/stream_share.rs +++ b/src/frontend/src/optimizer/plan_node/stream_share.rs @@ -79,7 +79,7 @@ impl StreamNode for StreamShare { impl StreamShare { pub fn adhoc_to_stream_prost(&self, state: &mut BuildFragmentGraphState) -> PbStreamNode { - let operator_id = self.base.id.0 as u32; + let operator_id = self.base.id().0 as u32; match state.get_share_stream_node(operator_id) { None => { 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 f0c0bab5fae77..95ad158cd36d6 100644 --- a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs @@ -21,6 +21,7 @@ use super::generic::{self, PlanAggCall}; 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::stream::StreamPlanRef; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -100,7 +101,7 @@ impl StreamNode for StreamSimpleAgg { .collect(), distribution_key: self .base - .dist + .distribution() .dist_column_indices() .iter() .map(|idx| *idx as u32) diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index a51380d630331..17ebc6f99063d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -37,6 +37,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use tracing::info; use super::derive::{derive_columns, derive_pk}; +use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill, IndicesDisplay, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, StreamNode}; use crate::optimizer::plan_node::PlanTreeNodeUnary; @@ -57,7 +58,7 @@ pub struct StreamSink { impl StreamSink { #[must_use] pub fn new(input: PlanRef, sink_desc: SinkDesc) -> Self { - let base = PlanBase::derive_stream_plan_base(&input); + let base = input.plan_base().clone_with_new_plan_id(); Self { base, input, @@ -389,7 +390,7 @@ impl Distill for StreamSink { .iter() .map(|k| k.column_index) .collect_vec(), - schema: &self.base.schema, + schema: &self.base.schema(), }; vec.push(("pk", pk.distill())); } diff --git a/src/frontend/src/optimizer/plan_node/stream_sort.rs b/src/frontend/src/optimizer/plan_node/stream_sort.rs index b82d71068d817..f3a5a1062b901 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sort.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sort.rs @@ -20,6 +20,7 @@ use risingwave_common::catalog::FieldDisplay; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; +use super::stream::StreamPlanRef; use super::utils::{childless_record, Distill, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -84,7 +85,7 @@ impl StreamEowcSort { tbl_builder.add_order_column(self.sort_column_index, OrderType::ascending()); order_cols.insert(self.sort_column_index); - let dist_key = self.base.dist.dist_column_indices().to_vec(); + let dist_key = self.base.distribution().dist_column_indices().to_vec(); for idx in &dist_key { if !order_cols.contains(idx) { tbl_builder.add_order_column(*idx, OrderType::ascending()); 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 dfe1243e881db..04f170a38b142 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -203,12 +203,12 @@ impl Distill for StreamTableScan { if verbose { let pk = IndicesDisplay { indices: self.stream_key().unwrap_or_default(), - schema: &self.base.schema, + schema: &self.base.schema(), }; vec.push(("pk", pk.distill())); let dist = Pretty::display(&DistributionDisplay { distribution: self.distribution(), - input_schema: &self.base.schema, + input_schema: &self.base.schema(), }); vec.push(("dist", dist)); } @@ -328,7 +328,7 @@ impl StreamTableScan { ..Default::default() })), stream_key, - operator_id: self.base.id.0 as u64, + operator_id: self.base.id().0 as u64, identity: { let s = self.distill_to_string(); s.replace("StreamTableScan", "Chain") diff --git a/src/frontend/src/optimizer/rule/expand_to_project_rule.rs b/src/frontend/src/optimizer/rule/expand_to_project_rule.rs index 1ed1da0037aba..01a39042efd98 100644 --- a/src/frontend/src/optimizer/rule/expand_to_project_rule.rs +++ b/src/frontend/src/optimizer/rule/expand_to_project_rule.rs @@ -36,7 +36,7 @@ impl Rule for ExpandToProjectRule { let column_subset = column_subsets.get(0).unwrap(); // if `column_subsets` len equals 1, convert it into a project - let mut exprs = Vec::with_capacity(expand.base.schema.len()); + let mut exprs = Vec::with_capacity(expand.base.schema().len()); // Add original input column first for i in 0..input.schema().len() { exprs.push(ExprImpl::InputRef( diff --git a/src/frontend/src/optimizer/rule/union_to_distinct_rule.rs b/src/frontend/src/optimizer/rule/union_to_distinct_rule.rs index 2a12f6b712e0d..f1d203fba1350 100644 --- a/src/frontend/src/optimizer/rule/union_to_distinct_rule.rs +++ b/src/frontend/src/optimizer/rule/union_to_distinct_rule.rs @@ -13,7 +13,7 @@ // limitations under the License. use super::{BoxedRule, Rule}; -use crate::optimizer::plan_node::generic::Agg; +use crate::optimizer::plan_node::generic::{Agg, GenericPlanRef}; use crate::optimizer::plan_node::{LogicalUnion, PlanTreeNode}; use crate::optimizer::PlanRef; @@ -24,7 +24,7 @@ impl Rule for UnionToDistinctRule { let union: &LogicalUnion = plan.as_logical_union()?; if !union.all() { let union_all = LogicalUnion::create(true, union.inputs().into_iter().collect()); - let distinct = Agg::new(vec![], (0..union.base.schema.len()).collect(), union_all) + let distinct = Agg::new(vec![], (0..union.base.schema().len()).collect(), union_all) .with_enable_two_phase(false); Some(distinct.into()) } else { diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index 7fa512fcbb05a..1e26a15d3b3c2 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -103,7 +103,7 @@ impl Serialize for ExecutionPlanNode { impl From for ExecutionPlanNode { fn from(plan_node: PlanRef) -> Self { Self { - plan_node_id: plan_node.plan_base().id, + plan_node_id: plan_node.plan_base().id(), plan_node_type: plan_node.node_type(), node: plan_node.to_batch_prost_body(), children: vec![], From b39035e9137af939417518c1ce4642f3f6eb764d Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 20 Oct 2023 17:53:52 +0800 Subject: [PATCH 02/11] refactor more and physical trait Signed-off-by: Bugen Zhao --- src/frontend/src/handler/create_mv.rs | 3 +- src/frontend/src/optimizer/plan_node/batch.rs | 11 ++-- .../src/optimizer/plan_node/batch_exchange.rs | 3 +- .../optimizer/plan_node/batch_hash_join.rs | 2 +- .../src/optimizer/plan_node/batch_insert.rs | 3 +- .../optimizer/plan_node/batch_lookup_join.rs | 4 +- .../plan_node/batch_nested_loop_join.rs | 4 +- .../src/optimizer/plan_node/batch_seq_scan.rs | 2 +- .../optimizer/plan_node/batch_simple_agg.rs | 9 ++-- .../plan_node/batch_table_function.rs | 3 +- .../src/optimizer/plan_node/batch_values.rs | 3 +- .../src/optimizer/plan_node/generic/mod.rs | 6 ++- .../src/optimizer/plan_node/logical_expand.rs | 3 +- .../src/optimizer/plan_node/logical_filter.rs | 3 +- .../optimizer/plan_node/logical_hop_window.rs | 3 +- .../src/optimizer/plan_node/logical_insert.rs | 5 +- .../src/optimizer/plan_node/logical_join.rs | 11 ++-- .../optimizer/plan_node/logical_multi_join.rs | 9 ++-- .../src/optimizer/plan_node/logical_now.rs | 3 +- .../src/optimizer/plan_node/logical_scan.rs | 8 +-- .../src/optimizer/plan_node/logical_values.rs | 3 +- src/frontend/src/optimizer/plan_node/mod.rs | 50 +++++++++++-------- .../src/optimizer/plan_node/plan_base.rs | 48 ++++++++++++------ .../src/optimizer/plan_node/stream.rs | 16 +++--- .../src/optimizer/plan_node/stream_dedup.rs | 2 +- .../optimizer/plan_node/stream_delta_join.rs | 4 +- .../plan_node/stream_dynamic_filter.rs | 7 +-- .../optimizer/plan_node/stream_exchange.rs | 1 + .../optimizer/plan_node/stream_group_topn.rs | 3 +- .../optimizer/plan_node/stream_hash_agg.rs | 2 +- .../optimizer/plan_node/stream_hash_join.rs | 7 +-- .../optimizer/plan_node/stream_hop_window.rs | 3 +- .../optimizer/plan_node/stream_materialize.rs | 5 +- .../src/optimizer/plan_node/stream_now.rs | 5 +- .../optimizer/plan_node/stream_over_window.rs | 3 +- .../src/optimizer/plan_node/stream_project.rs | 3 +- .../optimizer/plan_node/stream_simple_agg.rs | 1 + .../src/optimizer/plan_node/stream_sink.rs | 2 +- .../src/optimizer/plan_node/stream_sort.rs | 1 + .../optimizer/plan_node/stream_table_scan.rs | 2 +- .../plan_node/stream_temporal_join.rs | 6 ++- .../src/optimizer/plan_node/stream_union.rs | 3 +- .../plan_node/stream_watermark_filter.rs | 3 +- .../table_function_to_project_set_rule.rs | 3 +- 44 files changed, 174 insertions(+), 107 deletions(-) diff --git a/src/frontend/src/handler/create_mv.rs b/src/frontend/src/handler/create_mv.rs index 3fa9129f39743..053ba5aa30f19 100644 --- a/src/frontend/src/handler/create_mv.rs +++ b/src/frontend/src/handler/create_mv.rs @@ -26,6 +26,7 @@ use crate::binder::{Binder, BoundQuery, BoundSetExpr}; use crate::catalog::{check_valid_column_name, CatalogError}; use crate::handler::privilege::resolve_query_privileges; use crate::handler::HandlerArgs; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::Explain; use crate::optimizer::{OptimizerContext, OptimizerContextRef, PlanRef, RelationCollectorVisitor}; use crate::planner::Planner; @@ -175,7 +176,7 @@ It only indicates the physical clustering of the data, which may improve the per let (plan, table) = gen_create_mv_plan(&session, context.into(), query, name, columns, emit_mode)?; - let context = plan.plan_base().ctx.clone(); + let context = plan.plan_base().ctx().clone(); let mut graph = build_graph(plan); graph.parallelism = session .config() diff --git a/src/frontend/src/optimizer/plan_node/batch.rs b/src/frontend/src/optimizer/plan_node/batch.rs index d62a85095d21c..33e92b76ff393 100644 --- a/src/frontend/src/optimizer/plan_node/batch.rs +++ b/src/frontend/src/optimizer/plan_node/batch.rs @@ -12,14 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -use super::generic::GenericPlanRef; +use super::generic::PhysicalPlanRef; use crate::optimizer::property::Order; -/// A subtrait of [`GenericPlanRef`] for batch plans. +/// A subtrait of [`PhysicalPlanRef`] for batch plans. /// /// Due to the lack of refactoring, all plan nodes currently implement this trait /// through [`super::PlanBase`]. One may still use this trait as a bound for -/// expecting a batch plan, in contrast to [`GenericPlanRef`]. -pub trait BatchPlanRef: GenericPlanRef { +/// expecting a batch plan, in contrast to [`GenericPlanRef`] or +/// [`PhysicalPlanRef`]. +/// +/// [`GenericPlanRef`]: super::generic::GenericPlanRef +pub trait BatchPlanRef: PhysicalPlanRef { fn order(&self) -> &Order; } diff --git a/src/frontend/src/optimizer/plan_node/batch_exchange.rs b/src/frontend/src/optimizer/plan_node/batch_exchange.rs index 20816bf6e2fe0..aa88d9e8cc10a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/batch_exchange.rs @@ -18,8 +18,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{ExchangeNode, MergeSortExchangeNode}; use super::batch::BatchPlanRef; -use super::generic::GenericPlanRef; -use super::stream::StreamPlanRef; +use super::generic::{GenericPlanRef, PhysicalPlanRef}; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; use crate::optimizer::plan_node::ToLocalBatch; 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 a4ecf8311a479..2f5595508a970 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -99,7 +99,7 @@ impl BatchHashJoin { impl Distill for BatchHashJoin { fn distill<'a>(&self) -> XmlNode<'a> { - let verbose = self.base.ctx.is_explain_verbose(); + let verbose = self.base.ctx().is_explain_verbose(); let mut vec = Vec::with_capacity(if verbose { 3 } else { 2 }); vec.push(("type", Pretty::debug(&self.logical.join_type))); diff --git a/src/frontend/src/optimizer/plan_node/batch_insert.rs b/src/frontend/src/optimizer/plan_node/batch_insert.rs index 305de0e2f6eaa..dc6178eb1467b 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::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; use crate::expr::Expr; @@ -48,7 +49,7 @@ impl Distill for BatchInsert { fn distill<'a>(&self) -> XmlNode<'a> { let vec = self .logical - .fields_pretty(self.base.ctx.is_explain_verbose()); + .fields_pretty(self.base.ctx().is_explain_verbose()); childless_record("BatchInsert", vec) } } 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 3098019499b76..0bb00752f44d1 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -18,7 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{DistributedLookupJoinNode, LocalLookupJoinNode}; -use super::generic::{self}; +use super::generic::{self, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::ExprRewritable; use crate::expr::{Expr, ExprRewriter}; @@ -114,7 +114,7 @@ impl BatchLookupJoin { impl Distill for BatchLookupJoin { fn distill<'a>(&self) -> XmlNode<'a> { - let verbose = self.base.ctx.is_explain_verbose(); + let verbose = self.base.ctx().is_explain_verbose(); let mut vec = Vec::with_capacity(if verbose { 3 } else { 2 }); vec.push(("type", Pretty::debug(&self.logical.join_type))); 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 79d9f07d8eadc..ed8be832dde3d 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,7 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::NestedLoopJoinNode; -use super::generic::{self}; +use super::generic::{self, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchPb, ToDistributedBatch}; use crate::expr::{Expr, ExprImpl, ExprRewriter}; @@ -51,7 +51,7 @@ impl BatchNestedLoopJoin { impl Distill for BatchNestedLoopJoin { fn distill<'a>(&self) -> XmlNode<'a> { - let verbose = self.base.ctx.is_explain_verbose(); + let verbose = self.base.ctx().is_explain_verbose(); let mut vec = Vec::with_capacity(if verbose { 3 } else { 2 }); vec.push(("type", Pretty::debug(&self.logical.join_type))); 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 de68643b29eb8..c0b6aca036ac4 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -181,7 +181,7 @@ fn range_to_string(name: &str, range: &(Bound, Bound)) - impl Distill for BatchSeqScan { fn distill<'a>(&self) -> XmlNode<'a> { - let verbose = self.base.ctx.is_explain_verbose(); + let verbose = self.base.ctx().is_explain_verbose(); let mut vec = Vec::with_capacity(4); vec.push(("table", Pretty::from(self.logical.table_name.clone()))); vec.push(("columns", self.logical.columns_pretty(verbose))); 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 b414779385200..a98432b002028 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -16,7 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortAggNode; -use super::generic::{self, PlanAggCall}; +use super::generic::{self, GenericPlanRef, PlanAggCall}; use super::utils::impl_distill_by_unit; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; use crate::expr::ExprRewriter; @@ -41,8 +41,11 @@ impl BatchSimpleAgg { } fn two_phase_agg_enabled(&self) -> bool { - let session_ctx = self.base.ctx.session_ctx(); - session_ctx.config().get_enable_two_phase_agg() + self.base + .ctx() + .session_ctx() + .config() + .get_enable_two_phase_agg() } pub(crate) fn can_two_phase_agg(&self) -> bool { 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 91aa1af0abbe7..0b9887cd4aaba 100644 --- a/src/frontend/src/optimizer/plan_node/batch_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/batch_table_function.rs @@ -17,6 +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::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchPb, ToDistributedBatch}; use crate::expr::ExprRewriter; @@ -39,7 +40,7 @@ impl BatchTableFunction { } pub fn with_dist(logical: LogicalTableFunction, dist: Distribution) -> Self { - let ctx = logical.base.ctx.clone(); + let ctx = logical.base.ctx().clone(); let base = PlanBase::new_batch(ctx, logical.schema().clone(), dist, Order::any()); BatchTableFunction { base, logical } } diff --git a/src/frontend/src/optimizer/plan_node/batch_values.rs b/src/frontend/src/optimizer/plan_node/batch_values.rs index 5f4e2308493a9..9348cddba7422 100644 --- a/src/frontend/src/optimizer/plan_node/batch_values.rs +++ b/src/frontend/src/optimizer/plan_node/batch_values.rs @@ -18,6 +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::utils::{childless_record, Distill}; use super::{ ExprRewritable, LogicalValues, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchPb, @@ -42,7 +43,7 @@ impl BatchValues { } pub fn with_dist(logical: LogicalValues, dist: Distribution) -> Self { - let ctx = logical.base.ctx.clone(); + let ctx = logical.base.ctx().clone(); let base = PlanBase::new_batch(ctx, logical.schema().clone(), dist, Order::any()); BatchValues { base, logical } } diff --git a/src/frontend/src/optimizer/plan_node/generic/mod.rs b/src/frontend/src/optimizer/plan_node/generic/mod.rs index 87389ae74bd54..ca7205b3ac324 100644 --- a/src/frontend/src/optimizer/plan_node/generic/mod.rs +++ b/src/frontend/src/optimizer/plan_node/generic/mod.rs @@ -20,7 +20,7 @@ use risingwave_common::catalog::Schema; use super::{stream, EqJoinPredicate}; use crate::optimizer::optimizer_context::OptimizerContextRef; -use crate::optimizer::property::FunctionalDependencySet; +use crate::optimizer::property::{Distribution, FunctionalDependencySet}; pub mod dynamic_filter; pub use dynamic_filter::*; @@ -91,6 +91,10 @@ pub trait GenericPlanRef: Eq + Hash { fn ctx(&self) -> OptimizerContextRef; } +pub trait PhysicalPlanRef: GenericPlanRef { + fn distribution(&self) -> &Distribution; +} + pub trait GenericPlanNode { fn functional_dependency(&self) -> FunctionalDependencySet; fn schema(&self) -> Schema; diff --git a/src/frontend/src/optimizer/plan_node/logical_expand.rs b/src/frontend/src/optimizer/plan_node/logical_expand.rs index 2159165915b8c..5cc823cb1e1b7 100644 --- a/src/frontend/src/optimizer/plan_node/logical_expand.rs +++ b/src/frontend/src/optimizer/plan_node/logical_expand.rs @@ -192,7 +192,8 @@ mod tests { let mut values = LogicalValues::new(vec![], Schema { fields }, ctx); values .base - .functional_dependency_mut().add_functional_dependency_by_column_indices(&[0], &[1, 2]); + .functional_dependency_mut() + .add_functional_dependency_by_column_indices(&[0], &[1, 2]); let column_subsets = vec![vec![0, 1], vec![2]]; let expand = LogicalExpand::create(values.into(), column_subsets); diff --git a/src/frontend/src/optimizer/plan_node/logical_filter.rs b/src/frontend/src/optimizer/plan_node/logical_filter.rs index 27284126ddf14..4d5ab47c8fa1d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_filter.rs +++ b/src/frontend/src/optimizer/plan_node/logical_filter.rs @@ -462,7 +462,8 @@ mod tests { // 3 --> 1, 2 values .base - .functional_dependency_mut().add_functional_dependency_by_column_indices(&[3], &[1, 2]); + .functional_dependency_mut() + .add_functional_dependency_by_column_indices(&[3], &[1, 2]); // v1 = 0 AND v2 = v3 let predicate = ExprImpl::FunctionCall(Box::new( FunctionCall::new( 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 4d36e819e71e4..ac193f068ba0d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -446,7 +446,8 @@ mod test { // 0, 1 --> 2 values .base - .functional_dependency_mut().add_functional_dependency_by_column_indices(&[0, 1], &[2]); + .functional_dependency_mut() + .add_functional_dependency_by_column_indices(&[0, 1], &[2]); let hop_window: PlanRef = LogicalHopWindow::new( values.into(), InputRef::new(0, DataType::Date), diff --git a/src/frontend/src/optimizer/plan_node/logical_insert.rs b/src/frontend/src/optimizer/plan_node/logical_insert.rs index 482c034302a38..6f7ba86c05f52 100644 --- a/src/frontend/src/optimizer/plan_node/logical_insert.rs +++ b/src/frontend/src/optimizer/plan_node/logical_insert.rs @@ -16,6 +16,7 @@ use pretty_xmlish::XmlNode; use risingwave_common::catalog::TableVersionId; 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, @@ -90,7 +91,9 @@ impl_plan_tree_node_for_unary! {LogicalInsert} impl Distill for LogicalInsert { fn distill<'a>(&self) -> XmlNode<'a> { - let vec = self.core.fields_pretty(self.base.ctx.is_explain_verbose()); + let vec = self + .core + .fields_pretty(self.base.ctx().is_explain_verbose()); childless_record("LogicalInsert", vec) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index dfc01bcb3afa5..a586af2f0bf42 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -55,7 +55,7 @@ pub struct LogicalJoin { impl Distill for LogicalJoin { fn distill<'a>(&self) -> XmlNode<'a> { - let verbose = self.base.ctx.is_explain_verbose(); + let verbose = self.base.ctx().is_explain_verbose(); let mut vec = Vec::with_capacity(if verbose { 3 } else { 2 }); vec.push(("type", Pretty::debug(&self.join_type()))); @@ -1296,7 +1296,8 @@ impl ToBatch for LogicalJoin { logical_join.left = logical_join.left.to_batch()?; logical_join.right = logical_join.right.to_batch()?; - let config = self.base.ctx.session_ctx().config(); + let ctx = self.base.ctx(); + let config = ctx.session_ctx().config(); if predicate.has_eq() { if !predicate.eq_keys_are_type_aligned() { @@ -2000,7 +2001,8 @@ mod tests { // 0 --> 1 values .base - .functional_dependency_mut().add_functional_dependency_by_column_indices(&[0], &[1]); + .functional_dependency_mut() + .add_functional_dependency_by_column_indices(&[0], &[1]); values }; let right = { @@ -2013,7 +2015,8 @@ mod tests { // 0 --> 1, 2 values .base - .functional_dependency_mut().add_functional_dependency_by_column_indices(&[0], &[1, 2]); + .functional_dependency_mut() + .add_functional_dependency_by_column_indices(&[0], &[1, 2]); values }; // l0 = 0 AND l1 = r1 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 6f5cae139e15e..dc4d627f3f37b 100644 --- a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs @@ -883,7 +883,8 @@ mod test { // 0 --> 1 values .base - .functional_dependency_mut().add_functional_dependency_by_column_indices(&[0], &[1]); + .functional_dependency_mut() + .add_functional_dependency_by_column_indices(&[0], &[1]); values }; let t2 = { @@ -896,7 +897,8 @@ mod test { // 0 --> 1, 2 values .base - .functional_dependency_mut().add_functional_dependency_by_column_indices(&[0], &[1, 2]); + .functional_dependency_mut() + .add_functional_dependency_by_column_indices(&[0], &[1, 2]); values }; let t3 = { @@ -908,7 +910,8 @@ mod test { // {} --> 0 values .base - .functional_dependency_mut().add_functional_dependency_by_column_indices(&[], &[0]); + .functional_dependency_mut() + .add_functional_dependency_by_column_indices(&[], &[0]); values }; // On: v0 = 0 AND v1 = v3 AND v4 = v5 diff --git a/src/frontend/src/optimizer/plan_node/logical_now.rs b/src/frontend/src/optimizer/plan_node/logical_now.rs index 2792c4848e3b3..1d720db15b71a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_now.rs +++ b/src/frontend/src/optimizer/plan_node/logical_now.rs @@ -18,6 +18,7 @@ use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::Result; use risingwave_common::types::DataType; +use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{ ColPrunable, ColumnPruningContext, ExprRewritable, LogicalFilter, PlanBase, PlanRef, @@ -53,7 +54,7 @@ impl LogicalNow { impl Distill for LogicalNow { fn distill<'a>(&self) -> XmlNode<'a> { - let vec = if self.base.ctx.is_explain_verbose() { + let vec = if self.base.ctx().is_explain_verbose() { vec![("output", column_names_pretty(self.schema()))] } else { vec![] diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 59eafc3aac680..07d2a6c7653e7 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -273,7 +273,7 @@ impl LogicalScan { self.output_col_idx().to_vec(), self.core.table_desc.clone(), self.indexes().to_vec(), - self.base.ctx.clone(), + self.base.ctx().clone(), predicate, self.for_system_time_as_of_proctime(), self.table_cardinality(), @@ -288,7 +288,7 @@ impl LogicalScan { output_col_idx, self.core.table_desc.clone(), self.indexes().to_vec(), - self.base.ctx.clone(), + self.base.ctx().clone(), self.predicate().clone(), self.for_system_time_as_of_proctime(), self.table_cardinality(), @@ -309,7 +309,7 @@ impl_plan_tree_node_for_leaf! {LogicalScan} impl Distill for LogicalScan { fn distill<'a>(&self) -> XmlNode<'a> { - let verbose = self.base.ctx.is_explain_verbose(); + let verbose = self.base.ctx().is_explain_verbose(); let mut vec = Vec::with_capacity(5); vec.push(("table", Pretty::from(self.table_name().to_owned()))); let key_is_columns = @@ -440,7 +440,7 @@ impl LogicalScan { let (scan_ranges, predicate) = self.predicate().clone().split_to_scan_ranges( self.core.table_desc.clone(), self.base - .ctx + .ctx() .session_ctx() .config() .get_max_split_range_gap(), diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index c6a3d2ac0564e..e62c6400f2015 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -21,6 +21,7 @@ use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::Result; 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, @@ -144,7 +145,7 @@ impl ColPrunable for LogicalValues { .iter() .map(|i| self.schema().fields[*i].clone()) .collect(); - Self::new(rows, Schema { fields }, self.base.ctx.clone()).into() + Self::new(rows, Schema { fields }, self.base.ctx().clone()).into() } } diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 4b7bd65760d4f..e95d94cac4eed 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -46,7 +46,7 @@ use serde::Serialize; use smallvec::SmallVec; use self::batch::BatchPlanRef; -use self::generic::GenericPlanRef; +use self::generic::{GenericPlanRef, PhysicalPlanRef}; use self::stream::StreamPlanRef; use self::utils::Distill; use super::property::{Distribution, FunctionalDependencySet, Order}; @@ -419,41 +419,47 @@ impl PlanTreeNode for PlanRef { } } -impl StreamPlanRef for PlanRef { - fn distribution(&self) -> &Distribution { - &self.plan_base().distribution() +impl GenericPlanRef for PlanRef { + fn schema(&self) -> &Schema { + &self.plan_base().schema() } - fn append_only(&self) -> bool { - self.plan_base().append_only() + fn stream_key(&self) -> Option<&[usize]> { + self.plan_base().stream_key() } - fn emit_on_window_close(&self) -> bool { - self.plan_base().emit_on_window_close() + fn ctx(&self) -> OptimizerContextRef { + self.plan_base().ctx() + } + + fn functional_dependency(&self) -> &FunctionalDependencySet { + self.plan_base().functional_dependency() } } -impl BatchPlanRef for PlanRef { - fn order(&self) -> &Order { - &self.plan_base().order() +impl PhysicalPlanRef for PlanRef { + fn distribution(&self) -> &Distribution { + &self.plan_base().distribution() } } -impl GenericPlanRef for PlanRef { - fn schema(&self) -> &Schema { - &self.plan_base().schema() +impl StreamPlanRef for PlanRef { + fn append_only(&self) -> bool { + self.plan_base().append_only() } - fn stream_key(&self) -> Option<&[usize]> { - self.plan_base().stream_key() + fn emit_on_window_close(&self) -> bool { + self.plan_base().emit_on_window_close() } - fn ctx(&self) -> OptimizerContextRef { - self.plan_base().ctx() + fn watermark_columns(&self) -> &FixedBitSet { + &self.plan_base().watermark_columns() } +} - fn functional_dependency(&self) -> &FunctionalDependencySet { - self.plan_base().functional_dependency() +impl BatchPlanRef for PlanRef { + fn order(&self) -> &Order { + &self.plan_base().order() } } @@ -516,7 +522,7 @@ impl dyn PlanNode { } pub fn ctx(&self) -> OptimizerContextRef { - self.plan_base().ctx.clone() + self.plan_base().ctx().clone() } pub fn schema(&self) -> &Schema { @@ -548,7 +554,7 @@ impl dyn PlanNode { } pub fn watermark_columns(&self) -> &FixedBitSet { - &self.plan_base().watermark_columns + &self.plan_base().watermark_columns() } /// Serialize the plan node and its children to a stream plan proto. diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs index bd4254709f19e..03704d0a97c4e 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -25,15 +25,21 @@ use crate::optimizer::property::{Distribution, FunctionalDependencySet, Order}; /// the common fields of all nodes, 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`]. +/// - To mutate them, use methods like `new_*` or `clone_with_*`. #[derive(Clone, Debug, Educe)] #[educe(PartialEq, Eq, Hash)] -// #[readonly::make] pub struct PlanBase { // -- common fields -- #[educe(PartialEq(ignore), Hash(ignore))] id: PlanNodeId, #[educe(PartialEq(ignore), Hash(ignore))] - pub ctx: OptimizerContextRef, + ctx: OptimizerContextRef, schema: Schema, /// the pk indices of the PlanNode's output, a empty stream key vec means there is no stream key @@ -56,7 +62,14 @@ pub struct PlanBase { emit_on_window_close: bool, /// The watermark column indices of the PlanNode's output. There could be watermark output from /// this stream operator. - pub watermark_columns: FixedBitSet, + watermark_columns: FixedBitSet, +} + +impl PlanBase { + /// The unique id of the PlanNode in this optimizer context. + pub fn id(&self) -> PlanNodeId { + self.id + } } impl generic::GenericPlanRef for PlanBase { @@ -77,11 +90,13 @@ impl generic::GenericPlanRef for PlanBase { } } -impl stream::StreamPlanRef for PlanBase { +impl generic::PhysicalPlanRef for PlanBase { fn distribution(&self) -> &Distribution { &self.dist } +} +impl stream::StreamPlanRef for PlanBase { fn append_only(&self) -> bool { self.append_only } @@ -89,6 +104,10 @@ impl stream::StreamPlanRef for PlanBase { fn emit_on_window_close(&self) -> bool { self.emit_on_window_close } + + fn watermark_columns(&self) -> &FixedBitSet { + &self.watermark_columns + } } impl batch::BatchPlanRef for PlanBase { @@ -97,17 +116,6 @@ impl batch::BatchPlanRef for PlanBase { } } -impl PlanBase { - #[cfg(test)] - pub fn functional_dependency_mut(&mut self) -> &mut FunctionalDependencySet { - &mut self.functional_dependency - } - - pub fn id(&self) -> PlanNodeId { - self.id - } -} - impl PlanBase { pub fn new_logical( ctx: OptimizerContextRef, @@ -220,7 +228,7 @@ 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.id = self.ctx().next_plan_node_id(); new } @@ -231,6 +239,14 @@ impl PlanBase { } } +// Mutators for testing only. +#[cfg(test)] +impl PlanBase { + pub fn functional_dependency_mut(&mut self) -> &mut FunctionalDependencySet { + &mut self.functional_dependency + } +} + macro_rules! impl_base_delegate { ($( { $convention:ident, $name:ident }),*) => { $(paste! { diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 2edf997bf91fd..41707e9816f15 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -12,16 +12,20 @@ // See the License for the specific language governing permissions and // limitations under the License. -use super::generic::GenericPlanRef; -use crate::optimizer::property::Distribution; +use fixedbitset::FixedBitSet; -/// A subtrait of [`GenericPlanRef`] for stream plans. +use super::generic::PhysicalPlanRef; + +/// A subtrait of [`PhysicalPlanRef`] for stream plans. /// /// Due to the lack of refactoring, all plan nodes currently implement this trait /// through [`super::PlanBase`]. One may still use this trait as a bound for -/// expecting a stream plan, in contrast to [`GenericPlanRef`]. -pub trait StreamPlanRef: GenericPlanRef { - fn distribution(&self) -> &Distribution; +/// expecting a stream plan, in contrast to [`GenericPlanRef`] or +/// [`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; } diff --git a/src/frontend/src/optimizer/plan_node/stream_dedup.rs b/src/frontend/src/optimizer/plan_node/stream_dedup.rs index 847616629355e..d658be70d9df9 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dedup.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dedup.rs @@ -17,7 +17,7 @@ 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}; +use super::generic::{self, GenericPlanNode, GenericPlanRef, PhysicalPlanRef}; use super::stream::StreamPlanRef; use super::utils::{impl_distill_by_unit, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanTreeNodeUnary, StreamNode}; 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 580c3563bfa09..2ec332f85a513 100644 --- a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs @@ -20,7 +20,7 @@ use risingwave_pb::plan_common::JoinType; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ArrangementInfo, DeltaIndexJoinNode}; -use super::generic::{self}; +use super::generic::{self, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, StreamNode}; use crate::expr::{Expr, ExprRewriter}; @@ -90,7 +90,7 @@ impl StreamDeltaJoin { impl Distill for StreamDeltaJoin { fn distill<'a>(&self) -> XmlNode<'a> { - let verbose = self.base.ctx.is_explain_verbose(); + let verbose = self.base.ctx().is_explain_verbose(); let mut vec = Vec::with_capacity(if verbose { 3 } else { 2 }); vec.push(("type", Pretty::debug(&self.logical.join_type))); 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 e1ca18da937e9..2f2a601436fce 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs @@ -17,7 +17,8 @@ pub use risingwave_pb::expr::expr_node::Type as ExprType; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::DynamicFilterNode; -use super::generic::DynamicFilter; +use super::generic::{DynamicFilter, GenericPlanRef}; +use super::stream::StreamPlanRef; use super::utils::{childless_record, column_names_pretty, watermark_pretty, Distill}; use super::{generic, ExprRewritable}; use crate::expr::{Expr, ExprImpl}; @@ -78,11 +79,11 @@ impl StreamDynamicFilter { impl Distill for StreamDynamicFilter { fn distill<'a>(&self) -> XmlNode<'a> { - let verbose = self.base.ctx.is_explain_verbose(); + let verbose = self.base.ctx().is_explain_verbose(); let pred = self.core.pretty_field(); let mut vec = Vec::with_capacity(if verbose { 3 } else { 2 }); vec.push(("predicate", pred)); - if let Some(ow) = watermark_pretty(&self.base.watermark_columns, self.schema()) { + if let Some(ow) = watermark_pretty(&self.base.watermark_columns(), self.schema()) { vec.push(("output_watermarks", ow)); } vec.push(("output", column_names_pretty(self.schema()))); diff --git a/src/frontend/src/optimizer/plan_node/stream_exchange.rs b/src/frontend/src/optimizer/plan_node/stream_exchange.rs index 99d4322530e88..0bc52b474e3fa 100644 --- a/src/frontend/src/optimizer/plan_node/stream_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/stream_exchange.rs @@ -16,6 +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::PhysicalPlanRef; use super::stream::StreamPlanRef; use super::utils::{childless_record, plan_node_name, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; 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 a6147164b8d45..f3f9e87ee03f9 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, TopNLimit}; +use super::stream::StreamPlanRef; use super::utils::{plan_node_name, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::plan_node::generic::GenericPlanNode; @@ -135,7 +136,7 @@ impl Distill for StreamGroupTopN { { "append_only", self.input().append_only() }, ); let mut node = self.logical.distill_with_name(name); - if let Some(ow) = watermark_pretty(&self.base.watermark_columns, self.schema()) { + if let Some(ow) = watermark_pretty(&self.base.watermark_columns(), self.schema()) { node.fields.push(("output_watermarks".into(), ow)); } node 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 1098fa8ab6d42..970d4a8b8d97a 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -143,7 +143,7 @@ impl StreamHashAgg { impl Distill for StreamHashAgg { fn distill<'a>(&self) -> XmlNode<'a> { let mut vec = self.logical.fields_pretty(); - if let Some(ow) = watermark_pretty(&self.base.watermark_columns, self.schema()) { + if let Some(ow) = watermark_pretty(&self.base.watermark_columns(), self.schema()) { vec.push(("output_watermarks", ow)); } childless_record( 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 989de9f8757e2..425345f846bdd 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -20,7 +20,8 @@ use risingwave_pb::plan_common::JoinType; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{DeltaExpression, HashJoinNode, PbInequalityPair}; -use super::generic::Join; +use super::generic::{GenericPlanRef, Join}; +use super::stream::StreamPlanRef; use super::utils::{childless_record, plan_node_name, watermark_pretty, Distill}; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, StreamDeltaJoin, StreamNode, @@ -301,7 +302,7 @@ impl Distill for StreamHashJoin { { "interval", self.clean_left_state_conjunction_idx.is_some() && self.clean_right_state_conjunction_idx.is_some() }, { "append_only", self.is_append_only }, ); - let verbose = self.base.ctx.is_explain_verbose(); + let verbose = self.base.ctx().is_explain_verbose(); let mut vec = Vec::with_capacity(6); vec.push(("type", Pretty::debug(&self.logical.join_type))); @@ -326,7 +327,7 @@ impl Distill for StreamHashJoin { if let Some(i) = self.clean_right_state_conjunction_idx { vec.push(("conditions_to_clean_right_state_table", get_cond(i))); } - if let Some(ow) = watermark_pretty(&self.base.watermark_columns, self.schema()) { + if let Some(ow) = watermark_pretty(&self.base.watermark_columns(), self.schema()) { vec.push(("output_watermarks", ow)); } 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 3780a6cda3f57..32dcaaac0f31a 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -17,6 +17,7 @@ use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::HopWindowNode; +use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::{Expr, ExprImpl, ExprRewriter}; @@ -75,7 +76,7 @@ impl StreamHopWindow { impl Distill for StreamHopWindow { fn distill<'a>(&self) -> XmlNode<'a> { let mut vec = self.logical.fields_pretty(); - if let Some(ow) = watermark_pretty(&self.base.watermark_columns, self.schema()) { + if let Some(ow) = watermark_pretty(&self.base.watermark_columns(), self.schema()) { vec.push(("output_watermarks", ow)); } childless_record("StreamHopWindow", vec) diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 33750a2ff4eac..53a12ba186f8b 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::StreamPlanRef; use super::utils::{childless_record, Distill}; use super::{reorganize_elements_id, ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::catalog::table_catalog::{TableCatalog, TableType, TableVersion}; @@ -273,8 +274,8 @@ impl Distill for StreamMaterialize { vec.push(("pk_conflict", Pretty::from(pk_conflict_behavior))); - let watermark_columns = &self.base.watermark_columns; - if self.base.watermark_columns.count_ones(..) > 0 { + let watermark_columns = &self.base.watermark_columns(); + if self.base.watermark_columns().count_ones(..) > 0 { let watermark_column_names = watermark_columns .ones() .map(|i| table.columns()[i].name_with_hidden().to_string()) diff --git a/src/frontend/src/optimizer/plan_node/stream_now.rs b/src/frontend/src/optimizer/plan_node/stream_now.rs index 9eb0a0e0f143e..91ebc344fa51d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_now.rs +++ b/src/frontend/src/optimizer/plan_node/stream_now.rs @@ -19,8 +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; -use super::stream::StreamPlanRef; +use super::generic::{GenericPlanRef, PhysicalPlanRef}; use super::utils::{childless_record, Distill, TableCatalogBuilder}; use super::{ExprRewritable, LogicalNow, PlanBase, StreamNode}; use crate::optimizer::plan_node::utils::column_names_pretty; @@ -59,7 +58,7 @@ impl StreamNow { impl Distill for StreamNow { fn distill<'a>(&self) -> XmlNode<'a> { - let vec = if self.base.ctx.is_explain_verbose() { + let vec = if self.base.ctx().is_explain_verbose() { vec![("output", column_names_pretty(self.schema()))] } else { vec![] 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 d3a89129b4b82..c4816c8639db3 100644 --- a/src/frontend/src/optimizer/plan_node/stream_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_over_window.rs @@ -21,6 +21,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{GenericPlanNode, PlanWindowFunction}; use super::utils::{impl_distill_by_unit, TableCatalogBuilder}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::TableCatalog; @@ -122,7 +123,7 @@ impl StreamNode for StreamOverWindow { .to_internal_table_prost(); let cache_policy = self .base - .ctx + .ctx() .session_ctx() .config() .get_streaming_over_window_cache_policy(); diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index 8e1b30eaafad2..1b9f7135a61d2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -17,6 +17,7 @@ use pretty_xmlish::XmlNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::ProjectNode; +use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::{try_derive_watermark, Expr, ExprImpl, ExprRewriter, WatermarkDerivation}; @@ -41,7 +42,7 @@ impl Distill for StreamProject { let schema = self.schema(); let mut vec = self.logical.fields_pretty(schema); if let Some(display_output_watermarks) = - watermark_pretty(&self.base.watermark_columns, schema) + watermark_pretty(&self.base.watermark_columns(), schema) { vec.push(("output_watermarks", display_output_watermarks)); } 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 95ad158cd36d6..07dc801b390b9 100644 --- a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs @@ -21,6 +21,7 @@ use super::generic::{self, PlanAggCall}; 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; diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 17ebc6f99063d..f978cdf403125 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -410,7 +410,7 @@ impl StreamNode for StreamSink { PbNodeBody::Sink(SinkNode { sink_desc: Some(self.sink_desc.to_proto()), table: Some(table.to_internal_table_prost()), - log_store_type: match self.base.ctx.session_ctx().config().get_sink_decouple() { + log_store_type: match self.base.ctx().session_ctx().config().get_sink_decouple() { SinkDecouple::Default => { let enable_sink_decouple = match_sink_name_str!( diff --git a/src/frontend/src/optimizer/plan_node/stream_sort.rs b/src/frontend/src/optimizer/plan_node/stream_sort.rs index f3a5a1062b901..0ca0429ad8bff 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sort.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sort.rs @@ -20,6 +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::PhysicalPlanRef; use super::stream::StreamPlanRef; use super::utils::{childless_record, Distill, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; 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 04f170a38b142..e92b704f899a9 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -195,7 +195,7 @@ impl_plan_tree_node_for_leaf! { StreamTableScan } impl Distill for StreamTableScan { fn distill<'a>(&self) -> XmlNode<'a> { - let verbose = self.base.ctx.is_explain_verbose(); + let verbose = self.base.ctx().is_explain_verbose(); let mut vec = Vec::with_capacity(4); vec.push(("table", Pretty::from(self.logical.table_name.clone()))); vec.push(("columns", self.logical.columns_pretty(verbose))); 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 eb883103fe511..5b047cd274c57 100644 --- a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs @@ -18,6 +18,8 @@ use risingwave_pb::plan_common::JoinType; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::TemporalJoinNode; +use super::generic::GenericPlanRef; +use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, StreamNode}; use crate::expr::{Expr, ExprRewriter}; @@ -90,7 +92,7 @@ impl StreamTemporalJoin { impl Distill for StreamTemporalJoin { fn distill<'a>(&self) -> XmlNode<'a> { - let verbose = self.base.ctx.is_explain_verbose(); + let verbose = self.base.ctx().is_explain_verbose(); let mut vec = Vec::with_capacity(if verbose { 3 } else { 2 }); vec.push(("type", Pretty::debug(&self.logical.join_type))); @@ -103,7 +105,7 @@ impl Distill for StreamTemporalJoin { }), )); - if let Some(ow) = watermark_pretty(&self.base.watermark_columns, self.schema()) { + if let Some(ow) = watermark_pretty(&self.base.watermark_columns(), self.schema()) { vec.push(("output_watermarks", ow)); } diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs index 1d259115b5ced..ce670d4e0b7b9 100644 --- a/src/frontend/src/optimizer/plan_node/stream_union.rs +++ b/src/frontend/src/optimizer/plan_node/stream_union.rs @@ -19,6 +19,7 @@ use pretty_xmlish::XmlNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::UnionNode; +use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanRef}; use crate::optimizer::plan_node::generic::GenericPlanNode; @@ -60,7 +61,7 @@ impl StreamUnion { impl Distill for StreamUnion { fn distill<'a>(&self) -> XmlNode<'a> { let mut vec = self.logical.fields_pretty(); - if let Some(ow) = watermark_pretty(&self.base.watermark_columns, self.schema()) { + if let Some(ow) = watermark_pretty(&self.base.watermark_columns(), self.schema()) { vec.push(("output_watermarks", ow)); } childless_record("StreamUnion", vec) 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 ed5a946603ee4..2a06a1636594e 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::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::{ExprDisplay, ExprImpl}; @@ -85,7 +86,7 @@ impl Distill for StreamWatermarkFilter { }) .collect(); let display_output_watermarks = - watermark_pretty(&self.base.watermark_columns, input_schema).unwrap(); + watermark_pretty(&self.base.watermark_columns(), input_schema).unwrap(); let fields = vec![ ("watermark_descs", Pretty::Array(display_watermark_descs)), ("output_watermarks", display_output_watermarks), diff --git a/src/frontend/src/optimizer/rule/table_function_to_project_set_rule.rs b/src/frontend/src/optimizer/rule/table_function_to_project_set_rule.rs index 5a6f1187fdd02..f85ffc2318459 100644 --- a/src/frontend/src/optimizer/rule/table_function_to_project_set_rule.rs +++ b/src/frontend/src/optimizer/rule/table_function_to_project_set_rule.rs @@ -18,6 +18,7 @@ use risingwave_common::types::DataType; use super::{BoxedRule, Rule}; use crate::expr::{Expr, ExprImpl, ExprType, FunctionCall, InputRef}; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ LogicalProject, LogicalProjectSet, LogicalTableFunction, LogicalValues, PlanTreeNodeUnary, }; @@ -51,7 +52,7 @@ impl Rule for TableFunctionToProjectSetRule { let logical_values = LogicalValues::create( vec![vec![]], Schema::new(vec![]), - logical_table_function.base.ctx.clone(), + logical_table_function.base.ctx().clone(), ); let logical_project_set = LogicalProjectSet::create(logical_values, vec![table_function]); // We need a project to align schema type because From 361b2ae64632f228ce0efe9cd3218dc09684cfcd Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 20 Oct 2023 18:00:23 +0800 Subject: [PATCH 03/11] fix clippy Signed-off-by: Bugen Zhao --- src/frontend/src/optimizer/plan_node/batch.rs | 2 +- .../src/optimizer/plan_node/batch_exchange.rs | 4 ++-- .../src/optimizer/plan_node/batch_seq_scan.rs | 2 +- .../src/optimizer/plan_node/logical_source.rs | 2 +- src/frontend/src/optimizer/plan_node/mod.rs | 18 +++++++-------- .../src/optimizer/plan_node/plan_base.rs | 22 +++++++++---------- .../src/optimizer/plan_node/stream.rs | 2 +- .../plan_node/stream_dynamic_filter.rs | 2 +- .../optimizer/plan_node/stream_exchange.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 +- .../src/optimizer/plan_node/stream_project.rs | 2 +- .../src/optimizer/plan_node/stream_sink.rs | 2 +- .../optimizer/plan_node/stream_table_scan.rs | 4 ++-- .../plan_node/stream_temporal_join.rs | 2 +- .../src/optimizer/plan_node/stream_union.rs | 2 +- .../plan_node/stream_watermark_filter.rs | 2 +- 19 files changed, 39 insertions(+), 39 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/batch.rs b/src/frontend/src/optimizer/plan_node/batch.rs index 33e92b76ff393..2cb2360b3e51d 100644 --- a/src/frontend/src/optimizer/plan_node/batch.rs +++ b/src/frontend/src/optimizer/plan_node/batch.rs @@ -19,7 +19,7 @@ use crate::optimizer::property::Order; /// /// Due to the lack of refactoring, all plan nodes currently implement this trait /// through [`super::PlanBase`]. One may still use this trait as a bound for -/// expecting a batch plan, in contrast to [`GenericPlanRef`] or +/// accessing a batch plan, in contrast to [`GenericPlanRef`] or /// [`PhysicalPlanRef`]. /// /// [`GenericPlanRef`]: super::generic::GenericPlanRef diff --git a/src/frontend/src/optimizer/plan_node/batch_exchange.rs b/src/frontend/src/optimizer/plan_node/batch_exchange.rs index aa88d9e8cc10a..6477c7ec213e2 100644 --- a/src/frontend/src/optimizer/plan_node/batch_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/batch_exchange.rs @@ -45,12 +45,12 @@ impl Distill for BatchExchange { fn distill<'a>(&self) -> XmlNode<'a> { let input_schema = self.input.schema(); let order = OrderDisplay { - order: &self.base.order(), + order: self.base.order(), input_schema, } .distill(); let dist = Pretty::display(&DistributionDisplay { - distribution: &self.base.distribution(), + distribution: self.base.distribution(), input_schema, }); childless_record("BatchExchange", vec![("order", order), ("dist", dist)]) 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 c0b6aca036ac4..d2e982d11c449 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -197,7 +197,7 @@ impl Distill for BatchSeqScan { if verbose { let dist = Pretty::display(&DistributionDisplay { distribution: self.distribution(), - input_schema: &self.base.schema(), + input_schema: self.base.schema(), }); vec.push(("distribution", dist)); } diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 4accd4ce20cf4..7f3d50b8ee010 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -507,7 +507,7 @@ impl PredicatePushdown for LogicalSource { let mut new_conjunctions = Vec::with_capacity(predicate.conjunctions.len()); for expr in predicate.conjunctions { - if let Some(e) = expr_to_kafka_timestamp_range(expr, &mut range, &self.base.schema()) { + if let Some(e) = expr_to_kafka_timestamp_range(expr, &mut range, self.base.schema()) { // Not recognized, so push back new_conjunctions.push(e); } diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index e95d94cac4eed..75113a631d8de 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -421,7 +421,7 @@ impl PlanTreeNode for PlanRef { impl GenericPlanRef for PlanRef { fn schema(&self) -> &Schema { - &self.plan_base().schema() + self.plan_base().schema() } fn stream_key(&self) -> Option<&[usize]> { @@ -439,7 +439,7 @@ impl GenericPlanRef for PlanRef { impl PhysicalPlanRef for PlanRef { fn distribution(&self) -> &Distribution { - &self.plan_base().distribution() + self.plan_base().distribution() } } @@ -453,13 +453,13 @@ impl StreamPlanRef for PlanRef { } fn watermark_columns(&self) -> &FixedBitSet { - &self.plan_base().watermark_columns() + self.plan_base().watermark_columns() } } impl BatchPlanRef for PlanRef { fn order(&self) -> &Order { - &self.plan_base().order() + self.plan_base().order() } } @@ -526,7 +526,7 @@ impl dyn PlanNode { } pub fn schema(&self) -> &Schema { - &self.plan_base().schema() + self.plan_base().schema() } pub fn stream_key(&self) -> Option<&[usize]> { @@ -534,11 +534,11 @@ impl dyn PlanNode { } pub fn order(&self) -> &Order { - &self.plan_base().order() + self.plan_base().order() } pub fn distribution(&self) -> &Distribution { - &self.plan_base().distribution() + self.plan_base().distribution() } pub fn append_only(&self) -> bool { @@ -550,11 +550,11 @@ impl dyn PlanNode { } pub fn functional_dependency(&self) -> &FunctionalDependencySet { - &self.plan_base().functional_dependency() + self.plan_base().functional_dependency() } pub fn watermark_columns(&self) -> &FixedBitSet { - &self.plan_base().watermark_columns() + self.plan_base().watermark_columns() } /// Serialize the plan node and its children to a stream plan proto. diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs index 03704d0a97c4e..3f5a169976153 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -66,7 +66,7 @@ pub struct PlanBase { } impl PlanBase { - /// The unique id of the PlanNode in this optimizer context. + /// The unique id of the `PlanNode` in this optimizer context. pub fn id(&self) -> PlanNodeId { self.id } @@ -128,8 +128,8 @@ impl PlanBase { Self { id, ctx, - schema: schema, - stream_key: stream_key, + schema, + stream_key, dist: Distribution::Single, order: Order::any(), // Logical plan node won't touch `append_only` field @@ -183,12 +183,12 @@ impl PlanBase { Self { id, ctx, - schema: schema, - dist: dist, + schema, + dist, order: Order::any(), - stream_key: stream_key, - append_only: append_only, - emit_on_window_close: emit_on_window_close, + stream_key, + append_only, + emit_on_window_close, functional_dependency, watermark_columns, } @@ -214,9 +214,9 @@ impl PlanBase { Self { id, ctx, - schema: schema, - dist: dist, - order: order, + schema, + dist, + order, stream_key: None, // Batch plan node won't touch `append_only` field append_only: true, diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 41707e9816f15..866c62c2413a5 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -20,7 +20,7 @@ use super::generic::PhysicalPlanRef; /// /// Due to the lack of refactoring, all plan nodes currently implement this trait /// through [`super::PlanBase`]. One may still use this trait as a bound for -/// expecting a stream plan, in contrast to [`GenericPlanRef`] or +/// accessing a stream plan, in contrast to [`GenericPlanRef`] or /// [`PhysicalPlanRef`]. /// /// [`GenericPlanRef`]: super::generic::GenericPlanRef 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 2f2a601436fce..6f1a617ee1236 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs @@ -83,7 +83,7 @@ impl Distill for StreamDynamicFilter { let pred = self.core.pretty_field(); let mut vec = Vec::with_capacity(if verbose { 3 } else { 2 }); vec.push(("predicate", pred)); - if let Some(ow) = watermark_pretty(&self.base.watermark_columns(), self.schema()) { + if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) { vec.push(("output_watermarks", ow)); } vec.push(("output", column_names_pretty(self.schema()))); diff --git a/src/frontend/src/optimizer/plan_node/stream_exchange.rs b/src/frontend/src/optimizer/plan_node/stream_exchange.rs index 0bc52b474e3fa..6166a5553cc9d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/stream_exchange.rs @@ -80,7 +80,7 @@ impl StreamExchange { impl Distill for StreamExchange { fn distill<'a>(&self) -> XmlNode<'a> { let distribution_display = DistributionDisplay { - distribution: &self.base.distribution(), + distribution: self.base.distribution(), input_schema: self.input.schema(), }; childless_record( 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 f3f9e87ee03f9..79240a27d3fd4 100644 --- a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs @@ -136,7 +136,7 @@ impl Distill for StreamGroupTopN { { "append_only", self.input().append_only() }, ); let mut node = self.logical.distill_with_name(name); - if let Some(ow) = watermark_pretty(&self.base.watermark_columns(), self.schema()) { + if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) { node.fields.push(("output_watermarks".into(), ow)); } node 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 970d4a8b8d97a..7c9a22e0bb095 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -143,7 +143,7 @@ impl StreamHashAgg { impl Distill for StreamHashAgg { fn distill<'a>(&self) -> XmlNode<'a> { let mut vec = self.logical.fields_pretty(); - if let Some(ow) = watermark_pretty(&self.base.watermark_columns(), self.schema()) { + if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) { vec.push(("output_watermarks", ow)); } childless_record( 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 425345f846bdd..96ca5b28fa823 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -327,7 +327,7 @@ impl Distill for StreamHashJoin { if let Some(i) = self.clean_right_state_conjunction_idx { vec.push(("conditions_to_clean_right_state_table", get_cond(i))); } - if let Some(ow) = watermark_pretty(&self.base.watermark_columns(), self.schema()) { + if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) { vec.push(("output_watermarks", ow)); } 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 32dcaaac0f31a..abe1b33426343 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -76,7 +76,7 @@ impl StreamHopWindow { impl Distill for StreamHopWindow { fn distill<'a>(&self) -> XmlNode<'a> { let mut vec = self.logical.fields_pretty(); - if let Some(ow) = watermark_pretty(&self.base.watermark_columns(), self.schema()) { + if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) { vec.push(("output_watermarks", ow)); } childless_record("StreamHopWindow", vec) diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index 1b9f7135a61d2..c609a86d050f8 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -42,7 +42,7 @@ impl Distill for StreamProject { let schema = self.schema(); let mut vec = self.logical.fields_pretty(schema); if let Some(display_output_watermarks) = - watermark_pretty(&self.base.watermark_columns(), schema) + watermark_pretty(self.base.watermark_columns(), schema) { vec.push(("output_watermarks", display_output_watermarks)); } diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index f978cdf403125..32e9fb487910c 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -390,7 +390,7 @@ impl Distill for StreamSink { .iter() .map(|k| k.column_index) .collect_vec(), - schema: &self.base.schema(), + schema: self.base.schema(), }; vec.push(("pk", pk.distill())); } 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 e92b704f899a9..7b0800798b862 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -203,12 +203,12 @@ impl Distill for StreamTableScan { if verbose { let pk = IndicesDisplay { indices: self.stream_key().unwrap_or_default(), - schema: &self.base.schema(), + schema: self.base.schema(), }; vec.push(("pk", pk.distill())); let dist = Pretty::display(&DistributionDisplay { distribution: self.distribution(), - input_schema: &self.base.schema(), + input_schema: self.base.schema(), }); vec.push(("dist", dist)); } 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 5b047cd274c57..0d53aab22ef52 100644 --- a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs @@ -105,7 +105,7 @@ impl Distill for StreamTemporalJoin { }), )); - if let Some(ow) = watermark_pretty(&self.base.watermark_columns(), self.schema()) { + if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) { vec.push(("output_watermarks", ow)); } diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs index ce670d4e0b7b9..9d3f279f28b7f 100644 --- a/src/frontend/src/optimizer/plan_node/stream_union.rs +++ b/src/frontend/src/optimizer/plan_node/stream_union.rs @@ -61,7 +61,7 @@ impl StreamUnion { impl Distill for StreamUnion { fn distill<'a>(&self) -> XmlNode<'a> { let mut vec = self.logical.fields_pretty(); - if let Some(ow) = watermark_pretty(&self.base.watermark_columns(), self.schema()) { + if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) { vec.push(("output_watermarks", ow)); } childless_record("StreamUnion", vec) 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 2a06a1636594e..066bc9a234ca5 100644 --- a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs @@ -86,7 +86,7 @@ impl Distill for StreamWatermarkFilter { }) .collect(); let display_output_watermarks = - watermark_pretty(&self.base.watermark_columns(), input_schema).unwrap(); + watermark_pretty(self.base.watermark_columns(), input_schema).unwrap(); let fields = vec![ ("watermark_descs", Pretty::Array(display_watermark_descs)), ("output_watermarks", display_output_watermarks), From 71ab194e96f5b421526544c07f44388d0f681605 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 20 Oct 2023 18:15:47 +0800 Subject: [PATCH 04/11] rename methods Signed-off-by: Bugen Zhao --- .../src/optimizer/plan_node/batch_delete.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 | 2 +- .../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_agg.rs | 2 +- .../src/optimizer/plan_node/batch_source.rs | 2 +- .../src/optimizer/plan_node/batch_topn.rs | 2 +- .../src/optimizer/plan_node/batch_union.rs | 2 +- .../src/optimizer/plan_node/logical_source.rs | 2 +- .../src/optimizer/plan_node/plan_base.rs | 65 ++++++++++--------- .../src/optimizer/plan_node/stream_dedup.rs | 2 +- .../optimizer/plan_node/stream_delta_join.rs | 2 +- .../plan_node/stream_dynamic_filter.rs | 2 +- .../plan_node/stream_eowc_over_window.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_hash_agg.rs | 2 +- .../optimizer/plan_node/stream_hash_join.rs | 2 +- .../optimizer/plan_node/stream_hop_window.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 +- .../src/optimizer/plan_node/stream_share.rs | 2 +- .../optimizer/plan_node/stream_simple_agg.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 +- 43 files changed, 76 insertions(+), 73 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/batch_delete.rs b/src/frontend/src/optimizer/plan_node/batch_delete.rs index 600ec6827e3eb..ccb99360e13d3 100644 --- a/src/frontend/src/optimizer/plan_node/batch_delete.rs +++ b/src/frontend/src/optimizer/plan_node/batch_delete.rs @@ -33,7 +33,7 @@ pub struct BatchDelete { impl BatchDelete { pub fn new(logical: generic::Delete) -> Self { assert_eq!(logical.input.distribution(), &Distribution::Single); - let base: PlanBase = PlanBase::new_batch_from_logical( + let base: PlanBase = PlanBase::new_batch_with_core( &logical, logical.input.distribution().clone(), Order::any(), diff --git a/src/frontend/src/optimizer/plan_node/batch_expand.rs b/src/frontend/src/optimizer/plan_node/batch_expand.rs index 72caa27858be1..0ab34cb731e20 100644 --- a/src/frontend/src/optimizer/plan_node/batch_expand.rs +++ b/src/frontend/src/optimizer/plan_node/batch_expand.rs @@ -41,7 +41,7 @@ impl BatchExpand { | Distribution::UpstreamHashShard(_, _) => Distribution::SomeShard, Distribution::Broadcast => unreachable!(), }; - let base = PlanBase::new_batch_from_logical(&logical, dist, Order::any()); + let base = PlanBase::new_batch_with_core(&logical, dist, Order::any()); BatchExpand { base, logical } } diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index aadbda9800b16..49cada015bced 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -32,7 +32,7 @@ pub struct BatchFilter { impl BatchFilter { pub fn new(logical: generic::Filter) -> Self { // TODO: derive from input - let base = PlanBase::new_batch_from_logical( + let base = PlanBase::new_batch_with_core( &logical, logical.input.distribution().clone(), logical.input.order().clone(), 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 1d61b4e9eb379..1fcbafe6cedc2 100644 --- a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs @@ -33,7 +33,7 @@ pub struct BatchGroupTopN { impl BatchGroupTopN { pub fn new(logical: generic::TopN) -> Self { assert!(!logical.group_key.is_empty()); - let base = PlanBase::new_batch_from_logical( + let base = PlanBase::new_batch_with_core( &logical, logical.input.distribution().clone(), Order::any(), 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 7100125dcee99..72d5fbd785be3 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -42,7 +42,7 @@ impl BatchHashAgg { let dist = logical .i2o_col_mapping() .rewrite_provided_distribution(input_dist); - let base = PlanBase::new_batch_from_logical(&logical, dist, Order::any()); + let base = PlanBase::new_batch_with_core(&logical, dist, Order::any()); BatchHashAgg { base, logical } } 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 2f5595508a970..3453846f292fa 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -50,7 +50,7 @@ impl BatchHashJoin { logical.right.distribution(), &logical, ); - let base = PlanBase::new_batch_from_logical(&logical, dist, Order::any()); + let base = PlanBase::new_batch_with_core(&logical, dist, Order::any()); Self { base, 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 c4b84b7232d1a..b3c7e96e9d51f 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs @@ -44,7 +44,7 @@ impl BatchHopWindow { let distribution = logical .i2o_col_mapping() .rewrite_provided_distribution(logical.input.distribution()); - let base = PlanBase::new_batch_from_logical( + let base = PlanBase::new_batch_with_core( &logical, distribution, logical.get_out_column_index_order(), diff --git a/src/frontend/src/optimizer/plan_node/batch_insert.rs b/src/frontend/src/optimizer/plan_node/batch_insert.rs index dc6178eb1467b..458bf037ba710 100644 --- a/src/frontend/src/optimizer/plan_node/batch_insert.rs +++ b/src/frontend/src/optimizer/plan_node/batch_insert.rs @@ -35,7 +35,7 @@ pub struct BatchInsert { impl BatchInsert { pub fn new(logical: generic::Insert) -> Self { assert_eq!(logical.input.distribution(), &Distribution::Single); - let base: PlanBase = PlanBase::new_batch_from_logical( + let base: PlanBase = PlanBase::new_batch_with_core( &logical, logical.input.distribution().clone(), Order::any(), diff --git a/src/frontend/src/optimizer/plan_node/batch_limit.rs b/src/frontend/src/optimizer/plan_node/batch_limit.rs index e617f1c2cd544..c395f443e57d8 100644 --- a/src/frontend/src/optimizer/plan_node/batch_limit.rs +++ b/src/frontend/src/optimizer/plan_node/batch_limit.rs @@ -32,7 +32,7 @@ pub struct BatchLimit { impl BatchLimit { pub fn new(logical: generic::Limit) -> Self { - let base = PlanBase::new_batch_from_logical( + let base = PlanBase::new_batch_with_core( &logical, logical.input.distribution().clone(), logical.input.order().clone(), 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 0bb00752f44d1..5dd98f18322b6 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -68,7 +68,7 @@ impl BatchLookupJoin { assert!(eq_join_predicate.has_eq()); assert!(eq_join_predicate.eq_keys_are_type_aligned()); let dist = Self::derive_dist(logical.left.distribution(), &logical); - let base = PlanBase::new_batch_from_logical(&logical, dist, Order::any()); + let base = PlanBase::new_batch_with_core(&logical, dist, Order::any()); Self { base, logical, 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 ed8be832dde3d..3a2a69021faf0 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 @@ -37,7 +37,7 @@ pub struct BatchNestedLoopJoin { impl BatchNestedLoopJoin { pub fn new(logical: generic::Join) -> Self { let dist = Self::derive_dist(logical.left.distribution(), logical.right.distribution()); - let base = PlanBase::new_batch_from_logical(&logical, dist, Order::any()); + let base = PlanBase::new_batch_with_core(&logical, dist, Order::any()); Self { base, logical } } 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 aa6e53246697e..9de951ae22eef 100644 --- a/src/frontend/src/optimizer/plan_node/batch_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_over_window.rs @@ -47,7 +47,7 @@ impl BatchOverWindow { .collect(), ); - let base = PlanBase::new_batch_from_logical(&logical, input_dist, order); + let base = PlanBase::new_batch_with_core(&logical, input_dist, order); BatchOverWindow { base, logical } } diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index d3979b8aebdee..d82970605db50 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -43,7 +43,7 @@ impl BatchProject { .i2o_col_mapping() .rewrite_provided_order(logical.input.order()); - let base = PlanBase::new_batch_from_logical(&logical, distribution, order); + let base = PlanBase::new_batch_with_core(&logical, distribution, order); BatchProject { base, logical } } 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 b86211aaaa211..c9cbc88a45461 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project_set.rs @@ -38,7 +38,7 @@ impl BatchProjectSet { .i2o_col_mapping() .rewrite_provided_distribution(logical.input.distribution()); - let base = PlanBase::new_batch_from_logical( + let base = PlanBase::new_batch_with_core( &logical, distribution, logical.get_out_column_index_order(), 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 d2e982d11c449..353254eb84b16 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -47,7 +47,7 @@ impl BatchSeqScan { } else { logical.get_out_column_index_order() }; - let base = PlanBase::new_batch_from_logical(&logical, dist, order); + let base = PlanBase::new_batch_with_core(&logical, dist, order); { // validate scan_range 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 a98432b002028..6171d44cb69c7 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -32,7 +32,7 @@ pub struct BatchSimpleAgg { impl BatchSimpleAgg { pub fn new(logical: generic::Agg) -> Self { let input_dist = logical.input.distribution().clone(); - let base = PlanBase::new_batch_from_logical(&logical, input_dist, Order::any()); + let base = PlanBase::new_batch_with_core(&logical, input_dist, Order::any()); BatchSimpleAgg { base, logical } } 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 241f1195352e3..f72025ef6c757 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -56,7 +56,7 @@ impl BatchSortAgg { .i2o_col_mapping() .rewrite_provided_order(&input_order); - let base = PlanBase::new_batch_from_logical(&logical, dist, order); + let base = PlanBase::new_batch_with_core(&logical, dist, order); BatchSortAgg { base, diff --git a/src/frontend/src/optimizer/plan_node/batch_source.rs b/src/frontend/src/optimizer/plan_node/batch_source.rs index 5557a8fba0c8c..e9d18c4a09135 100644 --- a/src/frontend/src/optimizer/plan_node/batch_source.rs +++ b/src/frontend/src/optimizer/plan_node/batch_source.rs @@ -35,7 +35,7 @@ pub struct BatchSource { impl BatchSource { pub fn new(logical: generic::Source) -> Self { - let base = PlanBase::new_batch_from_logical( + let base = PlanBase::new_batch_with_core( &logical, // Use `Single` by default, will be updated later with `clone_with_dist`. Distribution::Single, diff --git a/src/frontend/src/optimizer/plan_node/batch_topn.rs b/src/frontend/src/optimizer/plan_node/batch_topn.rs index b8b5ba710e468..4c787840171e1 100644 --- a/src/frontend/src/optimizer/plan_node/batch_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_topn.rs @@ -35,7 +35,7 @@ pub struct BatchTopN { impl BatchTopN { pub fn new(logical: generic::TopN) -> Self { assert!(logical.group_key.is_empty()); - let base = PlanBase::new_batch_from_logical( + let base = PlanBase::new_batch_with_core( &logical, logical.input.distribution().clone(), // BatchTopN outputs data in the order of specified order diff --git a/src/frontend/src/optimizer/plan_node/batch_union.rs b/src/frontend/src/optimizer/plan_node/batch_union.rs index 1626d32db2cc8..8d23a3d9367c0 100644 --- a/src/frontend/src/optimizer/plan_node/batch_union.rs +++ b/src/frontend/src/optimizer/plan_node/batch_union.rs @@ -40,7 +40,7 @@ impl BatchUnion { Distribution::SomeShard }; - let base = PlanBase::new_batch_from_logical(&logical, dist, Order::any()); + let base = PlanBase::new_batch_with_core(&logical, dist, Order::any()); BatchUnion { base, logical } } } diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 7f3d50b8ee010..ee118c2beb8fb 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -205,7 +205,7 @@ impl LogicalSource { ..self.core.clone() }; let mut new_s3_plan: PlanRef = StreamSource { - base: PlanBase::new_stream_with_logical( + base: PlanBase::new_stream_with_core( &logical_source, Distribution::Single, true, // `list` will keep listing all objects, it must be 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 3f5a169976153..31dc16123be5f 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -43,18 +43,21 @@ 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 stream_key: Option>, functional_dependency: FunctionalDependencySet, + // -- physical-only fields -- + /// 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, + // -- batch-only fields -- /// The order property of the PlanNode's output, store an `&Order::any()` here will not affect /// correctness, but insert unnecessary sort in plan order: Order, // -- stream-only fields -- - /// 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, /// The append-only property of the PlanNode's output is a stream-only property. Append-only /// means the stream contains only insert operation. append_only: bool, @@ -140,31 +143,12 @@ impl PlanBase { } } - pub fn new_logical_with_core(node: &impl GenericPlanNode) -> Self { + pub fn new_logical_with_core(core: &impl GenericPlanNode) -> Self { Self::new_logical( - node.ctx(), - node.schema(), - node.stream_key(), - node.functional_dependency(), - ) - } - - pub fn new_stream_with_logical( - logical: &impl GenericPlanNode, - dist: Distribution, - append_only: bool, - emit_on_window_close: bool, - watermark_columns: FixedBitSet, - ) -> Self { - Self::new_stream( - logical.ctx(), - logical.schema(), - logical.stream_key(), - logical.functional_dependency(), - dist, - append_only, - emit_on_window_close, - watermark_columns, + core.ctx(), + core.schema(), + core.stream_key(), + core.functional_dependency(), ) } @@ -194,12 +178,23 @@ impl PlanBase { } } - pub fn new_batch_from_logical( - logical: &impl GenericPlanNode, + pub fn new_stream_with_core( + core: &impl GenericPlanNode, dist: Distribution, - order: Order, + append_only: bool, + emit_on_window_close: bool, + watermark_columns: FixedBitSet, ) -> Self { - Self::new_batch(logical.ctx(), logical.schema(), dist, order) + Self::new_stream( + core.ctx(), + core.schema(), + core.stream_key(), + core.functional_dependency(), + dist, + append_only, + emit_on_window_close, + watermark_columns, + ) } pub fn new_batch( @@ -226,6 +221,14 @@ impl PlanBase { } } + pub fn new_batch_with_core( + core: &impl GenericPlanNode, + dist: Distribution, + order: Order, + ) -> Self { + Self::new_batch(core.ctx(), core.schema(), dist, order) + } + pub fn clone_with_new_plan_id(&self) -> Self { let mut new = self.clone(); new.id = self.ctx().next_plan_node_id(); diff --git a/src/frontend/src/optimizer/plan_node/stream_dedup.rs b/src/frontend/src/optimizer/plan_node/stream_dedup.rs index d658be70d9df9..9aa470c671e96 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dedup.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dedup.rs @@ -37,7 +37,7 @@ impl StreamDedup { // A dedup operator must be append-only. assert!(input.append_only()); - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &logical, input.distribution().clone(), true, 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 2ec332f85a513..61f6aeb9c72df 100644 --- a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs @@ -67,7 +67,7 @@ impl StreamDeltaJoin { logical.i2o_col_mapping().rewrite_bitset(&watermark_columns) }; // TODO: derive from input - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &logical, dist, append_only, 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 6f1a617ee1236..a4b74f37208e7 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs @@ -38,7 +38,7 @@ impl StreamDynamicFilter { let watermark_columns = core.watermark_columns(core.right().watermark_columns()[0]); // TODO: derive from input - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, core.left().distribution().clone(), false, /* we can have a new abstraction for append only and monotonically increasing 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 797091338915e..922fd1e59812d 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 @@ -50,7 +50,7 @@ impl StreamEowcOverWindow { // ancient rows in some rarely updated partitions that are emitted at the end of time. let watermark_columns = FixedBitSet::with_capacity(logical.output_len()); - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &logical, input.distribution().clone(), true, diff --git a/src/frontend/src/optimizer/plan_node/stream_expand.rs b/src/frontend/src/optimizer/plan_node/stream_expand.rs index c7a59b1f847fe..079374760507c 100644 --- a/src/frontend/src/optimizer/plan_node/stream_expand.rs +++ b/src/frontend/src/optimizer/plan_node/stream_expand.rs @@ -48,7 +48,7 @@ impl StreamExpand { .map(|idx| idx + input.schema().len()), ); - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &logical, dist, input.append_only(), diff --git a/src/frontend/src/optimizer/plan_node/stream_filter.rs b/src/frontend/src/optimizer/plan_node/stream_filter.rs index ed4d506b47aef..7b920b9fe30e2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_filter.rs @@ -34,7 +34,7 @@ impl StreamFilter { let input = logical.input.clone(); let dist = input.distribution().clone(); // Filter executor won't change the append-only behavior of the stream. - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &logical, dist, input.append_only(), 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 4a1e449a21ab6..f33b65604f443 100644 --- a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs +++ b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs @@ -48,7 +48,7 @@ impl_plan_tree_node_for_unary! { StreamFsFetch } impl StreamFsFetch { pub fn new(input: PlanRef, source: generic::Source) -> Self { - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &source, Distribution::SomeShard, source.catalog.as_ref().map_or(true, |s| s.append_only), 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 7c9a22e0bb095..5959da98609e0 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -86,7 +86,7 @@ impl StreamHashAgg { } // Hash agg executor might change the append-only behavior of the stream. - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &logical, dist, emit_on_window_close, // in EOWC mode, we produce append only output 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 96ca5b28fa823..61272a41d47c3 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -189,7 +189,7 @@ impl StreamHashJoin { }; // TODO: derive from input - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &logical, dist, append_only, 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 abe1b33426343..e39f536172e19 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -57,7 +57,7 @@ impl StreamHopWindow { ) .rewrite_bitset(&watermark_columns); - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &logical, dist, input.append_only(), 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 c4816c8639db3..20d45e4bb01c7 100644 --- a/src/frontend/src/optimizer/plan_node/stream_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_over_window.rs @@ -38,7 +38,7 @@ impl StreamOverWindow { let input = &logical.input; let watermark_columns = FixedBitSet::with_capacity(logical.output_len()); - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &logical, input.distribution().clone(), false, // general over window cannot be append-only diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index c609a86d050f8..50fef2f1d937d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -80,7 +80,7 @@ impl StreamProject { } // Project executor won't change the append-only behavior of the stream, so it depends on // input's `append_only`. - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &logical, distribution, input.append_only(), 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 97c4b70433cb1..0130c53ff1840 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project_set.rs @@ -66,7 +66,7 @@ impl StreamProjectSet { // ProjectSet executor won't change the append-only behavior of the stream, so it depends on // input's `append_only`. - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &logical, distribution, input.append_only(), diff --git a/src/frontend/src/optimizer/plan_node/stream_share.rs b/src/frontend/src/optimizer/plan_node/stream_share.rs index 32273f6994b87..06de3eb9f602c 100644 --- a/src/frontend/src/optimizer/plan_node/stream_share.rs +++ b/src/frontend/src/optimizer/plan_node/stream_share.rs @@ -34,7 +34,7 @@ impl StreamShare { let input = logical.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_logical( + let base = PlanBase::new_stream_with_core( &logical, dist, input.append_only(), 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 07dc801b390b9..9471d2333c5e6 100644 --- a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs @@ -51,7 +51,7 @@ impl StreamSimpleAgg { // Simple agg executor might change the append-only behavior of the stream. let base = - PlanBase::new_stream_with_logical(&logical, dist, false, false, watermark_columns); + PlanBase::new_stream_with_core(&logical, dist, false, false, watermark_columns); StreamSimpleAgg { base, logical, diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index a870be1de5840..e0b044f357ab0 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -37,7 +37,7 @@ pub struct StreamSource { impl StreamSource { pub fn new(logical: generic::Source) -> Self { - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &logical, Distribution::SomeShard, logical.catalog.as_ref().map_or(true, |s| s.append_only), 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 296c58944fac0..0046bca258bf9 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 @@ -49,7 +49,7 @@ impl StreamStatelessSimpleAgg { } } - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &logical, input_dist.clone(), input.append_only(), 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 7b0800798b862..21a02581b9aee 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -69,7 +69,7 @@ impl StreamTableScan { None => Distribution::SomeShard, } }; - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &logical, distribution, logical.table_desc.append_only, 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 0d53aab22ef52..b904485da5d6c 100644 --- a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs @@ -65,7 +65,7 @@ impl StreamTemporalJoin { .rewrite_bitset(logical.left.watermark_columns()), ); - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &logical, dist, true, diff --git a/src/frontend/src/optimizer/plan_node/stream_topn.rs b/src/frontend/src/optimizer/plan_node/stream_topn.rs index 641b144242a92..969d747941a76 100644 --- a/src/frontend/src/optimizer/plan_node/stream_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_topn.rs @@ -41,7 +41,7 @@ impl StreamTopN { let watermark_columns = FixedBitSet::with_capacity(input.schema().len()); let base = - PlanBase::new_stream_with_logical(&logical, dist, false, false, watermark_columns); + PlanBase::new_stream_with_core(&logical, dist, false, false, watermark_columns); StreamTopN { base, logical } } diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs index 9d3f279f28b7f..9348ccf132a59 100644 --- a/src/frontend/src/optimizer/plan_node/stream_union.rs +++ b/src/frontend/src/optimizer/plan_node/stream_union.rs @@ -47,7 +47,7 @@ impl StreamUnion { |acc_watermark_columns, input| acc_watermark_columns.bitand(input.watermark_columns()), ); - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &logical, dist, inputs.iter().all(|x| x.append_only()), From 5c0d5f98bc19aa0ea7c8aa932b5af9c531f862cc Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 20 Oct 2023 18:50:59 +0800 Subject: [PATCH 05/11] more type exercises Signed-off-by: Bugen Zhao --- .../src/optimizer/plan_node/batch_limit.rs | 1 + .../optimizer/plan_node/batch_over_window.rs | 1 + .../src/optimizer/plan_node/batch_project.rs | 1 + .../src/optimizer/plan_node/batch_seq_scan.rs | 3 +- .../src/optimizer/plan_node/batch_source.rs | 1 + .../src/optimizer/plan_node/batch_update.rs | 1 + .../src/optimizer/plan_node/generic/mod.rs | 3 +- .../src/optimizer/plan_node/logical_apply.rs | 4 +- .../src/optimizer/plan_node/logical_expand.rs | 1 + .../src/optimizer/plan_node/logical_filter.rs | 1 + .../optimizer/plan_node/logical_hop_window.rs | 2 +- .../optimizer/plan_node/logical_multi_join.rs | 1 + .../src/optimizer/plan_node/logical_update.rs | 1 + .../src/optimizer/plan_node/merge_eq_nodes.rs | 1 + src/frontend/src/optimizer/plan_node/mod.rs | 38 +++- .../src/optimizer/plan_node/plan_base.rs | 170 +++++++++--------- .../src/optimizer/plan_node/stream_dml.rs | 1 + .../plan_node/stream_eowc_over_window.rs | 2 +- .../optimizer/plan_node/stream_exchange.rs | 2 +- .../optimizer/plan_node/stream_group_topn.rs | 2 +- .../optimizer/plan_node/stream_hash_agg.rs | 2 +- .../optimizer/plan_node/stream_hop_window.rs | 1 + .../src/optimizer/plan_node/stream_project.rs | 1 + .../src/optimizer/plan_node/stream_share.rs | 2 + .../optimizer/plan_node/stream_simple_agg.rs | 3 +- .../src/optimizer/plan_node/stream_sort.rs | 2 +- .../plan_node/stream_stateless_simple_agg.rs | 1 + .../optimizer/plan_node/stream_table_scan.rs | 2 + .../src/optimizer/plan_node/stream_topn.rs | 3 +- .../src/optimizer/plan_node/stream_union.rs | 1 + .../src/optimizer/plan_node/stream_values.rs | 1 + .../optimizer/plan_rewriter/plan_cloner.rs | 1 + .../plan_rewriter/share_source_rewriter.rs | 1 + .../plan_visitor/share_parent_counter.rs | 1 + .../rule/always_false_filter_rule.rs | 1 + .../rule/apply_join_transpose_rule.rs | 1 + .../optimizer/rule/index_selection_rule.rs | 1 + .../rule/left_deep_tree_join_ordering_rule.rs | 1 + .../optimizer/rule/merge_multijoin_rule.rs | 1 + .../optimizer/rule/min_max_on_index_rule.rs | 2 +- .../rule/over_window_to_topn_rule.rs | 1 + .../rule/pull_up_correlated_predicate_rule.rs | 1 + .../rule/trivial_project_to_values_rule.rs | 1 + .../rule/union_input_values_merge_rule.rs | 1 + 44 files changed, 166 insertions(+), 104 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/batch_limit.rs b/src/frontend/src/optimizer/plan_node/batch_limit.rs index c395f443e57d8..c6e8091705443 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::generic::PhysicalPlanRef; 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_over_window.rs b/src/frontend/src/optimizer/plan_node/batch_over_window.rs index 9de951ae22eef..ba2ec93053406 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::BatchPlanRef; use super::generic::PlanWindowFunction; use super::utils::impl_distill_by_unit; use super::{ diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index d82970605db50..34ee2c6de174c 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -18,6 +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::utils::{childless_record, Distill}; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, 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 353254eb84b16..4785e64b91562 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -24,7 +24,8 @@ 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::generic::GenericPlanRef; +use super::batch::BatchPlanRef; +use super::generic::{GenericPlanRef, PhysicalPlanRef}; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch}; use crate::catalog::ColumnId; diff --git a/src/frontend/src/optimizer/plan_node/batch_source.rs b/src/frontend/src/optimizer/plan_node/batch_source.rs index e9d18c4a09135..e956892e78a5f 100644 --- a/src/frontend/src/optimizer/plan_node/batch_source.rs +++ b/src/frontend/src/optimizer/plan_node/batch_source.rs @@ -19,6 +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::utils::{childless_record, column_names_pretty, Distill}; use super::{ generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch, ToLocalBatch, diff --git a/src/frontend/src/optimizer/plan_node/batch_update.rs b/src/frontend/src/optimizer/plan_node/batch_update.rs index 19bb60b9aa1d8..49c2d57b138c1 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::generic::GenericPlanRef; 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/generic/mod.rs b/src/frontend/src/optimizer/plan_node/generic/mod.rs index ca7205b3ac324..aec59c90bcc4e 100644 --- a/src/frontend/src/optimizer/plan_node/generic/mod.rs +++ b/src/frontend/src/optimizer/plan_node/generic/mod.rs @@ -18,7 +18,7 @@ use std::hash::Hash; use pretty_xmlish::XmlNode; use risingwave_common::catalog::Schema; -use super::{stream, EqJoinPredicate}; +use super::{stream, EqJoinPredicate, PlanNodeId}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::property::{Distribution, FunctionalDependencySet}; @@ -85,6 +85,7 @@ macro_rules! impl_distill_unit_from_fields { pub(super) use impl_distill_unit_from_fields; pub trait GenericPlanRef: Eq + Hash { + fn id(&self) -> PlanNodeId; fn schema(&self) -> &Schema; fn stream_key(&self) -> Option<&[usize]>; fn functional_dependency(&self) -> &FunctionalDependencySet; diff --git a/src/frontend/src/optimizer/plan_node/logical_apply.rs b/src/frontend/src/optimizer/plan_node/logical_apply.rs index 7640f093fc933..b398ce7494f61 100644 --- a/src/frontend/src/optimizer/plan_node/logical_apply.rs +++ b/src/frontend/src/optimizer/plan_node/logical_apply.rs @@ -18,7 +18,9 @@ use risingwave_common::catalog::Schema; use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_pb::plan_common::JoinType; -use super::generic::{self, push_down_into_join, push_down_join_condition, GenericPlanNode}; +use super::generic::{ + self, push_down_into_join, push_down_join_condition, GenericPlanNode, GenericPlanRef, +}; use super::utils::{childless_record, Distill}; use super::{ ColPrunable, LogicalJoin, LogicalProject, PlanBase, PlanRef, PlanTreeNodeBinary, diff --git a/src/frontend/src/optimizer/plan_node/logical_expand.rs b/src/frontend/src/optimizer/plan_node/logical_expand.rs index 5cc823cb1e1b7..d1f3b666feef5 100644 --- a/src/frontend/src/optimizer/plan_node/logical_expand.rs +++ b/src/frontend/src/optimizer/plan_node/logical_expand.rs @@ -15,6 +15,7 @@ use itertools::Itertools; 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, diff --git a/src/frontend/src/optimizer/plan_node/logical_filter.rs b/src/frontend/src/optimizer/plan_node/logical_filter.rs index 4d5ab47c8fa1d..a62b91aac5277 100644 --- a/src/frontend/src/optimizer/plan_node/logical_filter.rs +++ b/src/frontend/src/optimizer/plan_node/logical_filter.rs @@ -18,6 +18,7 @@ use risingwave_common::bail; use risingwave_common::error::Result; 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, 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 ac193f068ba0d..da2ec2138c3d1 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -17,7 +17,7 @@ use itertools::Itertools; use risingwave_common::error::Result; use risingwave_common::types::Interval; -use super::generic::GenericPlanNode; +use super::generic::{GenericPlanNode, GenericPlanRef}; use super::utils::impl_distill_by_unit; use super::{ gen_filter_and_pushdown, generic, BatchHopWindow, ColPrunable, ExprRewritable, LogicalFilter, 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 dc4d627f3f37b..9b740abd7718e 100644 --- a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs @@ -863,6 +863,7 @@ mod test { use super::*; use crate::expr::{FunctionCall, InputRef}; use crate::optimizer::optimizer_context::OptimizerContext; + use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::LogicalValues; use crate::optimizer::property::FunctionalDependency; #[tokio::test] diff --git a/src/frontend/src/optimizer/plan_node/logical_update.rs b/src/frontend/src/optimizer/plan_node/logical_update.rs index 80e4f350d8edb..1dbe1d3d3c5c9 100644 --- a/src/frontend/src/optimizer/plan_node/logical_update.rs +++ b/src/frontend/src/optimizer/plan_node/logical_update.rs @@ -15,6 +15,7 @@ use risingwave_common::catalog::TableVersionId; 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, diff --git a/src/frontend/src/optimizer/plan_node/merge_eq_nodes.rs b/src/frontend/src/optimizer/plan_node/merge_eq_nodes.rs index 73f82e86aa260..9f2e8d94634be 100644 --- a/src/frontend/src/optimizer/plan_node/merge_eq_nodes.rs +++ b/src/frontend/src/optimizer/plan_node/merge_eq_nodes.rs @@ -15,6 +15,7 @@ use std::collections::HashMap; use std::hash::Hash; +use super::generic::GenericPlanRef; use super::{EndoPlan, LogicalShare, PlanNodeId, PlanRef, PlanTreeNodeUnary, VisitPlan}; use crate::optimizer::plan_visitor; use crate::utils::{Endo, Visit}; diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 75113a631d8de..b133f2d140233 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -419,7 +419,28 @@ impl PlanTreeNode for PlanRef { } } -impl GenericPlanRef for PlanRef { +impl PlanNodeMeta for PlanRef { + fn node_type(&self) -> PlanNodeType { + self.0.node_type() + } + + fn plan_base(&self) -> &PlanBase { + self.0.plan_base() + } + + fn convention(&self) -> Convention { + self.0.convention() + } +} + +impl

GenericPlanRef for P +where + P: PlanNodeMeta + Eq + Hash, +{ + fn id(&self) -> PlanNodeId { + self.plan_base().id() + } + fn schema(&self) -> &Schema { self.plan_base().schema() } @@ -437,13 +458,19 @@ impl GenericPlanRef for PlanRef { } } -impl PhysicalPlanRef for PlanRef { +impl

PhysicalPlanRef for P +where + P: PlanNodeMeta + Eq + Hash, +{ fn distribution(&self) -> &Distribution { self.plan_base().distribution() } } -impl StreamPlanRef for PlanRef { +impl

StreamPlanRef for P +where + P: PlanNodeMeta + Eq + Hash, +{ fn append_only(&self) -> bool { self.plan_base().append_only() } @@ -457,7 +484,10 @@ impl StreamPlanRef for PlanRef { } } -impl BatchPlanRef for PlanRef { +impl

BatchPlanRef for P +where + P: PlanNodeMeta + Eq + Hash, +{ fn order(&self) -> &Order { self.plan_base().order() } diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs index 31dc16123be5f..2ccbb1b59aeeb 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -14,15 +14,63 @@ use educe::Educe; use fixedbitset::FixedBitSet; -use paste::paste; use risingwave_common::catalog::Schema; use super::generic::GenericPlanNode; use super::*; -use crate::for_all_plan_nodes; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::property::{Distribution, FunctionalDependencySet, Order}; +#[derive(Clone, Debug, PartialEq, Eq, Hash)] +struct StreamExtra { + /// The append-only property of the PlanNode's output is a stream-only property. Append-only + /// means the stream contains only insert operation. + append_only: bool, + /// Whether the output is emitted on window close. + emit_on_window_close: bool, + /// The watermark column indices of the PlanNode's output. There could be watermark output from + /// this stream operator. + watermark_columns: FixedBitSet, +} + +#[derive(Clone, Debug, PartialEq, Eq, Hash)] +struct BatchExtra { + /// The order property of the PlanNode's output, store an `&Order::any()` here will not affect + /// correctness, but insert unnecessary sort in plan + order: Order, +} + +#[derive(Clone, Debug, PartialEq, Eq, Hash)] +enum PhysicalExtraInner { + Stream(StreamExtra), + Batch(BatchExtra), +} + +#[derive(Clone, Debug, PartialEq, Eq, Hash)] +struct PhysicalExtra { + /// 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, + + inner: PhysicalExtraInner, +} + +impl PhysicalExtra { + fn stream(&self) -> &StreamExtra { + match &self.inner { + PhysicalExtraInner::Stream(extra) => extra, + _ => panic!("access stream properties from batch plan node"), + } + } + + fn batch(&self) -> &BatchExtra { + match &self.inner { + PhysicalExtraInner::Batch(extra) => extra, + _ => panic!("access batch properties from stream plan node"), + } + } +} + /// the common fields of all nodes, please make a field named `base` in /// every planNode and correctly value it when construct the planNode. /// @@ -47,35 +95,28 @@ pub struct PlanBase { stream_key: Option>, functional_dependency: FunctionalDependencySet, - // -- physical-only fields -- - /// 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, - - // -- batch-only fields -- - /// The order property of the PlanNode's output, store an `&Order::any()` here will not affect - /// correctness, but insert unnecessary sort in plan - order: Order, - - // -- stream-only fields -- - /// The append-only property of the PlanNode's output is a stream-only property. Append-only - /// means the stream contains only insert operation. - append_only: bool, - /// Whether the output is emitted on window close. - emit_on_window_close: bool, - /// The watermark column indices of the PlanNode's output. There could be watermark output from - /// this stream operator. - watermark_columns: FixedBitSet, + physical_extra: Option, } impl PlanBase { - /// The unique id of the `PlanNode` in this optimizer context. - pub fn id(&self) -> PlanNodeId { - self.id + 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 + } + fn schema(&self) -> &Schema { &self.schema } @@ -95,27 +136,27 @@ impl generic::GenericPlanRef for PlanBase { impl generic::PhysicalPlanRef for PlanBase { fn distribution(&self) -> &Distribution { - &self.dist + &self.physical_extra().dist } } impl stream::StreamPlanRef for PlanBase { fn append_only(&self) -> bool { - self.append_only + self.physical_extra().stream().append_only } fn emit_on_window_close(&self) -> bool { - self.emit_on_window_close + self.physical_extra().stream().emit_on_window_close } fn watermark_columns(&self) -> &FixedBitSet { - &self.watermark_columns + &self.physical_extra().stream().watermark_columns } } impl batch::BatchPlanRef for PlanBase { fn order(&self) -> &Order { - &self.order + &self.physical_extra().batch().order } } @@ -127,19 +168,13 @@ impl PlanBase { functional_dependency: FunctionalDependencySet, ) -> Self { let id = ctx.next_plan_node_id(); - let watermark_columns = FixedBitSet::with_capacity(schema.len()); Self { id, ctx, schema, stream_key, - dist: Distribution::Single, - order: Order::any(), - // Logical plan node won't touch `append_only` field - append_only: true, - emit_on_window_close: false, functional_dependency, - watermark_columns, + physical_extra: None, } } @@ -168,13 +203,16 @@ impl PlanBase { id, ctx, schema, - dist, - order: Order::any(), stream_key, - append_only, - emit_on_window_close, functional_dependency, - watermark_columns, + physical_extra: Some(PhysicalExtra { + dist, + inner: PhysicalExtraInner::Stream(StreamExtra { + append_only, + emit_on_window_close, + watermark_columns, + }), + }), } } @@ -205,19 +243,16 @@ impl PlanBase { ) -> Self { let id = ctx.next_plan_node_id(); let functional_dependency = FunctionalDependencySet::new(schema.len()); - let watermark_columns = FixedBitSet::with_capacity(schema.len()); Self { id, ctx, schema, - dist, - order, stream_key: None, - // Batch plan node won't touch `append_only` field - append_only: true, - emit_on_window_close: false, // TODO(rc): batch EOWC support? functional_dependency, - watermark_columns, + physical_extra: Some(PhysicalExtra { + dist, + inner: PhysicalExtraInner::Batch(BatchExtra { order }), + }), } } @@ -237,7 +272,7 @@ impl PlanBase { pub fn clone_with_new_distribution(&self, dist: Distribution) -> Self { let mut new = self.clone(); - new.dist = dist; + new.physical_extra_mut().dist = dist; new } } @@ -249,40 +284,3 @@ impl PlanBase { &mut self.functional_dependency } } - -macro_rules! impl_base_delegate { - ($( { $convention:ident, $name:ident }),*) => { - $(paste! { - impl [<$convention $name>] { - pub fn id(&self) -> PlanNodeId { - self.plan_base().id - } - pub fn ctx(&self) -> OptimizerContextRef { - self.plan_base().ctx() - } - 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 - } - pub fn distribution(&self) -> &Distribution { - &self.plan_base().dist - } - 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 functional_dependency(&self) -> &FunctionalDependencySet { - &self.plan_base().functional_dependency - } - } - })* - } -} -for_all_plan_nodes! { impl_base_delegate } diff --git a/src/frontend/src/optimizer/plan_node/stream_dml.rs b/src/frontend/src/optimizer/plan_node/stream_dml.rs index c9f969384c3a4..9b000974786e4 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::StreamPlanRef; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::stream_fragmenter::BuildFragmentGraphState; 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 922fd1e59812d..4e8aff5bba262 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 @@ -18,7 +18,7 @@ use fixedbitset::FixedBitSet; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; -use super::generic::{self, PlanWindowFunction}; +use super::generic::{self, GenericPlanRef, PlanWindowFunction}; 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_exchange.rs b/src/frontend/src/optimizer/plan_node/stream_exchange.rs index 6166a5553cc9d..99e6c3c5161a1 100644 --- a/src/frontend/src/optimizer/plan_node/stream_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/stream_exchange.rs @@ -16,7 +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::PhysicalPlanRef; +use super::generic::{GenericPlanRef, PhysicalPlanRef}; use super::stream::StreamPlanRef; use super::utils::{childless_record, plan_node_name, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; 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 79240a27d3fd4..03bbafbcb2274 100644 --- a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs @@ -16,7 +16,7 @@ use fixedbitset::FixedBitSet; use pretty_xmlish::XmlNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; -use super::generic::{DistillUnit, TopNLimit}; +use super::generic::{DistillUnit, GenericPlanRef, TopNLimit}; 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 5959da98609e0..78c3d758a5a47 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -18,7 +18,7 @@ use pretty_xmlish::XmlNode; use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; -use super::generic::{self, PlanAggCall}; +use super::generic::{self, GenericPlanRef, PlanAggCall}; 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_hop_window.rs b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs index e39f536172e19..26822b6a5cb53 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -17,6 +17,7 @@ use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::HopWindowNode; +use super::generic::GenericPlanRef; 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.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index 50fef2f1d937d..3648cc21d17a0 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -17,6 +17,7 @@ use pretty_xmlish::XmlNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::ProjectNode; +use super::generic::GenericPlanRef; 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_share.rs b/src/frontend/src/optimizer/plan_node/stream_share.rs index 06de3eb9f602c..fb53ffea364d2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_share.rs +++ b/src/frontend/src/optimizer/plan_node/stream_share.rs @@ -16,6 +16,8 @@ use pretty_xmlish::XmlNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::PbStreamNode; +use super::generic::GenericPlanRef; +use super::stream::StreamPlanRef; use super::utils::Distill; use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamExchange, StreamNode}; use crate::optimizer::plan_node::{LogicalShare, PlanBase, PlanTreeNode}; 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 9471d2333c5e6..ad1e80910c6b2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs @@ -50,8 +50,7 @@ impl StreamSimpleAgg { let watermark_columns = FixedBitSet::with_capacity(logical.output_len()); // Simple agg executor might change the append-only behavior of the stream. - let base = - PlanBase::new_stream_with_core(&logical, dist, false, false, watermark_columns); + let base = PlanBase::new_stream_with_core(&logical, dist, false, false, watermark_columns); StreamSimpleAgg { base, logical, diff --git a/src/frontend/src/optimizer/plan_node/stream_sort.rs b/src/frontend/src/optimizer/plan_node/stream_sort.rs index 0ca0429ad8bff..41a56a0fd5df2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sort.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sort.rs @@ -20,7 +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::PhysicalPlanRef; +use super::generic::{GenericPlanRef, PhysicalPlanRef}; use super::stream::StreamPlanRef; use super::utils::{childless_record, Distill, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; 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 0046bca258bf9..3e8f560267ecc 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 @@ -20,6 +20,7 @@ use super::generic::{self, PlanAggCall}; use super::utils::impl_distill_by_unit; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; +use crate::optimizer::plan_node::generic::PhysicalPlanRef; use crate::optimizer::property::RequiredDist; use crate::stream_fragmenter::BuildFragmentGraphState; 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 21a02581b9aee..6d8e75ebec6e6 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -24,11 +24,13 @@ 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::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; diff --git a/src/frontend/src/optimizer/plan_node/stream_topn.rs b/src/frontend/src/optimizer/plan_node/stream_topn.rs index 969d747941a76..20a0fe01b47a9 100644 --- a/src/frontend/src/optimizer/plan_node/stream_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_topn.rs @@ -40,8 +40,7 @@ impl StreamTopN { }; let watermark_columns = FixedBitSet::with_capacity(input.schema().len()); - let base = - PlanBase::new_stream_with_core(&logical, dist, false, false, watermark_columns); + let base = PlanBase::new_stream_with_core(&logical, dist, false, false, watermark_columns); StreamTopN { base, logical } } diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs index 9348ccf132a59..f5ce62f334519 100644 --- a/src/frontend/src/optimizer/plan_node/stream_union.rs +++ b/src/frontend/src/optimizer/plan_node/stream_union.rs @@ -19,6 +19,7 @@ use pretty_xmlish::XmlNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::UnionNode; +use super::generic::GenericPlanRef; 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_values.rs b/src/frontend/src/optimizer/plan_node/stream_values.rs index fb0b844411f63..f8cc5db851159 100644 --- a/src/frontend/src/optimizer/plan_node/stream_values.rs +++ b/src/frontend/src/optimizer/plan_node/stream_values.rs @@ -18,6 +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::utils::{childless_record, Distill}; use super::{ExprRewritable, LogicalValues, PlanBase, StreamNode}; use crate::expr::{Expr, ExprImpl}; diff --git a/src/frontend/src/optimizer/plan_rewriter/plan_cloner.rs b/src/frontend/src/optimizer/plan_rewriter/plan_cloner.rs index f30f3d9fa4966..7e53b903ac962 100644 --- a/src/frontend/src/optimizer/plan_rewriter/plan_cloner.rs +++ b/src/frontend/src/optimizer/plan_rewriter/plan_cloner.rs @@ -16,6 +16,7 @@ use std::collections::HashMap; use itertools::Itertools; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{LogicalShare, PlanNodeId, PlanTreeNode, StreamShare}; use crate::optimizer::PlanRewriter; use crate::PlanRef; diff --git a/src/frontend/src/optimizer/plan_rewriter/share_source_rewriter.rs b/src/frontend/src/optimizer/plan_rewriter/share_source_rewriter.rs index db7ecd92c7aa5..b1042c3c3f98f 100644 --- a/src/frontend/src/optimizer/plan_rewriter/share_source_rewriter.rs +++ b/src/frontend/src/optimizer/plan_rewriter/share_source_rewriter.rs @@ -17,6 +17,7 @@ use std::collections::{HashMap, HashSet}; use itertools::Itertools; use crate::catalog::SourceId; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ LogicalShare, LogicalSource, PlanNodeId, PlanTreeNode, StreamShare, }; diff --git a/src/frontend/src/optimizer/plan_visitor/share_parent_counter.rs b/src/frontend/src/optimizer/plan_visitor/share_parent_counter.rs index 2b21f9d806e72..907fa183c7a99 100644 --- a/src/frontend/src/optimizer/plan_visitor/share_parent_counter.rs +++ b/src/frontend/src/optimizer/plan_visitor/share_parent_counter.rs @@ -15,6 +15,7 @@ use std::collections::HashMap; use super::{DefaultBehavior, DefaultValue}; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{LogicalShare, PlanNodeId, PlanTreeNodeUnary}; use crate::optimizer::plan_visitor::PlanVisitor; diff --git a/src/frontend/src/optimizer/rule/always_false_filter_rule.rs b/src/frontend/src/optimizer/rule/always_false_filter_rule.rs index 02165232372e4..eeba7d9f3be3b 100644 --- a/src/frontend/src/optimizer/rule/always_false_filter_rule.rs +++ b/src/frontend/src/optimizer/rule/always_false_filter_rule.rs @@ -15,6 +15,7 @@ use risingwave_common::types::ScalarImpl; use super::Rule; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{LogicalFilter, LogicalValues}; use crate::PlanRef; diff --git a/src/frontend/src/optimizer/rule/apply_join_transpose_rule.rs b/src/frontend/src/optimizer/rule/apply_join_transpose_rule.rs index 66579248a76f9..7ac121692c81d 100644 --- a/src/frontend/src/optimizer/rule/apply_join_transpose_rule.rs +++ b/src/frontend/src/optimizer/rule/apply_join_transpose_rule.rs @@ -23,6 +23,7 @@ use crate::expr::{ CorrelatedId, CorrelatedInputRef, Expr, ExprImpl, ExprRewriter, ExprType, FunctionCall, InputRef, }; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{LogicalApply, LogicalFilter, LogicalJoin, PlanTreeNodeBinary}; use crate::optimizer::plan_visitor::{ExprCorrelatedIdFinder, PlanCorrelatedIdFinder}; use crate::optimizer::rule::apply_offset_rewriter::ApplyCorrelatedIndicesConverter; diff --git a/src/frontend/src/optimizer/rule/index_selection_rule.rs b/src/frontend/src/optimizer/rule/index_selection_rule.rs index 9103d1bc906bc..323cc59ef3558 100644 --- a/src/frontend/src/optimizer/rule/index_selection_rule.rs +++ b/src/frontend/src/optimizer/rule/index_selection_rule.rs @@ -66,6 +66,7 @@ use crate::expr::{ FunctionCall, InputRef, }; use crate::optimizer::optimizer_context::OptimizerContextRef; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ generic, ColumnPruningContext, LogicalJoin, LogicalScan, LogicalUnion, PlanTreeNode, PlanTreeNodeBinary, PredicatePushdown, PredicatePushdownContext, diff --git a/src/frontend/src/optimizer/rule/left_deep_tree_join_ordering_rule.rs b/src/frontend/src/optimizer/rule/left_deep_tree_join_ordering_rule.rs index dcbb6f7b015ee..bd2db0ac67cca 100644 --- a/src/frontend/src/optimizer/rule/left_deep_tree_join_ordering_rule.rs +++ b/src/frontend/src/optimizer/rule/left_deep_tree_join_ordering_rule.rs @@ -47,6 +47,7 @@ mod tests { use super::*; use crate::expr::{ExprImpl, FunctionCall, InputRef}; use crate::optimizer::optimizer_context::OptimizerContext; + use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::utils::Condition; #[tokio::test] diff --git a/src/frontend/src/optimizer/rule/merge_multijoin_rule.rs b/src/frontend/src/optimizer/rule/merge_multijoin_rule.rs index c496a906400ae..8682db8491a1d 100644 --- a/src/frontend/src/optimizer/rule/merge_multijoin_rule.rs +++ b/src/frontend/src/optimizer/rule/merge_multijoin_rule.rs @@ -46,6 +46,7 @@ mod tests { use super::*; use crate::expr::{ExprImpl, FunctionCall, InputRef}; use crate::optimizer::optimizer_context::OptimizerContext; + use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::utils::Condition; #[tokio::test] diff --git a/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs b/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs index ea8386bc227f8..c32ae40531cd0 100644 --- a/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs +++ b/src/frontend/src/optimizer/rule/min_max_on_index_rule.rs @@ -27,7 +27,7 @@ use risingwave_expr::aggregate::AggKind; use super::{BoxedRule, Rule}; use crate::expr::{ExprImpl, ExprType, FunctionCall, InputRef}; -use crate::optimizer::plan_node::generic::Agg; +use crate::optimizer::plan_node::generic::{Agg, GenericPlanRef}; use crate::optimizer::plan_node::{ LogicalAgg, LogicalFilter, LogicalScan, LogicalTopN, PlanAggCall, PlanTreeNodeUnary, }; diff --git a/src/frontend/src/optimizer/rule/over_window_to_topn_rule.rs b/src/frontend/src/optimizer/rule/over_window_to_topn_rule.rs index dfb6963c7fb4f..93637d3ba8193 100644 --- a/src/frontend/src/optimizer/rule/over_window_to_topn_rule.rs +++ b/src/frontend/src/optimizer/rule/over_window_to_topn_rule.rs @@ -18,6 +18,7 @@ use risingwave_expr::window_function::WindowFuncKind; use super::Rule; use crate::expr::{collect_input_refs, ExprImpl, ExprType}; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{LogicalFilter, LogicalTopN, PlanTreeNodeUnary}; use crate::optimizer::property::Order; use crate::planner::LIMIT_ALL_COUNT; diff --git a/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_rule.rs b/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_rule.rs index dc5f9c2bc9aba..f34146ba80050 100644 --- a/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_rule.rs +++ b/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_rule.rs @@ -18,6 +18,7 @@ use risingwave_common::util::column_index_mapping::ColIndexMapping; use super::super::plan_node::*; use super::{BoxedRule, Rule}; use crate::expr::{CorrelatedId, CorrelatedInputRef, Expr, ExprImpl, ExprRewriter, InputRef}; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_visitor::{PlanCorrelatedIdFinder, PlanVisitor}; use crate::optimizer::PlanRef; use crate::utils::Condition; diff --git a/src/frontend/src/optimizer/rule/trivial_project_to_values_rule.rs b/src/frontend/src/optimizer/rule/trivial_project_to_values_rule.rs index 9759739490fe6..a13bef3baa9d9 100644 --- a/src/frontend/src/optimizer/rule/trivial_project_to_values_rule.rs +++ b/src/frontend/src/optimizer/rule/trivial_project_to_values_rule.rs @@ -13,6 +13,7 @@ // limitations under the License. use super::{BoxedRule, Rule}; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{LogicalValues, PlanTreeNodeUnary}; use crate::optimizer::plan_visitor::{LogicalCardinalityExt, SideEffectVisitor}; use crate::optimizer::{PlanRef, PlanVisitor}; diff --git a/src/frontend/src/optimizer/rule/union_input_values_merge_rule.rs b/src/frontend/src/optimizer/rule/union_input_values_merge_rule.rs index 8119b8847b600..7b83c017ab781 100644 --- a/src/frontend/src/optimizer/rule/union_input_values_merge_rule.rs +++ b/src/frontend/src/optimizer/rule/union_input_values_merge_rule.rs @@ -13,6 +13,7 @@ // limitations under the License. use super::{BoxedRule, Rule}; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::LogicalValues; use crate::optimizer::{PlanRef, PlanTreeNode}; From f6e34446757d3679d2ee561eae6101738c8c8617 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 20 Oct 2023 21:41:05 +0800 Subject: [PATCH 06/11] fix order Signed-off-by: Bugen Zhao --- src/frontend/src/optimizer/plan_node/mod.rs | 1 + src/frontend/src/optimizer/property/distribution.rs | 6 ++++-- src/frontend/src/optimizer/property/order.rs | 2 +- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index b133f2d140233..8d13badedeb4b 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -567,6 +567,7 @@ impl dyn PlanNode { self.plan_base().order() } + // TODO: avoid no manual delegation pub fn distribution(&self) -> &Distribution { self.plan_base().distribution() } diff --git a/src/frontend/src/optimizer/property/distribution.rs b/src/frontend/src/optimizer/property/distribution.rs index 4fcaf959eac87..b6e7715dd155f 100644 --- a/src/frontend/src/optimizer/property/distribution.rs +++ b/src/frontend/src/optimizer/property/distribution.rs @@ -295,10 +295,12 @@ impl RequiredDist { pub fn enforce_if_not_satisfies( &self, - plan: PlanRef, + mut plan: PlanRef, required_order: &Order, ) -> Result { - let plan = required_order.enforce_if_not_satisfies(plan)?; + if let Convention::Batch = plan.convention() { + plan = required_order.enforce_if_not_satisfies(plan)?; + } if !plan.distribution().satisfies(self) { Ok(self.enforce(plan, required_order)) } else { diff --git a/src/frontend/src/optimizer/property/order.rs b/src/frontend/src/optimizer/property/order.rs index a70bffb13a8ba..19ad7586e1c11 100644 --- a/src/frontend/src/optimizer/property/order.rs +++ b/src/frontend/src/optimizer/property/order.rs @@ -92,7 +92,7 @@ impl Order { } } - pub fn enforce(&self, plan: PlanRef) -> PlanRef { + fn enforce(&self, plan: PlanRef) -> PlanRef { assert_eq!(plan.convention(), Convention::Batch); BatchSort::new(plan, self.clone()).into() } From 6266b96bca9f3ba59355042110e457f985d17772 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Mon, 23 Oct 2023 13:03:53 +0800 Subject: [PATCH 07/11] refine docs and extra structs Signed-off-by: Bugen Zhao --- src/frontend/src/optimizer/plan_node/mod.rs | 7 ++ .../src/optimizer/plan_node/plan_base.rs | 76 +++++++++++++------ 2 files changed, 59 insertions(+), 24 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 8d13badedeb4b..f16ebfb0c792c 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -433,6 +433,7 @@ impl PlanNodeMeta for PlanRef { } } +/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. impl

GenericPlanRef for P where P: PlanNodeMeta + Eq + Hash, @@ -458,6 +459,8 @@ where } } +/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. +// TODO: further constrain the convention to be `Stream` or `Batch`. impl

PhysicalPlanRef for P where P: PlanNodeMeta + Eq + Hash, @@ -467,6 +470,8 @@ where } } +/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. +// TODO: further constrain the convention to be `Stream`. impl

StreamPlanRef for P where P: PlanNodeMeta + Eq + Hash, @@ -484,6 +489,8 @@ where } } +/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. +// TODO: further constrain the convention to be `Batch`. impl

BatchPlanRef for P where P: PlanNodeMeta + Eq + Hash, diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs index 2ccbb1b59aeeb..51b1aa5f41141 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -21,8 +21,20 @@ use super::*; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::property::{Distribution, FunctionalDependencySet, Order}; +/// 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, +} + +/// Extra fields for stream plan nodes. #[derive(Clone, Debug, PartialEq, Eq, Hash)] struct StreamExtra { + /// Common fields for physical plan nodes. + physical: PhysicalCommonExtra, + /// The append-only property of the PlanNode's output is a stream-only property. Append-only /// means the stream contains only insert operation. append_only: bool, @@ -33,39 +45,49 @@ struct StreamExtra { watermark_columns: FixedBitSet, } +/// Extra fields for batch plan nodes. #[derive(Clone, Debug, PartialEq, Eq, Hash)] struct BatchExtra { + /// Common fields for physical plan nodes. + physical: PhysicalCommonExtra, + /// The order property of the PlanNode's output, store an `&Order::any()` here will not affect /// correctness, but insert unnecessary sort in plan order: Order, } +/// Extra fields for physical plan nodes. #[derive(Clone, Debug, PartialEq, Eq, Hash)] -enum PhysicalExtraInner { +enum PhysicalExtra { Stream(StreamExtra), Batch(BatchExtra), } -#[derive(Clone, Debug, PartialEq, Eq, Hash)] -struct PhysicalExtra { - /// 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, +impl PhysicalExtra { + fn common(&self) -> &PhysicalCommonExtra { + match self { + PhysicalExtra::Stream(stream) => &stream.physical, + PhysicalExtra::Batch(batch) => &batch.physical, + } + } - inner: PhysicalExtraInner, -} + fn common_mut(&mut self) -> &mut PhysicalCommonExtra { + match self { + PhysicalExtra::Stream(stream) => &mut stream.physical, + PhysicalExtra::Batch(batch) => &mut batch.physical, + } + } -impl PhysicalExtra { fn stream(&self) -> &StreamExtra { - match &self.inner { - PhysicalExtraInner::Stream(extra) => extra, + match self { + PhysicalExtra::Stream(extra) => extra, _ => panic!("access stream properties from batch plan node"), } } fn batch(&self) -> &BatchExtra { - match &self.inner { - PhysicalExtraInner::Batch(extra) => extra, + match self { + PhysicalExtra::Batch(extra) => extra, _ => panic!("access batch properties from stream plan node"), } } @@ -95,6 +117,7 @@ pub struct PlanBase { stream_key: Option>, functional_dependency: FunctionalDependencySet, + /// Extra fields if the plan node is physical. physical_extra: Option, } @@ -136,7 +159,7 @@ impl generic::GenericPlanRef for PlanBase { impl generic::PhysicalPlanRef for PlanBase { fn distribution(&self) -> &Distribution { - &self.physical_extra().dist + &self.physical_extra().common().dist } } @@ -205,14 +228,14 @@ impl PlanBase { schema, stream_key, functional_dependency, - physical_extra: Some(PhysicalExtra { - dist, - inner: PhysicalExtraInner::Stream(StreamExtra { + physical_extra: Some(PhysicalExtra::Stream({ + StreamExtra { + physical: PhysicalCommonExtra { dist }, append_only, emit_on_window_close, watermark_columns, - }), - }), + } + })), } } @@ -249,10 +272,12 @@ impl PlanBase { schema, stream_key: None, functional_dependency, - physical_extra: Some(PhysicalExtra { - dist, - inner: PhysicalExtraInner::Batch(BatchExtra { order }), - }), + physical_extra: Some(PhysicalExtra::Batch({ + BatchExtra { + physical: PhysicalCommonExtra { dist }, + order, + } + })), } } @@ -270,9 +295,12 @@ impl PlanBase { new } + /// 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().dist = dist; + new.physical_extra_mut().common_mut().dist = dist; new } } From 45fc8059ee208fea24760c4036fb4aab5b3e7e2c Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 24 Oct 2023 13:07:19 +0800 Subject: [PATCH 08/11] revert rename Signed-off-by: Bugen Zhao --- src/frontend/src/optimizer/plan_node/batch_delete.rs | 2 +- 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 | 2 +- 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 | 2 +- src/frontend/src/optimizer/plan_node/batch_simple_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_sort_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_source.rs | 2 +- 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/logical_source.rs | 2 +- src/frontend/src/optimizer/plan_node/plan_base.rs | 4 ++-- src/frontend/src/optimizer/plan_node/stream_dedup.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_delta_join.rs | 2 +- src/frontend/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_expand.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_filter.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_fs_fetch.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/frontend/src/optimizer/plan_node/stream_over_window.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_project.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_project_set.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_share.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_simple_agg.rs | 3 ++- src/frontend/src/optimizer/plan_node/stream_source.rs | 2 +- .../src/optimizer/plan_node/stream_stateless_simple_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_table_scan.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_temporal_join.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_topn.rs | 3 ++- src/frontend/src/optimizer/plan_node/stream_union.rs | 2 +- 43 files changed, 46 insertions(+), 44 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/batch_delete.rs b/src/frontend/src/optimizer/plan_node/batch_delete.rs index ccb99360e13d3..600ec6827e3eb 100644 --- a/src/frontend/src/optimizer/plan_node/batch_delete.rs +++ b/src/frontend/src/optimizer/plan_node/batch_delete.rs @@ -33,7 +33,7 @@ pub struct BatchDelete { impl BatchDelete { pub fn new(logical: generic::Delete) -> Self { assert_eq!(logical.input.distribution(), &Distribution::Single); - let base: PlanBase = PlanBase::new_batch_with_core( + let base: PlanBase = PlanBase::new_batch_from_logical( &logical, logical.input.distribution().clone(), Order::any(), diff --git a/src/frontend/src/optimizer/plan_node/batch_expand.rs b/src/frontend/src/optimizer/plan_node/batch_expand.rs index 0ab34cb731e20..72caa27858be1 100644 --- a/src/frontend/src/optimizer/plan_node/batch_expand.rs +++ b/src/frontend/src/optimizer/plan_node/batch_expand.rs @@ -41,7 +41,7 @@ impl BatchExpand { | Distribution::UpstreamHashShard(_, _) => Distribution::SomeShard, Distribution::Broadcast => unreachable!(), }; - let base = PlanBase::new_batch_with_core(&logical, dist, Order::any()); + let base = PlanBase::new_batch_from_logical(&logical, dist, Order::any()); BatchExpand { base, logical } } diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index 49cada015bced..aadbda9800b16 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -32,7 +32,7 @@ pub struct BatchFilter { impl BatchFilter { pub fn new(logical: generic::Filter) -> Self { // TODO: derive from input - let base = PlanBase::new_batch_with_core( + let base = PlanBase::new_batch_from_logical( &logical, logical.input.distribution().clone(), logical.input.order().clone(), 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 1fcbafe6cedc2..1d61b4e9eb379 100644 --- a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs @@ -33,7 +33,7 @@ pub struct BatchGroupTopN { impl BatchGroupTopN { pub fn new(logical: generic::TopN) -> Self { assert!(!logical.group_key.is_empty()); - let base = PlanBase::new_batch_with_core( + let base = PlanBase::new_batch_from_logical( &logical, logical.input.distribution().clone(), Order::any(), 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 72d5fbd785be3..7100125dcee99 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -42,7 +42,7 @@ impl BatchHashAgg { let dist = logical .i2o_col_mapping() .rewrite_provided_distribution(input_dist); - let base = PlanBase::new_batch_with_core(&logical, dist, Order::any()); + let base = PlanBase::new_batch_from_logical(&logical, dist, Order::any()); BatchHashAgg { base, logical } } 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 3453846f292fa..2f5595508a970 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -50,7 +50,7 @@ impl BatchHashJoin { logical.right.distribution(), &logical, ); - let base = PlanBase::new_batch_with_core(&logical, dist, Order::any()); + let base = PlanBase::new_batch_from_logical(&logical, dist, Order::any()); Self { base, 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 b3c7e96e9d51f..c4b84b7232d1a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs @@ -44,7 +44,7 @@ impl BatchHopWindow { let distribution = logical .i2o_col_mapping() .rewrite_provided_distribution(logical.input.distribution()); - let base = PlanBase::new_batch_with_core( + let base = PlanBase::new_batch_from_logical( &logical, distribution, logical.get_out_column_index_order(), diff --git a/src/frontend/src/optimizer/plan_node/batch_insert.rs b/src/frontend/src/optimizer/plan_node/batch_insert.rs index 458bf037ba710..dc6178eb1467b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_insert.rs +++ b/src/frontend/src/optimizer/plan_node/batch_insert.rs @@ -35,7 +35,7 @@ pub struct BatchInsert { impl BatchInsert { pub fn new(logical: generic::Insert) -> Self { assert_eq!(logical.input.distribution(), &Distribution::Single); - let base: PlanBase = PlanBase::new_batch_with_core( + let base: PlanBase = PlanBase::new_batch_from_logical( &logical, logical.input.distribution().clone(), Order::any(), diff --git a/src/frontend/src/optimizer/plan_node/batch_limit.rs b/src/frontend/src/optimizer/plan_node/batch_limit.rs index c6e8091705443..cc8abf9fe6fde 100644 --- a/src/frontend/src/optimizer/plan_node/batch_limit.rs +++ b/src/frontend/src/optimizer/plan_node/batch_limit.rs @@ -33,7 +33,7 @@ pub struct BatchLimit { impl BatchLimit { pub fn new(logical: generic::Limit) -> Self { - let base = PlanBase::new_batch_with_core( + let base = PlanBase::new_batch_from_logical( &logical, logical.input.distribution().clone(), logical.input.order().clone(), 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 5dd98f18322b6..0bb00752f44d1 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -68,7 +68,7 @@ impl BatchLookupJoin { assert!(eq_join_predicate.has_eq()); assert!(eq_join_predicate.eq_keys_are_type_aligned()); let dist = Self::derive_dist(logical.left.distribution(), &logical); - let base = PlanBase::new_batch_with_core(&logical, dist, Order::any()); + let base = PlanBase::new_batch_from_logical(&logical, dist, Order::any()); Self { base, logical, 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 3a2a69021faf0..ed8be832dde3d 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 @@ -37,7 +37,7 @@ pub struct BatchNestedLoopJoin { impl BatchNestedLoopJoin { pub fn new(logical: generic::Join) -> Self { let dist = Self::derive_dist(logical.left.distribution(), logical.right.distribution()); - let base = PlanBase::new_batch_with_core(&logical, dist, Order::any()); + let base = PlanBase::new_batch_from_logical(&logical, dist, Order::any()); Self { base, logical } } 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 ba2ec93053406..e7f38fd4e24b4 100644 --- a/src/frontend/src/optimizer/plan_node/batch_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_over_window.rs @@ -48,7 +48,7 @@ impl BatchOverWindow { .collect(), ); - let base = PlanBase::new_batch_with_core(&logical, input_dist, order); + let base = PlanBase::new_batch_from_logical(&logical, input_dist, order); BatchOverWindow { base, logical } } diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index 34ee2c6de174c..e92ec10b6867f 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -44,7 +44,7 @@ impl BatchProject { .i2o_col_mapping() .rewrite_provided_order(logical.input.order()); - let base = PlanBase::new_batch_with_core(&logical, distribution, order); + let base = PlanBase::new_batch_from_logical(&logical, distribution, order); BatchProject { base, logical } } 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 c9cbc88a45461..b86211aaaa211 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project_set.rs @@ -38,7 +38,7 @@ impl BatchProjectSet { .i2o_col_mapping() .rewrite_provided_distribution(logical.input.distribution()); - let base = PlanBase::new_batch_with_core( + let base = PlanBase::new_batch_from_logical( &logical, distribution, logical.get_out_column_index_order(), 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 4785e64b91562..b2989dc47a3a1 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -48,7 +48,7 @@ impl BatchSeqScan { } else { logical.get_out_column_index_order() }; - let base = PlanBase::new_batch_with_core(&logical, dist, order); + let base = PlanBase::new_batch_from_logical(&logical, dist, order); { // validate scan_range 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 6171d44cb69c7..a98432b002028 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -32,7 +32,7 @@ pub struct BatchSimpleAgg { impl BatchSimpleAgg { pub fn new(logical: generic::Agg) -> Self { let input_dist = logical.input.distribution().clone(); - let base = PlanBase::new_batch_with_core(&logical, input_dist, Order::any()); + let base = PlanBase::new_batch_from_logical(&logical, input_dist, Order::any()); BatchSimpleAgg { base, logical } } 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 f72025ef6c757..241f1195352e3 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -56,7 +56,7 @@ impl BatchSortAgg { .i2o_col_mapping() .rewrite_provided_order(&input_order); - let base = PlanBase::new_batch_with_core(&logical, dist, order); + let base = PlanBase::new_batch_from_logical(&logical, dist, order); BatchSortAgg { base, diff --git a/src/frontend/src/optimizer/plan_node/batch_source.rs b/src/frontend/src/optimizer/plan_node/batch_source.rs index e956892e78a5f..fc9215a13289b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_source.rs +++ b/src/frontend/src/optimizer/plan_node/batch_source.rs @@ -36,7 +36,7 @@ pub struct BatchSource { impl BatchSource { pub fn new(logical: generic::Source) -> Self { - let base = PlanBase::new_batch_with_core( + let base = PlanBase::new_batch_from_logical( &logical, // Use `Single` by default, will be updated later with `clone_with_dist`. Distribution::Single, diff --git a/src/frontend/src/optimizer/plan_node/batch_topn.rs b/src/frontend/src/optimizer/plan_node/batch_topn.rs index 4c787840171e1..b8b5ba710e468 100644 --- a/src/frontend/src/optimizer/plan_node/batch_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_topn.rs @@ -35,7 +35,7 @@ pub struct BatchTopN { impl BatchTopN { pub fn new(logical: generic::TopN) -> Self { assert!(logical.group_key.is_empty()); - let base = PlanBase::new_batch_with_core( + let base = PlanBase::new_batch_from_logical( &logical, logical.input.distribution().clone(), // BatchTopN outputs data in the order of specified order diff --git a/src/frontend/src/optimizer/plan_node/batch_union.rs b/src/frontend/src/optimizer/plan_node/batch_union.rs index 8d23a3d9367c0..1626d32db2cc8 100644 --- a/src/frontend/src/optimizer/plan_node/batch_union.rs +++ b/src/frontend/src/optimizer/plan_node/batch_union.rs @@ -40,7 +40,7 @@ impl BatchUnion { Distribution::SomeShard }; - let base = PlanBase::new_batch_with_core(&logical, dist, Order::any()); + let base = PlanBase::new_batch_from_logical(&logical, dist, Order::any()); BatchUnion { base, logical } } } diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index ee118c2beb8fb..7f3d50b8ee010 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -205,7 +205,7 @@ impl LogicalSource { ..self.core.clone() }; let mut new_s3_plan: PlanRef = StreamSource { - base: PlanBase::new_stream_with_core( + base: PlanBase::new_stream_with_logical( &logical_source, Distribution::Single, true, // `list` will keep listing all objects, it must be 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 51b1aa5f41141..e76bbee27dd45 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -239,7 +239,7 @@ impl PlanBase { } } - pub fn new_stream_with_core( + pub fn new_stream_with_logical( core: &impl GenericPlanNode, dist: Distribution, append_only: bool, @@ -281,7 +281,7 @@ impl PlanBase { } } - pub fn new_batch_with_core( + pub fn new_batch_from_logical( core: &impl GenericPlanNode, dist: Distribution, order: Order, diff --git a/src/frontend/src/optimizer/plan_node/stream_dedup.rs b/src/frontend/src/optimizer/plan_node/stream_dedup.rs index 9aa470c671e96..d658be70d9df9 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dedup.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dedup.rs @@ -37,7 +37,7 @@ impl StreamDedup { // A dedup operator must be append-only. assert!(input.append_only()); - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &logical, input.distribution().clone(), true, 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 61f6aeb9c72df..2ec332f85a513 100644 --- a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs @@ -67,7 +67,7 @@ impl StreamDeltaJoin { logical.i2o_col_mapping().rewrite_bitset(&watermark_columns) }; // TODO: derive from input - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &logical, dist, append_only, 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..6f1a617ee1236 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs @@ -38,7 +38,7 @@ impl StreamDynamicFilter { let watermark_columns = core.watermark_columns(core.right().watermark_columns()[0]); // TODO: derive from input - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &core, core.left().distribution().clone(), false, /* we can have a new abstraction for append only and monotonically increasing 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 4e8aff5bba262..cdb7785740baa 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 @@ -50,7 +50,7 @@ impl StreamEowcOverWindow { // ancient rows in some rarely updated partitions that are emitted at the end of time. let watermark_columns = FixedBitSet::with_capacity(logical.output_len()); - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &logical, input.distribution().clone(), true, diff --git a/src/frontend/src/optimizer/plan_node/stream_expand.rs b/src/frontend/src/optimizer/plan_node/stream_expand.rs index 079374760507c..c7a59b1f847fe 100644 --- a/src/frontend/src/optimizer/plan_node/stream_expand.rs +++ b/src/frontend/src/optimizer/plan_node/stream_expand.rs @@ -48,7 +48,7 @@ impl StreamExpand { .map(|idx| idx + input.schema().len()), ); - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &logical, dist, input.append_only(), diff --git a/src/frontend/src/optimizer/plan_node/stream_filter.rs b/src/frontend/src/optimizer/plan_node/stream_filter.rs index 7b920b9fe30e2..ed4d506b47aef 100644 --- a/src/frontend/src/optimizer/plan_node/stream_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_filter.rs @@ -34,7 +34,7 @@ impl StreamFilter { let input = logical.input.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( + let base = PlanBase::new_stream_with_logical( &logical, dist, input.append_only(), 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 f33b65604f443..4a1e449a21ab6 100644 --- a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs +++ b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs @@ -48,7 +48,7 @@ impl_plan_tree_node_for_unary! { StreamFsFetch } impl StreamFsFetch { pub fn new(input: PlanRef, source: generic::Source) -> Self { - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &source, Distribution::SomeShard, source.catalog.as_ref().map_or(true, |s| s.append_only), 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 78c3d758a5a47..e6d9b4e4407e0 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -86,7 +86,7 @@ impl StreamHashAgg { } // Hash agg executor might change the append-only behavior of the stream. - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &logical, dist, emit_on_window_close, // in EOWC mode, we produce append only output 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 61272a41d47c3..96ca5b28fa823 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -189,7 +189,7 @@ impl StreamHashJoin { }; // TODO: derive from input - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &logical, dist, append_only, 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 26822b6a5cb53..abf01fa403559 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -58,7 +58,7 @@ impl StreamHopWindow { ) .rewrite_bitset(&watermark_columns); - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &logical, dist, input.append_only(), 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 20d45e4bb01c7..c4816c8639db3 100644 --- a/src/frontend/src/optimizer/plan_node/stream_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_over_window.rs @@ -38,7 +38,7 @@ impl StreamOverWindow { let input = &logical.input; let watermark_columns = FixedBitSet::with_capacity(logical.output_len()); - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &logical, input.distribution().clone(), false, // general over window cannot be append-only diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index 3648cc21d17a0..f4c0aefadb5c4 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -81,7 +81,7 @@ impl StreamProject { } // Project executor won't change the append-only behavior of the stream, so it depends on // input's `append_only`. - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &logical, distribution, input.append_only(), 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 0130c53ff1840..97c4b70433cb1 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project_set.rs @@ -66,7 +66,7 @@ impl StreamProjectSet { // ProjectSet executor won't change the append-only behavior of the stream, so it depends on // input's `append_only`. - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &logical, distribution, input.append_only(), diff --git a/src/frontend/src/optimizer/plan_node/stream_share.rs b/src/frontend/src/optimizer/plan_node/stream_share.rs index fb53ffea364d2..5c01a8c54149a 100644 --- a/src/frontend/src/optimizer/plan_node/stream_share.rs +++ b/src/frontend/src/optimizer/plan_node/stream_share.rs @@ -36,7 +36,7 @@ impl StreamShare { let input = logical.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( + let base = PlanBase::new_stream_with_logical( &logical, dist, input.append_only(), 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 ad1e80910c6b2..07dc801b390b9 100644 --- a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs @@ -50,7 +50,8 @@ impl StreamSimpleAgg { let watermark_columns = FixedBitSet::with_capacity(logical.output_len()); // Simple agg executor might change the append-only behavior of the stream. - let base = PlanBase::new_stream_with_core(&logical, dist, false, false, watermark_columns); + let base = + PlanBase::new_stream_with_logical(&logical, dist, false, false, watermark_columns); StreamSimpleAgg { base, logical, diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index e0b044f357ab0..a870be1de5840 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -37,7 +37,7 @@ pub struct StreamSource { impl StreamSource { pub fn new(logical: generic::Source) -> Self { - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &logical, Distribution::SomeShard, logical.catalog.as_ref().map_or(true, |s| s.append_only), 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 3e8f560267ecc..b99d456eb1d1a 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 @@ -50,7 +50,7 @@ impl StreamStatelessSimpleAgg { } } - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &logical, input_dist.clone(), input.append_only(), 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 6d8e75ebec6e6..797d942c680ee 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -71,7 +71,7 @@ impl StreamTableScan { None => Distribution::SomeShard, } }; - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &logical, distribution, logical.table_desc.append_only, 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 b904485da5d6c..0d53aab22ef52 100644 --- a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs @@ -65,7 +65,7 @@ impl StreamTemporalJoin { .rewrite_bitset(logical.left.watermark_columns()), ); - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &logical, dist, true, diff --git a/src/frontend/src/optimizer/plan_node/stream_topn.rs b/src/frontend/src/optimizer/plan_node/stream_topn.rs index 20a0fe01b47a9..641b144242a92 100644 --- a/src/frontend/src/optimizer/plan_node/stream_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_topn.rs @@ -40,7 +40,8 @@ impl StreamTopN { }; let watermark_columns = FixedBitSet::with_capacity(input.schema().len()); - let base = PlanBase::new_stream_with_core(&logical, dist, false, false, watermark_columns); + let base = + PlanBase::new_stream_with_logical(&logical, dist, false, false, watermark_columns); StreamTopN { base, logical } } diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs index f5ce62f334519..3f5cab7ff67e8 100644 --- a/src/frontend/src/optimizer/plan_node/stream_union.rs +++ b/src/frontend/src/optimizer/plan_node/stream_union.rs @@ -48,7 +48,7 @@ impl StreamUnion { |acc_watermark_columns, input| acc_watermark_columns.bitand(input.watermark_columns()), ); - let base = PlanBase::new_stream_with_core( + let base = PlanBase::new_stream_with_logical( &logical, dist, inputs.iter().all(|x| x.append_only()), From 53b361ccdfb189898f044cbb38e902e1fb3c6b32 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 24 Oct 2023 13:15:15 +0800 Subject: [PATCH 09/11] rename again Signed-off-by: Bugen Zhao --- src/frontend/src/optimizer/plan_node/batch_delete.rs | 2 +- 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 | 2 +- src/frontend/src/optimizer/plan_node/batch_limit.rs | 2 +- .../src/optimizer/plan_node/batch_lookup_join.rs | 2 +- .../src/optimizer/plan_node/batch_nested_loop_join.rs | 2 +- .../src/optimizer/plan_node/batch_over_window.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_project.rs | 2 +- .../src/optimizer/plan_node/batch_project_set.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_seq_scan.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_simple_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_sort_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_source.rs | 2 +- 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/logical_source.rs | 2 +- src/frontend/src/optimizer/plan_node/plan_base.rs | 4 ++-- src/frontend/src/optimizer/plan_node/stream_dedup.rs | 2 +- .../src/optimizer/plan_node/stream_delta_join.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_expand.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_filter.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_hash_agg.rs | 9 ++------- src/frontend/src/optimizer/plan_node/stream_hash_join.rs | 2 +- .../src/optimizer/plan_node/stream_hop_window.rs | 2 +- .../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_share.rs | 2 +- .../src/optimizer/plan_node/stream_simple_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_source.rs | 2 +- .../optimizer/plan_node/stream_stateless_simple_agg.rs | 2 +- .../src/optimizer/plan_node/stream_table_scan.rs | 2 +- .../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 +- 43 files changed, 45 insertions(+), 50 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/batch_delete.rs b/src/frontend/src/optimizer/plan_node/batch_delete.rs index 42db0a1c4a774..18a10833e3980 100644 --- a/src/frontend/src/optimizer/plan_node/batch_delete.rs +++ b/src/frontend/src/optimizer/plan_node/batch_delete.rs @@ -33,7 +33,7 @@ pub struct BatchDelete { impl BatchDelete { pub fn new(core: generic::Delete) -> Self { assert_eq!(core.input.distribution(), &Distribution::Single); - let base: PlanBase = PlanBase::new_batch_from_logical( + let base: PlanBase = PlanBase::new_batch_with_core( &core, core.input.distribution().clone(), Order::any(), diff --git a/src/frontend/src/optimizer/plan_node/batch_expand.rs b/src/frontend/src/optimizer/plan_node/batch_expand.rs index 870368701be44..af4413b9e5152 100644 --- a/src/frontend/src/optimizer/plan_node/batch_expand.rs +++ b/src/frontend/src/optimizer/plan_node/batch_expand.rs @@ -41,7 +41,7 @@ impl BatchExpand { | Distribution::UpstreamHashShard(_, _) => Distribution::SomeShard, Distribution::Broadcast => unreachable!(), }; - let base = PlanBase::new_batch_from_logical(&core, dist, Order::any()); + let base = PlanBase::new_batch_with_core(&core, dist, Order::any()); BatchExpand { base, core } } diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index 6bc5086c7a29b..4bff7cbfee3c0 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -32,7 +32,7 @@ pub struct BatchFilter { impl BatchFilter { pub fn new(core: generic::Filter) -> Self { // TODO: derive from input - let base = PlanBase::new_batch_from_logical( + let base = PlanBase::new_batch_with_core( &core, core.input.distribution().clone(), core.input.order().clone(), 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 8f6684dc4d85b..d8188d2c3aeaa 100644 --- a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs @@ -33,7 +33,7 @@ pub struct BatchGroupTopN { impl BatchGroupTopN { pub fn new(core: generic::TopN) -> Self { assert!(!core.group_key.is_empty()); - let base = PlanBase::new_batch_from_logical( + let base = PlanBase::new_batch_with_core( &core, core.input.distribution().clone(), Order::any(), 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 fa14076912689..b4ab3341ace29 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -42,7 +42,7 @@ impl BatchHashAgg { let dist = core .i2o_col_mapping() .rewrite_provided_distribution(input_dist); - let base = PlanBase::new_batch_from_logical(&core, dist, Order::any()); + let base = PlanBase::new_batch_with_core(&core, dist, Order::any()); BatchHashAgg { base, core } } 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 3430eab81e7dc..bad586d4af1e4 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -46,7 +46,7 @@ pub struct BatchHashJoin { impl BatchHashJoin { pub fn new(core: generic::Join, eq_join_predicate: EqJoinPredicate) -> Self { let dist = Self::derive_dist(core.left.distribution(), core.right.distribution(), &core); - let base = PlanBase::new_batch_from_logical(&core, dist, Order::any()); + let base = PlanBase::new_batch_with_core(&core, dist, Order::any()); Self { base, 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 68381956b8a9a..404ea4a0ea920 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs @@ -44,7 +44,7 @@ impl BatchHopWindow { let distribution = core .i2o_col_mapping() .rewrite_provided_distribution(core.input.distribution()); - let base = PlanBase::new_batch_from_logical( + let base = PlanBase::new_batch_with_core( &core, distribution, core.get_out_column_index_order(), diff --git a/src/frontend/src/optimizer/plan_node/batch_insert.rs b/src/frontend/src/optimizer/plan_node/batch_insert.rs index 2c6f9915fd310..180ace63b1a86 100644 --- a/src/frontend/src/optimizer/plan_node/batch_insert.rs +++ b/src/frontend/src/optimizer/plan_node/batch_insert.rs @@ -35,7 +35,7 @@ pub struct BatchInsert { impl BatchInsert { pub fn new(core: generic::Insert) -> Self { assert_eq!(core.input.distribution(), &Distribution::Single); - let base: PlanBase = PlanBase::new_batch_from_logical( + let base: PlanBase = PlanBase::new_batch_with_core( &core, core.input.distribution().clone(), Order::any(), diff --git a/src/frontend/src/optimizer/plan_node/batch_limit.rs b/src/frontend/src/optimizer/plan_node/batch_limit.rs index 9d84f13aab88d..93b14d0198979 100644 --- a/src/frontend/src/optimizer/plan_node/batch_limit.rs +++ b/src/frontend/src/optimizer/plan_node/batch_limit.rs @@ -33,7 +33,7 @@ pub struct BatchLimit { impl BatchLimit { pub fn new(core: generic::Limit) -> Self { - let base = PlanBase::new_batch_from_logical( + let base = PlanBase::new_batch_with_core( &core, core.input.distribution().clone(), core.input.order().clone(), 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 1ea00d3f49d0a..48f99668c3af7 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -68,7 +68,7 @@ impl BatchLookupJoin { assert!(eq_join_predicate.has_eq()); assert!(eq_join_predicate.eq_keys_are_type_aligned()); let dist = Self::derive_dist(core.left.distribution(), &core); - let base = PlanBase::new_batch_from_logical(&core, dist, Order::any()); + let base = PlanBase::new_batch_with_core(&core, dist, Order::any()); Self { base, core, 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 d7e2c13acb07d..8980ad2f23f6d 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 @@ -37,7 +37,7 @@ pub struct BatchNestedLoopJoin { impl BatchNestedLoopJoin { pub fn new(core: generic::Join) -> Self { let dist = Self::derive_dist(core.left.distribution(), core.right.distribution()); - let base = PlanBase::new_batch_from_logical(&core, dist, Order::any()); + let base = PlanBase::new_batch_with_core(&core, dist, Order::any()); Self { base, core } } 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 1d53d695f2cf2..fb455758f331a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_over_window.rs @@ -47,7 +47,7 @@ impl BatchOverWindow { .collect(), ); - let base = PlanBase::new_batch_from_logical(&core, input_dist, order); + let base = PlanBase::new_batch_with_core(&core, input_dist, order); BatchOverWindow { base, core } } diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index b7934ee56e54d..642683967c5c3 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -44,7 +44,7 @@ impl BatchProject { .i2o_col_mapping() .rewrite_provided_order(core.input.order()); - let base = PlanBase::new_batch_from_logical(&core, distribution, order); + let base = PlanBase::new_batch_with_core(&core, distribution, order); BatchProject { base, core } } 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 331ca8e5235de..1179e3bef44f3 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project_set.rs @@ -38,7 +38,7 @@ impl BatchProjectSet { .i2o_col_mapping() .rewrite_provided_distribution(core.input.distribution()); - let base = PlanBase::new_batch_from_logical( + let base = PlanBase::new_batch_with_core( &core, distribution, core.get_out_column_index_order(), 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 5b34dbb15c032..6834ed29353b9 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -48,7 +48,7 @@ impl BatchSeqScan { } else { core.get_out_column_index_order() }; - let base = PlanBase::new_batch_from_logical(&core, dist, order); + let base = PlanBase::new_batch_with_core(&core, dist, order); { // validate scan_range 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 5a72b95f492af..bae8d70c2eedf 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -32,7 +32,7 @@ pub struct BatchSimpleAgg { impl BatchSimpleAgg { pub fn new(core: generic::Agg) -> Self { let input_dist = core.input.distribution().clone(); - let base = PlanBase::new_batch_from_logical(&core, input_dist, Order::any()); + let base = PlanBase::new_batch_with_core(&core, input_dist, Order::any()); BatchSimpleAgg { base, core } } 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 00facef473a37..2252d4c0c0ee0 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -54,7 +54,7 @@ impl BatchSortAgg { let order = core.i2o_col_mapping().rewrite_provided_order(&input_order); - let base = PlanBase::new_batch_from_logical(&core, dist, order); + let base = PlanBase::new_batch_with_core(&core, dist, order); BatchSortAgg { base, diff --git a/src/frontend/src/optimizer/plan_node/batch_source.rs b/src/frontend/src/optimizer/plan_node/batch_source.rs index 45ae27ad02904..9e2cd6006db0b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_source.rs +++ b/src/frontend/src/optimizer/plan_node/batch_source.rs @@ -36,7 +36,7 @@ pub struct BatchSource { impl BatchSource { pub fn new(core: generic::Source) -> Self { - let base = PlanBase::new_batch_from_logical( + let base = PlanBase::new_batch_with_core( &core, // Use `Single` by default, will be updated later with `clone_with_dist`. Distribution::Single, diff --git a/src/frontend/src/optimizer/plan_node/batch_topn.rs b/src/frontend/src/optimizer/plan_node/batch_topn.rs index e5f44bd2ef0e2..b2eda24046d28 100644 --- a/src/frontend/src/optimizer/plan_node/batch_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_topn.rs @@ -35,7 +35,7 @@ pub struct BatchTopN { impl BatchTopN { pub fn new(core: generic::TopN) -> Self { assert!(core.group_key.is_empty()); - let base = PlanBase::new_batch_from_logical( + let base = PlanBase::new_batch_with_core( &core, core.input.distribution().clone(), // BatchTopN outputs data in the order of specified order diff --git a/src/frontend/src/optimizer/plan_node/batch_union.rs b/src/frontend/src/optimizer/plan_node/batch_union.rs index 31b4a541dfe4a..c7c71111174c6 100644 --- a/src/frontend/src/optimizer/plan_node/batch_union.rs +++ b/src/frontend/src/optimizer/plan_node/batch_union.rs @@ -40,7 +40,7 @@ impl BatchUnion { Distribution::SomeShard }; - let base = PlanBase::new_batch_from_logical(&core, dist, Order::any()); + let base = PlanBase::new_batch_with_core(&core, dist, Order::any()); BatchUnion { base, core } } } diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 70ad051e4dbf3..45a5fbcb2240f 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -205,7 +205,7 @@ impl LogicalSource { ..self.core.clone() }; let mut new_s3_plan: PlanRef = StreamSource { - base: PlanBase::new_stream_with_logical( + base: PlanBase::new_stream_with_core( &logical_source, Distribution::Single, true, // `list` will keep listing all objects, it must be 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 e76bbee27dd45..51b1aa5f41141 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -239,7 +239,7 @@ impl PlanBase { } } - pub fn new_stream_with_logical( + pub fn new_stream_with_core( core: &impl GenericPlanNode, dist: Distribution, append_only: bool, @@ -281,7 +281,7 @@ impl PlanBase { } } - pub fn new_batch_from_logical( + pub fn new_batch_with_core( core: &impl GenericPlanNode, dist: Distribution, order: Order, diff --git a/src/frontend/src/optimizer/plan_node/stream_dedup.rs b/src/frontend/src/optimizer/plan_node/stream_dedup.rs index ecbc109d9f513..51b5e589e886e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dedup.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dedup.rs @@ -37,7 +37,7 @@ impl StreamDedup { // A dedup operator must be append-only. assert!(input.append_only()); - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, input.distribution().clone(), true, 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 2a7600814e632..bb18f9cffdf0f 100644 --- a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs @@ -67,7 +67,7 @@ impl StreamDeltaJoin { core.i2o_col_mapping().rewrite_bitset(&watermark_columns) }; // TODO: derive from input - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, dist, append_only, 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 6f1a617ee1236..a4b74f37208e7 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs @@ -38,7 +38,7 @@ impl StreamDynamicFilter { let watermark_columns = core.watermark_columns(core.right().watermark_columns()[0]); // TODO: derive from input - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, core.left().distribution().clone(), false, /* we can have a new abstraction for append only and monotonically increasing 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 00bd8dcba9590..d8c5a9635ce59 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 @@ -50,7 +50,7 @@ impl StreamEowcOverWindow { // ancient rows in some rarely updated partitions that are emitted at the end of time. let watermark_columns = FixedBitSet::with_capacity(core.output_len()); - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, input.distribution().clone(), true, diff --git a/src/frontend/src/optimizer/plan_node/stream_expand.rs b/src/frontend/src/optimizer/plan_node/stream_expand.rs index e0f8852a19fb5..5959b8d6be4d2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_expand.rs +++ b/src/frontend/src/optimizer/plan_node/stream_expand.rs @@ -48,7 +48,7 @@ impl StreamExpand { .map(|idx| idx + input.schema().len()), ); - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, dist, input.append_only(), diff --git a/src/frontend/src/optimizer/plan_node/stream_filter.rs b/src/frontend/src/optimizer/plan_node/stream_filter.rs index ff4d344607776..0f000e6b8c0db 100644 --- a/src/frontend/src/optimizer/plan_node/stream_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_filter.rs @@ -34,7 +34,7 @@ impl StreamFilter { let input = core.input.clone(); let dist = input.distribution().clone(); // Filter executor won't change the append-only behavior of the stream. - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, dist, input.append_only(), 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 190c05c0a5ba1..95fd72e9f6aa0 100644 --- a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs +++ b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs @@ -48,7 +48,7 @@ impl_plan_tree_node_for_unary! { StreamFsFetch } impl StreamFsFetch { pub fn new(input: PlanRef, source: generic::Source) -> Self { - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &source, Distribution::SomeShard, source.catalog.as_ref().map_or(true, |s| s.append_only), 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 efb5c61374002..55ab6b5906e59 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -86,7 +86,7 @@ impl StreamHashAgg { } // Hash agg executor might change the append-only behavior of the stream. - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, dist, emit_on_window_close, // in EOWC mode, we produce append only output @@ -142,13 +142,8 @@ impl StreamHashAgg { impl Distill for StreamHashAgg { fn distill<'a>(&self) -> XmlNode<'a> { -<<<<<<< HEAD - let mut vec = self.logical.fields_pretty(); - if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) { -======= let mut vec = self.core.fields_pretty(); - if let Some(ow) = watermark_pretty(&self.base.watermark_columns, self.schema()) { ->>>>>>> origin/main + if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) { vec.push(("output_watermarks", ow)); } childless_record( 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 123607e77a651..9d9c41425c4b1 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -179,7 +179,7 @@ impl StreamHashJoin { }; // TODO: derive from input - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, dist, append_only, 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 96955b1e1912e..e177be6942360 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -58,7 +58,7 @@ impl StreamHopWindow { ) .rewrite_bitset(&watermark_columns); - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, dist, input.append_only(), 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 9beee2e46ee9d..5a2f9d98f1340 100644 --- a/src/frontend/src/optimizer/plan_node/stream_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_over_window.rs @@ -38,7 +38,7 @@ impl StreamOverWindow { let input = &core.input; let watermark_columns = FixedBitSet::with_capacity(core.output_len()); - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, input.distribution().clone(), false, // general over window cannot be append-only diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index 45fbc6b9c13bb..c0ff0d1cf2f43 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -81,7 +81,7 @@ impl StreamProject { } // Project executor won't change the append-only behavior of the stream, so it depends on // input's `append_only`. - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, distribution, input.append_only(), 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 cadd600f3c3b7..ba09d79c96c60 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project_set.rs @@ -66,7 +66,7 @@ impl StreamProjectSet { // ProjectSet executor won't change the append-only behavior of the stream, so it depends on // input's `append_only`. - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, distribution, input.append_only(), diff --git a/src/frontend/src/optimizer/plan_node/stream_share.rs b/src/frontend/src/optimizer/plan_node/stream_share.rs index 86631e42d2410..3acf0b132805e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_share.rs +++ b/src/frontend/src/optimizer/plan_node/stream_share.rs @@ -36,7 +36,7 @@ impl StreamShare { 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_logical( + let base = PlanBase::new_stream_with_core( &core, dist, input.append_only(), 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 612c302c65c6e..92d96fdf21b08 100644 --- a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs @@ -50,7 +50,7 @@ impl StreamSimpleAgg { let watermark_columns = FixedBitSet::with_capacity(core.output_len()); // Simple agg executor might change the append-only behavior of the stream. - let base = PlanBase::new_stream_with_logical(&core, dist, false, false, watermark_columns); + let base = PlanBase::new_stream_with_core(&core, dist, false, false, watermark_columns); StreamSimpleAgg { base, core, diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index 377e2704776bb..ae66cf568118b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -37,7 +37,7 @@ pub struct StreamSource { impl StreamSource { pub fn new(core: generic::Source) -> Self { - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, Distribution::SomeShard, core.catalog.as_ref().map_or(true, |s| s.append_only), 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 8d91ca59680a7..474582ec877c7 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 @@ -50,7 +50,7 @@ impl StreamStatelessSimpleAgg { } } - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, input_dist.clone(), input.append_only(), 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 b64acb08a5a1a..965ca217a3369 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -68,7 +68,7 @@ impl StreamTableScan { None => Distribution::SomeShard, } }; - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, distribution, core.table_desc.append_only, 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 87ef3ec54b357..675dbeb9ab381 100644 --- a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs @@ -63,7 +63,7 @@ impl StreamTemporalJoin { .rewrite_bitset(core.left.watermark_columns()), ); - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, dist, true, diff --git a/src/frontend/src/optimizer/plan_node/stream_topn.rs b/src/frontend/src/optimizer/plan_node/stream_topn.rs index e7a880fa7d757..87890625f6be7 100644 --- a/src/frontend/src/optimizer/plan_node/stream_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_topn.rs @@ -40,7 +40,7 @@ impl StreamTopN { }; let watermark_columns = FixedBitSet::with_capacity(input.schema().len()); - let base = PlanBase::new_stream_with_logical(&core, dist, false, false, watermark_columns); + let base = PlanBase::new_stream_with_core(&core, dist, false, false, watermark_columns); StreamTopN { base, core } } diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs index 377e288547ab8..6d6dca2d8dd02 100644 --- a/src/frontend/src/optimizer/plan_node/stream_union.rs +++ b/src/frontend/src/optimizer/plan_node/stream_union.rs @@ -48,7 +48,7 @@ impl StreamUnion { |acc_watermark_columns, input| acc_watermark_columns.bitand(input.watermark_columns()), ); - let base = PlanBase::new_stream_with_logical( + let base = PlanBase::new_stream_with_core( &core, dist, inputs.iter().all(|x| x.append_only()), From f038b1d43e881bcdd4ca6f244c8270eaa2396867 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 24 Oct 2023 13:15:56 +0800 Subject: [PATCH 10/11] fix format Signed-off-by: Bugen Zhao --- README.md | 6 +++--- src/frontend/src/optimizer/plan_node/batch_delete.rs | 7 ++----- src/frontend/src/optimizer/plan_node/batch_group_topn.rs | 7 ++----- src/frontend/src/optimizer/plan_node/batch_hop_window.rs | 7 ++----- src/frontend/src/optimizer/plan_node/batch_insert.rs | 7 ++----- src/frontend/src/optimizer/plan_node/batch_project_set.rs | 7 ++----- 6 files changed, 13 insertions(+), 28 deletions(-) diff --git a/README.md b/README.md index 319c7f7025ea3..05f8ed8e3fb75 100644 --- a/README.md +++ b/README.md @@ -8,7 +8,7 @@

- + ### 🌊Stream Processing Redefined.
@@ -67,8 +67,8 @@ tar xvf risingwave-v1.3.0-x86_64-unknown-linux.tar.gz ``` **Mac** ``` -brew tap risingwavelabs/risingwave -brew install risingwave +brew tap risingwavelabs/risingwave +brew install risingwave risingwave playground ``` Now connect to RisingWave using `psql`: diff --git a/src/frontend/src/optimizer/plan_node/batch_delete.rs b/src/frontend/src/optimizer/plan_node/batch_delete.rs index 18a10833e3980..85d22a46b450e 100644 --- a/src/frontend/src/optimizer/plan_node/batch_delete.rs +++ b/src/frontend/src/optimizer/plan_node/batch_delete.rs @@ -33,11 +33,8 @@ pub struct BatchDelete { impl BatchDelete { pub fn new(core: generic::Delete) -> Self { assert_eq!(core.input.distribution(), &Distribution::Single); - let base: PlanBase = PlanBase::new_batch_with_core( - &core, - core.input.distribution().clone(), - Order::any(), - ); + 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_group_topn.rs b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs index d8188d2c3aeaa..70ee8328623f5 100644 --- a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs @@ -33,11 +33,8 @@ pub struct BatchGroupTopN { impl BatchGroupTopN { pub fn new(core: generic::TopN) -> Self { assert!(!core.group_key.is_empty()); - let base = PlanBase::new_batch_with_core( - &core, - core.input.distribution().clone(), - Order::any(), - ); + let base = + PlanBase::new_batch_with_core(&core, core.input.distribution().clone(), Order::any()); BatchGroupTopN { base, core } } 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 404ea4a0ea920..2a4a27f9a0583 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs @@ -44,11 +44,8 @@ impl BatchHopWindow { let distribution = core .i2o_col_mapping() .rewrite_provided_distribution(core.input.distribution()); - let base = PlanBase::new_batch_with_core( - &core, - distribution, - core.get_out_column_index_order(), - ); + let base = + PlanBase::new_batch_with_core(&core, distribution, core.get_out_column_index_order()); BatchHopWindow { base, core, diff --git a/src/frontend/src/optimizer/plan_node/batch_insert.rs b/src/frontend/src/optimizer/plan_node/batch_insert.rs index 180ace63b1a86..aec05eee145b8 100644 --- a/src/frontend/src/optimizer/plan_node/batch_insert.rs +++ b/src/frontend/src/optimizer/plan_node/batch_insert.rs @@ -35,11 +35,8 @@ pub struct BatchInsert { impl BatchInsert { pub fn new(core: generic::Insert) -> Self { assert_eq!(core.input.distribution(), &Distribution::Single); - let base: PlanBase = PlanBase::new_batch_with_core( - &core, - core.input.distribution().clone(), - Order::any(), - ); + 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_project_set.rs b/src/frontend/src/optimizer/plan_node/batch_project_set.rs index 1179e3bef44f3..5888df9d15889 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project_set.rs @@ -38,11 +38,8 @@ impl BatchProjectSet { .i2o_col_mapping() .rewrite_provided_distribution(core.input.distribution()); - let base = PlanBase::new_batch_with_core( - &core, - distribution, - core.get_out_column_index_order(), - ); + let base = + PlanBase::new_batch_with_core(&core, distribution, core.get_out_column_index_order()); BatchProjectSet { base, core } } } From d5bf54858be084e6b32c409f85eb40e1015760f4 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 24 Oct 2023 14:03:48 +0800 Subject: [PATCH 11/11] remove unnecessary dependency Signed-off-by: Bugen Zhao --- Cargo.lock | 12 ------------ src/frontend/Cargo.toml | 1 - 2 files changed, 13 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 80b75cebd8896..b7e9b6c45ec06 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6629,17 +6629,6 @@ dependencies = [ "zstd-sys", ] -[[package]] -name = "readonly" -version = "0.2.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b8f439da1766942fe069954da6058b2e6c1760eb878bae76f5be9fc29f56f574" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.37", -] - [[package]] name = "redis" version = "0.23.3" @@ -7514,7 +7503,6 @@ dependencies = [ "pretty_assertions", "prometheus", "rand", - "readonly", "risingwave_batch", "risingwave_common", "risingwave_common_service", diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 8bb4ef8bcfcb6..37f9f6326faea 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -48,7 +48,6 @@ pretty-xmlish = "0.1.13" pretty_assertions = "1" prometheus = { version = "0.13", features = ["process"] } rand = "0.8" -readonly = "0.2" risingwave_batch = { workspace = true } risingwave_common = { workspace = true } risingwave_common_service = { workspace = true }