Skip to main content

risingwave_stream/from_proto/
simple_agg.rs

1// Copyright 2023 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15//! Streaming Simple Aggregator
16
17use risingwave_expr::aggregate::AggCall;
18use risingwave_pb::stream_plan::SimpleAggNode;
19
20use super::agg_common::{
21    build_agg_state_storages_from_proto, build_distinct_dedup_table_from_proto,
22};
23use super::*;
24use crate::common::table::state_table::StateTableBuilder;
25use crate::executor::aggregate::{AggExecutorArgs, SimpleAggExecutor, SimpleAggExecutorExtraArgs};
26
27pub struct SimpleAggExecutorBuilder;
28
29impl_stream_node_body!(SimpleAgg(SimpleAggNode) => SimpleAggExecutorBuilder);
30
31impl ExecutorBuilder for SimpleAggExecutorBuilder {
32    type Node = SimpleAggNode;
33
34    async fn new_boxed_executor(
35        params: ExecutorParams,
36        node: &Self::Node,
37        store: impl StateStore,
38    ) -> StreamResult<Executor> {
39        let [input]: [_; 1] = params.input.try_into().unwrap();
40        let agg_calls: Vec<AggCall> = node
41            .get_agg_calls()
42            .iter()
43            .map(AggCall::from_protobuf)
44            .try_collect()?;
45        let storages = build_agg_state_storages_from_proto(
46            node.get_agg_call_states(),
47            store.clone(),
48            None,
49            &params.config,
50        )
51        .await;
52        // disable sanity check so that old value is not required when updating states
53        let intermediate_state_table = StateTableBuilder::new(
54            node.get_intermediate_state_table().unwrap(),
55            store.clone(),
56            None,
57        )
58        .enable_preload_all_rows_by_config(&params.config)
59        .build()
60        .await;
61        let distinct_dedup_tables = build_distinct_dedup_table_from_proto(
62            node.get_distinct_dedup_tables(),
63            store,
64            None,
65            &params.config,
66        )
67        .await;
68        let must_output_per_barrier = node.get_must_output_per_barrier();
69
70        let exec = SimpleAggExecutor::new(AggExecutorArgs {
71            version: node.version(),
72
73            input,
74            actor_ctx: params.actor_context,
75            info: params.info.clone(),
76
77            extreme_cache_size: params.config.developer.unsafe_extreme_cache_size,
78
79            agg_calls,
80            row_count_index: node.get_row_count_index() as usize,
81            storages,
82            intermediate_state_table,
83            distinct_dedup_tables,
84            watermark_epoch: params.watermark_epoch,
85            extra: SimpleAggExecutorExtraArgs {
86                must_output_per_barrier,
87            },
88        })?;
89
90        Ok((params.info, exec).into())
91    }
92}