risingwave_frontend/optimizer/plan_node/
batch_sort.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use pretty_xmlish::XmlNode;
use risingwave_pb::batch_plan::plan_node::NodeBody;
use risingwave_pb::batch_plan::SortNode;

use super::batch::prelude::*;
use super::utils::{childless_record, Distill};
use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch};
use crate::error::Result;
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::plan_node::ToLocalBatch;
use crate::optimizer::property::{Order, OrderDisplay};

/// `BatchSort` buffers all data from input and sort these rows by specified order, providing the
/// collation required by user or parent plan node.
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub struct BatchSort {
    pub base: PlanBase<Batch>,
    input: PlanRef,
}

impl BatchSort {
    pub fn new(input: PlanRef, order: Order) -> Self {
        let ctx = input.ctx();
        let schema = input.schema().clone();
        let dist = input.distribution().clone();
        let base = PlanBase::new_batch(ctx, schema, dist, order);
        BatchSort { base, input }
    }
}

impl Distill for BatchSort {
    fn distill<'a>(&self) -> XmlNode<'a> {
        let data = OrderDisplay {
            order: self.order(),
            input_schema: self.input.schema(),
        };
        childless_record("BatchSort", vec![("order", data.distill())])
    }
}

impl PlanTreeNodeUnary for BatchSort {
    fn input(&self) -> PlanRef {
        self.input.clone()
    }

    fn clone_with_input(&self, input: PlanRef) -> Self {
        Self::new(input, self.base.order().clone())
    }
}
impl_plan_tree_node_for_unary! {BatchSort}

impl ToDistributedBatch for BatchSort {
    fn to_distributed(&self) -> Result<PlanRef> {
        let new_input = self.input().to_distributed()?;
        Ok(self.clone_with_input(new_input).into())
    }
}

impl ToBatchPb for BatchSort {
    fn to_batch_prost_body(&self) -> NodeBody {
        let column_orders = self.base.order().to_protobuf();
        NodeBody::Sort(SortNode { column_orders })
    }
}

impl ToLocalBatch for BatchSort {
    fn to_local(&self) -> Result<PlanRef> {
        let new_input = self.input().to_local()?;
        Ok(self.clone_with_input(new_input).into())
    }
}

impl ExprRewritable for BatchSort {}

impl ExprVisitable for BatchSort {}