risingwave_frontend/optimizer/rule/
union_merge_rule.rs1use super::prelude::{PlanRef, *};
16use crate::optimizer::plan_node::{LogicalUnion, PlanTreeNode};
17
18pub struct UnionMergeRule {}
19impl Rule<Logical> for UnionMergeRule {
20 fn apply(&self, plan: PlanRef) -> Option<PlanRef> {
21 let top_union: &LogicalUnion = plan.as_logical_union()?;
22 let top_all = top_union.all();
23 let mut new_inputs = vec![];
24 let mut has_merge = false;
25 for input in top_union.inputs() {
26 if let Some(bottom_union) = input.as_logical_union()
27 && bottom_union.all() == top_all
28 {
29 new_inputs.extend(bottom_union.inputs());
30 has_merge = true;
31 } else {
32 new_inputs.push(input);
33 }
34 }
35
36 if has_merge {
37 Some(top_union.clone_with_inputs(&new_inputs))
38 } else {
39 None
40 }
41 }
42}
43
44impl UnionMergeRule {
45 pub fn create() -> BoxedRule {
46 Box::new(UnionMergeRule {})
47 }
48}