risingwave_frontend/optimizer/plan_node/
stream_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
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
// 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 std::collections::HashSet;

use fixedbitset::FixedBitSet;
use risingwave_common::util::sort_util::{ColumnOrder, OrderType};
use risingwave_pb::stream_plan::stream_node::PbNodeBody;

use super::generic::{GenericPlanNode, PlanWindowFunction};
use super::stream::prelude::*;
use super::utils::{impl_distill_by_unit, TableCatalogBuilder};
use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode};
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::property::MonotonicityMap;
use crate::stream_fragmenter::BuildFragmentGraphState;
use crate::TableCatalog;

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

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

        let input = &core.input;
        let watermark_columns = FixedBitSet::with_capacity(core.output_len());

        let base = PlanBase::new_stream_with_core(
            &core,
            input.distribution().clone(),
            false, // general over window cannot be append-only
            false,
            watermark_columns,
            MonotonicityMap::new(), // TODO: derive monotonicity
        );
        StreamOverWindow { base, core }
    }

    fn infer_state_table(&self) -> TableCatalog {
        let mut tbl_builder = TableCatalogBuilder::default();

        let out_schema = self.core.schema();
        for field in out_schema.fields() {
            tbl_builder.add_column(field);
        }

        let mut order_cols = HashSet::new();
        for idx in self.core.partition_key_indices() {
            if order_cols.insert(idx) {
                tbl_builder.add_order_column(idx, OrderType::ascending());
            }
        }
        let read_prefix_len_hint = tbl_builder.get_current_pk_len();
        for o in self.core.order_key() {
            if order_cols.insert(o.column_index) {
                tbl_builder.add_order_column(o.column_index, o.order_type);
            }
        }
        for &idx in self.core.input.expect_stream_key() {
            if order_cols.insert(idx) {
                tbl_builder.add_order_column(idx, OrderType::ascending());
            }
        }

        let in_dist_key = self.core.input.distribution().dist_column_indices();
        tbl_builder.build(in_dist_key.to_vec(), read_prefix_len_hint)
    }
}

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

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

impl StreamNode for StreamOverWindow {
    fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> PbNodeBody {
        use risingwave_pb::stream_plan::*;

        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();
        let state_table = self
            .infer_state_table()
            .with_id(state.gen_table_id_wrapped())
            .to_internal_table_prost();
        let cache_policy = self
            .base
            .ctx()
            .session_ctx()
            .config()
            .streaming_over_window_cache_policy();

        PbNodeBody::OverWindow(OverWindowNode {
            calls,
            partition_by,
            order_by,
            state_table: Some(state_table),
            cache_policy: cache_policy.to_protobuf() as _,
        })
    }
}

impl ExprRewritable for StreamOverWindow {}

impl ExprVisitable for StreamOverWindow {}