risingwave_frontend/optimizer/plan_node/
stream_stateless_simple_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
// 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 fixedbitset::FixedBitSet;
use itertools::Itertools;
use risingwave_pb::stream_plan::stream_node::PbNodeBody;

use super::generic::{self, PlanAggCall};
use super::stream::prelude::*;
use super::utils::impl_distill_by_unit;
use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode};
use crate::expr::{ExprRewriter, ExprVisitor};
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::property::{MonotonicityMap, RequiredDist};
use crate::stream_fragmenter::BuildFragmentGraphState;

/// Streaming stateless simple agg.
///
/// Should only be used for stateless agg, including `sum`, `count` and *append-only* `min`/`max`.
///
/// The output of `StreamStatelessSimpleAgg` doesn't have pk columns, so the result can only be used
/// by `StreamSimpleAgg`.
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub struct StreamStatelessSimpleAgg {
    pub base: PlanBase<Stream>,
    core: generic::Agg<PlanRef>,
}

impl StreamStatelessSimpleAgg {
    pub fn new(core: generic::Agg<PlanRef>) -> Self {
        let input = core.input.clone();
        let input_dist = input.distribution();
        debug_assert!(input_dist.satisfies(&RequiredDist::AnyShard));

        let mut watermark_columns = FixedBitSet::with_capacity(core.output_len());
        // Watermark column(s) must be in group key.
        for (idx, input_idx) in core.group_key.indices().enumerate() {
            if input.watermark_columns().contains(input_idx) {
                watermark_columns.insert(idx);
            }
        }

        let base = PlanBase::new_stream_with_core(
            &core,
            input_dist.clone(),
            input.append_only(),
            input.emit_on_window_close(),
            watermark_columns,
            MonotonicityMap::new(),
        );
        StreamStatelessSimpleAgg { base, core }
    }

    pub fn agg_calls(&self) -> &[PlanAggCall] {
        &self.core.agg_calls
    }
}
impl_distill_by_unit!(StreamStatelessSimpleAgg, core, "StreamStatelessSimpleAgg");

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

impl StreamNode for StreamStatelessSimpleAgg {
    fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody {
        use risingwave_pb::stream_plan::*;
        PbNodeBody::StatelessSimpleAgg(SimpleAggNode {
            agg_calls: self
                .agg_calls()
                .iter()
                .map(PlanAggCall::to_protobuf)
                .collect(),
            row_count_index: u32::MAX, // this is not used
            distribution_key: self
                .distribution()
                .dist_column_indices()
                .iter()
                .map(|idx| *idx as u32)
                .collect_vec(),
            agg_call_states: vec![],
            intermediate_state_table: None,
            is_append_only: self.input().append_only(),
            distinct_dedup_tables: Default::default(),
            version: AggNodeVersion::Issue13465 as _,
            must_output_per_barrier: false, // this is not used
        })
    }
}

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

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

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