risingwave_frontend/optimizer/plan_node/
batch_max_one_row.rs1use pretty_xmlish::XmlNode;
16use risingwave_pb::batch_plan::MaxOneRowNode;
17use risingwave_pb::batch_plan::plan_node::NodeBody;
18
19use super::batch::prelude::*;
20use super::generic::DistillUnit;
21use super::utils::Distill;
22use super::{
23 ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, generic,
24};
25use crate::error::Result;
26use crate::optimizer::plan_node::ToLocalBatch;
27use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
28use crate::optimizer::property::{Order, RequiredDist};
29
30#[derive(Debug, Clone, PartialEq, Eq, Hash)]
33pub struct BatchMaxOneRow {
34 pub base: PlanBase<Batch>,
35 core: generic::MaxOneRow<PlanRef>,
36}
37
38impl BatchMaxOneRow {
39 pub fn new(core: generic::MaxOneRow<PlanRef>) -> Self {
40 let base = PlanBase::new_batch_with_core(
41 &core,
42 core.input.distribution().clone(),
43 core.input.order().clone(),
44 );
45 BatchMaxOneRow { base, core }
46 }
47}
48
49impl PlanTreeNodeUnary for BatchMaxOneRow {
50 fn input(&self) -> PlanRef {
51 self.core.input.clone()
52 }
53
54 fn clone_with_input(&self, input: PlanRef) -> Self {
55 let core = generic::MaxOneRow { input };
56
57 Self::new(core)
58 }
59}
60impl_plan_tree_node_for_unary! {BatchMaxOneRow}
61
62impl Distill for BatchMaxOneRow {
63 fn distill<'a>(&self) -> XmlNode<'a> {
64 self.core.distill_with_name("BatchMaxOneRow")
65 }
66}
67
68impl ToDistributedBatch for BatchMaxOneRow {
69 fn to_distributed(&self) -> Result<PlanRef> {
70 let new_input = RequiredDist::single()
71 .enforce_if_not_satisfies(self.input().to_distributed()?, &Order::any())?;
72 Ok(self.clone_with_input(new_input).into())
73 }
74}
75
76impl ToBatchPb for BatchMaxOneRow {
77 fn to_batch_prost_body(&self) -> NodeBody {
78 NodeBody::MaxOneRow(MaxOneRowNode {})
79 }
80}
81
82impl ToLocalBatch for BatchMaxOneRow {
83 fn to_local(&self) -> Result<PlanRef> {
84 let new_input = RequiredDist::single()
85 .enforce_if_not_satisfies(self.input().to_local()?, &Order::any())?;
86 Ok(self.clone_with_input(new_input).into())
87 }
88}
89
90impl ExprRewritable for BatchMaxOneRow {}
91
92impl ExprVisitable for BatchMaxOneRow {}