risingwave_frontend/optimizer/plan_visitor/
mod.rs1use paste::paste;
16mod apply_visitor;
17pub use apply_visitor::*;
18mod plan_correlated_id_finder;
19pub use plan_correlated_id_finder::*;
20mod share_parent_counter;
21pub use share_parent_counter::*;
22
23#[cfg(debug_assertions)]
24mod input_ref_validator;
25#[cfg(debug_assertions)]
26pub use input_ref_validator::*;
27
28mod execution_mode_decider;
29pub use execution_mode_decider::*;
30mod temporal_join_validator;
31pub use temporal_join_validator::*;
32mod relation_collector_visitor;
33mod sys_table_visitor;
34pub use relation_collector_visitor::*;
35pub use sys_table_visitor::*;
36mod side_effect_visitor;
37pub use side_effect_visitor::*;
38mod cardinality_visitor;
39pub use cardinality_visitor::*;
40mod jsonb_stream_key_checker;
41pub use jsonb_stream_key_checker::*;
42mod distributed_dml_visitor;
43mod read_storage_table_visitor;
44mod rw_timestamp_validator;
45pub use distributed_dml_visitor::*;
46pub use read_storage_table_visitor::*;
47pub use rw_timestamp_validator::*;
48
49use crate::for_each_convention_all_plan_nodes;
50use crate::optimizer::plan_node::*;
51
52pub trait DefaultBehavior<R> {
54 fn apply(&self, results: impl IntoIterator<Item = R>) -> R;
56}
57
58pub struct Merge<F>(F);
62
63impl<F, R> DefaultBehavior<R> for Merge<F>
64where
65 F: Fn(R, R) -> R,
66 R: Default,
67{
68 fn apply(&self, results: impl IntoIterator<Item = R>) -> R {
69 results.into_iter().reduce(&self.0).unwrap_or_default()
70 }
71}
72
73pub struct DefaultValue;
75
76impl<R> DefaultBehavior<R> for DefaultValue
77where
78 R: Default,
79{
80 fn apply(&self, results: impl IntoIterator<Item = R>) -> R {
81 let _ = results.into_iter().count(); R::default()
83 }
84}
85
86pub trait PlanVisitor<C: ConventionMarker> {
87 type Result;
88 fn visit(&mut self, plan: PlanRef<C>) -> Self::Result;
89}
90
91macro_rules! def_visitor {
93 ({
94 $( $convention:ident, { $( $name:ident ),* }),*
95 }) => {
96 paste! {
97 $(
98 pub trait [<$convention PlanVisitor>] {
101 type Result: Default;
102 type DefaultBehavior: DefaultBehavior<Self::Result>;
103
104 fn default_behavior() -> Self::DefaultBehavior;
106
107 fn [<visit_ $convention:snake>](&mut self, plan: PlanRef<$convention>) -> Self::Result {
108 use risingwave_common::util::recursive::{tracker, Recurse};
109 use crate::session::current::notice_to_user;
110
111 tracker!().recurse(|t| {
112 if t.depth_reaches(PLAN_DEPTH_THRESHOLD) {
113 notice_to_user(PLAN_TOO_DEEP_NOTICE);
114 }
115
116 match plan.node_type() {
117 $(
118 [<$convention PlanNodeType>]::[<$convention $name>] => self.[<visit_ $convention:snake _ $name:snake>](plan.downcast_ref::<[<$convention $name>]>().unwrap()),
119 )*
120 }
121 })
122 }
123
124 $(
125 #[doc = "Visit [`" [<$convention $name>] "`] , the function should visit the inputs."]
126 fn [<visit_ $convention:snake _ $name:snake>](&mut self, plan: &[<$convention $name>]) -> Self::Result {
127 let results = plan.inputs().into_iter().map(|input| self.[<visit_ $convention:snake>](input));
128 Self::default_behavior().apply(results)
129 }
130 )*
131
132 }
133
134 impl<V: [<$convention PlanVisitor>]> PlanVisitor<$convention> for V {
135 type Result = V::Result;
136 fn visit(&mut self, plan: PlanRef<$convention>) -> Self::Result {
137 self.[<visit_ $convention:snake>](plan)
138 }
139 }
140 )*
141 }
142 }
143}
144
145for_each_convention_all_plan_nodes! { def_visitor }
146
147macro_rules! impl_has_variant {
148 ( $({$convention:ident $variant_name:ident}),* ) => {
149 paste! {
150 $(
151 pub fn [<has_ $convention:snake _ $variant_name:snake _where>]<P>(plan: PlanRef<$convention>, pred: P) -> bool
152 where
153 P: FnMut(&[<$convention $variant_name>]) -> bool,
154 {
155 struct HasWhere<P> {
156 pred: P,
157 }
158
159 impl<P> [<$convention PlanVisitor>] for HasWhere<P>
160 where
161 P: FnMut(&[<$convention $variant_name>]) -> bool,
162 {
163 type Result = bool;
164 type DefaultBehavior = impl DefaultBehavior<Self::Result>;
165
166 fn default_behavior() -> Self::DefaultBehavior {
167 Merge(|a, b| a | b)
168 }
169
170 fn [<visit_ $convention:snake _ $variant_name:snake>](&mut self, node: &[<$convention $variant_name>]) -> Self::Result {
171 (self.pred)(node)
172 }
173 }
174
175 let mut visitor = HasWhere { pred };
176 visitor.visit(plan)
177 }
178
179 #[allow(dead_code)]
180 pub fn [<has_ $convention:snake _ $variant_name:snake>](plan: PlanRef<$convention>) -> bool {
181 [<has_ $convention:snake _$variant_name:snake _where>](plan, |_| true)
182 }
183 )*
184 }
185 };
186}
187
188impl_has_variant! {
189 {Logical Apply},
190 {Logical MaxOneRow},
191 {Logical OverWindow},
192 {Logical Scan},
193 {Logical Source},
194 {Batch Exchange},
195 {Batch SeqScan},
196 {Batch Source},
197 {Batch Insert},
198 {Batch Delete},
199 {Batch Update}
200}