risingwave_frontend/optimizer/plan_node/
batch_expand.rs1use itertools::Itertools;
16use risingwave_pb::batch_plan::ExpandNode;
17use risingwave_pb::batch_plan::expand_node::Subset;
18use risingwave_pb::batch_plan::plan_node::NodeBody;
19
20use super::batch::prelude::*;
21use super::utils::impl_distill_by_unit;
22use super::{ExprRewritable, generic};
23use crate::error::Result;
24use crate::optimizer::PlanRef;
25use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
26use crate::optimizer::plan_node::{
27 PlanBase, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, ToLocalBatch,
28};
29use crate::optimizer::property::{Distribution, Order};
30
31#[derive(Debug, Clone, PartialEq, Eq, Hash)]
32pub struct BatchExpand {
33 pub base: PlanBase<Batch>,
34 core: generic::Expand<PlanRef>,
35}
36
37impl BatchExpand {
38 pub fn new(core: generic::Expand<PlanRef>) -> Self {
39 let dist = match core.input.distribution() {
40 Distribution::Single => Distribution::Single,
41 Distribution::SomeShard
42 | Distribution::HashShard(_)
43 | Distribution::UpstreamHashShard(_, _) => Distribution::SomeShard,
44 Distribution::Broadcast => unreachable!(),
45 };
46 let base = PlanBase::new_batch_with_core(&core, dist, Order::any());
47 BatchExpand { base, core }
48 }
49
50 pub fn column_subsets(&self) -> &[Vec<usize>] {
51 &self.core.column_subsets
52 }
53}
54
55impl_distill_by_unit!(BatchExpand, core, "BatchExpand");
56
57impl PlanTreeNodeUnary for BatchExpand {
58 fn input(&self) -> PlanRef {
59 self.core.input.clone()
60 }
61
62 fn clone_with_input(&self, input: PlanRef) -> Self {
63 let mut core = self.core.clone();
64 core.input = input;
65 Self::new(core)
66 }
67}
68
69impl_plan_tree_node_for_unary! { BatchExpand }
70
71impl ToDistributedBatch for BatchExpand {
72 fn to_distributed(&self) -> Result<PlanRef> {
73 let new_input = self.input().to_distributed()?;
74 Ok(self.clone_with_input(new_input).into())
75 }
76}
77
78impl ToBatchPb for BatchExpand {
79 fn to_batch_prost_body(&self) -> NodeBody {
80 NodeBody::Expand(ExpandNode {
81 column_subsets: self
82 .column_subsets()
83 .iter()
84 .map(|subset| subset_to_protobuf(subset))
85 .collect_vec(),
86 })
87 }
88}
89
90fn subset_to_protobuf(subset: &[usize]) -> Subset {
91 let column_indices = subset.iter().map(|key| *key as u32).collect_vec();
92 Subset { column_indices }
93}
94
95impl ToLocalBatch for BatchExpand {
96 fn to_local(&self) -> Result<PlanRef> {
97 let new_input = self.input().to_local()?;
98 Ok(self.clone_with_input(new_input).into())
99 }
100}
101
102impl ExprRewritable for BatchExpand {}
103
104impl ExprVisitable for BatchExpand {}