risingwave_frontend/optimizer/plan_node/
batch_union.rs1use risingwave_pb::batch_plan::UnionNode;
16use risingwave_pb::batch_plan::plan_node::NodeBody;
17
18use super::batch::prelude::*;
19use super::utils::impl_distill_by_unit;
20use super::{ExprRewritable, PlanRef, ToBatchPb, ToDistributedBatch, generic};
21use crate::error::Result;
22use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
23use crate::optimizer::plan_node::{PlanBase, PlanTreeNode, ToLocalBatch};
24use crate::optimizer::property::{Distribution, Order, RequiredDist};
25
26#[derive(Debug, Clone, PartialEq, Eq, Hash)]
28pub struct BatchUnion {
29 pub base: PlanBase<Batch>,
30 core: generic::Union<PlanRef>,
31}
32
33impl BatchUnion {
34 pub fn new(core: generic::Union<PlanRef>) -> Self {
35 let dist = if core
36 .inputs
37 .iter()
38 .all(|input| *input.distribution() == Distribution::Single)
39 {
40 Distribution::Single
41 } else {
42 Distribution::SomeShard
43 };
44
45 let base = PlanBase::new_batch_with_core(&core, dist, Order::any());
46 BatchUnion { base, core }
47 }
48}
49
50impl_distill_by_unit!(BatchUnion, core, "BatchUnion");
51
52impl PlanTreeNode for BatchUnion {
53 fn inputs(&self) -> smallvec::SmallVec<[crate::optimizer::PlanRef; 2]> {
54 smallvec::SmallVec::from_vec(self.core.inputs.clone())
55 }
56
57 fn clone_with_inputs(&self, inputs: &[crate::optimizer::PlanRef]) -> PlanRef {
58 let mut new = self.core.clone();
60 new.inputs = inputs.to_vec();
61 Self::new(new).into()
62 }
63}
64
65impl ToDistributedBatch for BatchUnion {
66 fn to_distributed(&self) -> Result<PlanRef> {
67 let new_inputs: Result<Vec<_>> = self
69 .inputs()
70 .iter()
71 .map(|input| {
72 RequiredDist::single()
73 .enforce_if_not_satisfies(input.to_distributed()?, &Order::any())
74 })
75 .collect();
76 Ok(self.clone_with_inputs(&new_inputs?))
77 }
78}
79
80impl ToBatchPb for BatchUnion {
81 fn to_batch_prost_body(&self) -> NodeBody {
82 NodeBody::Union(UnionNode {})
83 }
84}
85
86impl ToLocalBatch for BatchUnion {
87 fn to_local(&self) -> Result<PlanRef> {
88 let new_inputs: Result<Vec<_>> = self
89 .inputs()
90 .iter()
91 .map(|input| {
92 RequiredDist::single().enforce_if_not_satisfies(input.to_local()?, &Order::any())
93 })
94 .collect();
95 Ok(self.clone_with_inputs(&new_inputs?))
96 }
97}
98
99impl ExprRewritable for BatchUnion {}
100
101impl ExprVisitable for BatchUnion {}