risingwave_stream/from_proto/
simple_agg.rs1use 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::StateTable;
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 =
44 build_agg_state_storages_from_proto(node.get_agg_call_states(), store.clone(), None)
45 .await;
46 let intermediate_state_table = StateTable::from_table_catalog(
48 node.get_intermediate_state_table().unwrap(),
49 store.clone(),
50 None,
51 )
52 .await;
53 let distinct_dedup_tables =
54 build_distinct_dedup_table_from_proto(node.get_distinct_dedup_tables(), store, None)
55 .await;
56 let must_output_per_barrier = node.get_must_output_per_barrier();
57
58 let exec = SimpleAggExecutor::new(AggExecutorArgs {
59 version: node.version(),
60
61 input,
62 actor_ctx: params.actor_context,
63 info: params.info.clone(),
64
65 extreme_cache_size: params.env.config().developer.unsafe_extreme_cache_size,
66
67 agg_calls,
68 row_count_index: node.get_row_count_index() as usize,
69 storages,
70 intermediate_state_table,
71 distinct_dedup_tables,
72 watermark_epoch: params.watermark_epoch,
73 extra: SimpleAggExecutorExtraArgs {
74 must_output_per_barrier,
75 },
76 })?;
77
78 Ok((params.info, exec).into())
79 }
80}