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