risingwave_stream/from_proto/
eowc_over_window.rs1use std::sync::Arc;
16
17use risingwave_expr::window_function::WindowFuncCall;
18use risingwave_pb::stream_plan::PbEowcOverWindowNode;
19use risingwave_storage::StateStore;
20
21use super::ExecutorBuilder;
22use crate::common::table::state_table::{StateTableBuilder, StateTableOpConsistencyLevel};
23use crate::error::StreamResult;
24use crate::executor::{EowcOverWindowExecutor, EowcOverWindowExecutorArgs, Executor};
25use crate::task::ExecutorParams;
26
27pub struct EowcOverWindowExecutorBuilder;
28
29impl ExecutorBuilder for EowcOverWindowExecutorBuilder {
30 type Node = PbEowcOverWindowNode;
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 calls: Vec<_> = node
39 .get_calls()
40 .iter()
41 .map(WindowFuncCall::from_protobuf)
42 .try_collect()?;
43 let partition_key_indices = node
44 .get_partition_by()
45 .iter()
46 .map(|i| *i as usize)
47 .collect();
48 let order_key_index = node.get_order_by()[0].get_column_index() as usize;
49 let vnodes = Some(Arc::new(
50 params
51 .vnode_bitmap
52 .expect("vnodes not set for EOWC over window"),
53 ));
54 let state_table = StateTableBuilder::new(node.get_state_table()?, store, vnodes)
55 .with_op_consistency_level(StateTableOpConsistencyLevel::Inconsistent)
56 .enable_preload_all_rows_by_config(¶ms.actor_context.streaming_config)
57 .build()
58 .await;
59 let exec = EowcOverWindowExecutor::new(EowcOverWindowExecutorArgs {
60 actor_ctx: params.actor_context,
61
62 input,
63
64 schema: params.info.schema.clone(),
65 calls,
66 partition_key_indices,
67 order_key_index,
68 state_table,
69 watermark_epoch: params.watermark_epoch,
70 });
71 Ok((params.info, exec).into())
72 }
73}