risingwave_frontend/optimizer/plan_node/
batch_sort_agg.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
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
// 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 risingwave_pb::batch_plan::plan_node::NodeBody;
use risingwave_pb::batch_plan::SortAggNode;
use risingwave_pb::expr::ExprNode;

use super::batch::prelude::*;
use super::generic::{self, PlanAggCall};
use super::utils::impl_distill_by_unit;
use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch};
use crate::error::Result;
use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprVisitor, InputRef};
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::plan_node::ToLocalBatch;
use crate::optimizer::property::{Order, RequiredDist};
use crate::utils::{ColIndexMappingRewriteExt, IndexSet};

#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub struct BatchSortAgg {
    pub base: PlanBase<Batch>,
    core: generic::Agg<PlanRef>,
    input_order: Order,
}

impl BatchSortAgg {
    pub fn new(core: generic::Agg<PlanRef>) -> Self {
        assert!(!core.group_key.is_empty());
        assert!(core.input_provides_order_on_group_keys());

        let input = core.input.clone();
        let input_dist = input.distribution();
        let dist = core
            .i2o_col_mapping()
            .rewrite_provided_distribution(input_dist);
        let input_order = Order {
            column_orders: input
                .order()
                .column_orders
                .iter()
                .filter(|o| core.group_key.indices().any(|g_k| g_k == o.column_index))
                .cloned()
                .collect(),
        };

        let order = core.i2o_col_mapping().rewrite_provided_order(&input_order);

        let base = PlanBase::new_batch_with_core(&core, dist, order);

        BatchSortAgg {
            base,
            core,
            input_order,
        }
    }

    pub fn agg_calls(&self) -> &[PlanAggCall] {
        &self.core.agg_calls
    }

    pub fn group_key(&self) -> &IndexSet {
        &self.core.group_key
    }
}

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

    fn clone_with_input(&self, input: PlanRef) -> Self {
        let mut core = self.core.clone();
        core.input = input;
        Self::new(core)
    }
}
impl_plan_tree_node_for_unary! { BatchSortAgg }
impl_distill_by_unit!(BatchSortAgg, core, "BatchSortAgg");

impl ToDistributedBatch for BatchSortAgg {
    fn to_distributed(&self) -> Result<PlanRef> {
        let new_input = self.input().to_distributed_with_required(
            &self.input_order,
            &RequiredDist::shard_by_key(self.input().schema().len(), &self.group_key().to_vec()),
        )?;
        Ok(self.clone_with_input(new_input).into())
    }
}

impl ToBatchPb for BatchSortAgg {
    fn to_batch_prost_body(&self) -> NodeBody {
        let input = self.input();
        NodeBody::SortAgg(SortAggNode {
            agg_calls: self
                .agg_calls()
                .iter()
                .map(PlanAggCall::to_protobuf)
                .collect(),
            group_key: self
                .group_key()
                .indices()
                .map(|idx| {
                    ExprImpl::InputRef(InputRef::new(idx, input.schema()[idx].data_type()).into())
                })
                .map(|expr| expr.to_expr_proto())
                .collect::<Vec<ExprNode>>(),
        })
    }
}

impl ToLocalBatch for BatchSortAgg {
    fn to_local(&self) -> Result<PlanRef> {
        let new_input = self.input().to_local()?;

        let new_input =
            RequiredDist::single().enforce_if_not_satisfies(new_input, self.input().order())?;

        Ok(self.clone_with_input(new_input).into())
    }
}

impl ExprRewritable for BatchSortAgg {
    fn has_rewritable_expr(&self) -> bool {
        true
    }

    fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef {
        let mut new_logical = self.core.clone();
        new_logical.rewrite_exprs(r);
        Self::new(new_logical).into()
    }
}

impl ExprVisitable for BatchSortAgg {
    fn visit_exprs(&self, v: &mut dyn ExprVisitor) {
        self.core.visit_exprs(v);
    }
}