risingwave_stream/from_proto/
simple_agg.rs

1// Copyright 2025 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 ExecutorBuilder for SimpleAggExecutorBuilder {
30    type Node = SimpleAggNode;
31
32    async fn new_boxed_executor(
33        params: ExecutorParams,
34        node: &Self::Node,
35        store: impl StateStore,
36    ) -> StreamResult<Executor> {
37        let [input]: [_; 1] = params.input.try_into().unwrap();
38        let agg_calls: Vec<AggCall> = node
39            .get_agg_calls()
40            .iter()
41            .map(AggCall::from_protobuf)
42            .try_collect()?;
43        let storages = build_agg_state_storages_from_proto(
44            node.get_agg_call_states(),
45            store.clone(),
46            None,
47            &params.actor_context.streaming_config,
48        )
49        .await;
50        // disable sanity check so that old value is not required when updating states
51        let intermediate_state_table = StateTableBuilder::new(
52            node.get_intermediate_state_table().unwrap(),
53            store.clone(),
54            None,
55        )
56        .enable_preload_all_rows_by_config(&params.actor_context.streaming_config)
57        .build()
58        .await;
59        let distinct_dedup_tables = build_distinct_dedup_table_from_proto(
60            node.get_distinct_dedup_tables(),
61            store,
62            None,
63            &params.actor_context.streaming_config,
64        )
65        .await;
66        let must_output_per_barrier = node.get_must_output_per_barrier();
67
68        let exec = SimpleAggExecutor::new(AggExecutorArgs {
69            version: node.version(),
70
71            input,
72            actor_ctx: params.actor_context,
73            info: params.info.clone(),
74
75            extreme_cache_size: params.env.config().developer.unsafe_extreme_cache_size,
76
77            agg_calls,
78            row_count_index: node.get_row_count_index() as usize,
79            storages,
80            intermediate_state_table,
81            distinct_dedup_tables,
82            watermark_epoch: params.watermark_epoch,
83            extra: SimpleAggExecutorExtraArgs {
84                must_output_per_barrier,
85            },
86        })?;
87
88        Ok((params.info, exec).into())
89    }
90}