risingwave_frontend/optimizer/plan_node/
batch_over_window.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
// 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_common::util::sort_util::{ColumnOrder, OrderType};
use risingwave_pb::batch_plan::plan_node::NodeBody;
use risingwave_pb::batch_plan::SortOverWindowNode;

use super::batch::prelude::*;
use super::generic::PlanWindowFunction;
use super::utils::impl_distill_by_unit;
use super::{
    generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch,
    ToLocalBatch,
};
use crate::error::Result;
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::property::{Order, RequiredDist};

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

impl BatchOverWindow {
    pub fn new(core: generic::OverWindow<PlanRef>) -> Self {
        assert!(core.funcs_have_same_partition_and_order());

        let input = &core.input;
        let input_dist = input.distribution().clone();

        let order = Order::new(
            core.partition_key_indices()
                .into_iter()
                .map(|idx| ColumnOrder::new(idx, OrderType::default()))
                .chain(core.order_key().iter().cloned())
                .collect(),
        );

        let base = PlanBase::new_batch_with_core(&core, input_dist, order);
        BatchOverWindow { base, core }
    }

    fn expected_input_order(&self) -> Order {
        self.order().clone()
    }
}

impl_distill_by_unit!(BatchOverWindow, core, "BatchOverWindow");

impl PlanTreeNodeUnary for BatchOverWindow {
    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! { BatchOverWindow }

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

impl ToLocalBatch for BatchOverWindow {
    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.expected_input_order())?;
        Ok(self.clone_with_input(new_input).into())
    }
}

impl ToBatchPb for BatchOverWindow {
    fn to_batch_prost_body(&self) -> NodeBody {
        let calls = self
            .core
            .window_functions()
            .iter()
            .map(PlanWindowFunction::to_protobuf)
            .collect();
        let partition_by = self
            .core
            .partition_key_indices()
            .into_iter()
            .map(|idx| idx as _)
            .collect();
        let order_by = self
            .core
            .order_key()
            .iter()
            .map(ColumnOrder::to_protobuf)
            .collect();

        NodeBody::SortOverWindow(SortOverWindowNode {
            calls,
            partition_by,
            order_by,
        })
    }
}

impl ExprRewritable for BatchOverWindow {}

impl ExprVisitable for BatchOverWindow {}