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