risingwave_stream/from_proto/
eowc_gap_fill.rs1use std::collections::HashMap;
16use std::sync::Arc;
17
18use itertools::Itertools;
19use risingwave_common::gap_fill::FillStrategy;
20use risingwave_expr::expr::build_non_strict_from_prost;
21use risingwave_pb::stream_plan::EowcGapFillNode;
22use risingwave_storage::StateStore;
23
24use super::ExecutorBuilder;
25use crate::common::table::state_table::StateTableBuilder;
26use crate::error::StreamResult;
27use crate::executor::Executor;
28use crate::executor::eowc::{EowcGapFillExecutor, EowcGapFillExecutorArgs};
29use crate::task::ExecutorParams;
30
31pub struct EowcGapFillExecutorBuilder;
32
33impl_stream_node_body!(EowcGapFill(EowcGapFillNode) => EowcGapFillExecutorBuilder);
34
35impl ExecutorBuilder for EowcGapFillExecutorBuilder {
36 type Node = EowcGapFillNode;
37
38 async fn new_boxed_executor(
39 params: ExecutorParams,
40 node: &EowcGapFillNode,
41 store: impl StateStore,
42 ) -> StreamResult<Executor> {
43 let [input]: [_; 1] = params.input.try_into().unwrap();
44
45 let time_column_index = node.get_time_column_index() as usize;
46
47 let interval_expr_node = node.get_interval()?;
49 let gap_interval =
50 build_non_strict_from_prost(interval_expr_node, params.eval_error_report.clone())?;
51
52 let fill_columns: Vec<usize> = node
53 .get_fill_columns()
54 .iter()
55 .map(|&x| x as usize)
56 .collect();
57
58 let fill_strategies: Vec<FillStrategy> = node
59 .get_fill_strategies()
60 .iter()
61 .map(|s| match s.as_str() {
62 "locf" => Ok(FillStrategy::Locf),
63 "interpolate" => Ok(FillStrategy::Interpolate),
64 "null" => Ok(FillStrategy::Null),
65 _ => anyhow::bail!("unknown fill strategy: {}", s),
66 })
67 .collect::<anyhow::Result<_>>()?;
68
69 let fill_columns_with_strategies: HashMap<usize, FillStrategy> =
70 fill_columns.into_iter().zip_eq(fill_strategies).collect();
71
72 let vnodes = params.vnode_bitmap.map(Arc::new);
73
74 let buffer_table = StateTableBuilder::new(
75 node.get_buffer_table().as_ref().unwrap(),
76 store.clone(),
77 vnodes.clone(),
78 )
79 .forbid_preload_all_rows()
80 .build()
81 .await;
82
83 let prev_row_table =
84 StateTableBuilder::new(node.get_prev_row_table().as_ref().unwrap(), store, vnodes)
85 .forbid_preload_all_rows()
86 .build()
87 .await;
88
89 if !node.get_partition_by_indices().is_empty() {
92 return Err(anyhow::anyhow!("partitioned EOWC gap fill is not supported").into());
93 }
94
95 let exec = EowcGapFillExecutor::new(EowcGapFillExecutorArgs {
96 actor_ctx: params.actor_context,
97 input,
98 schema: params.info.schema.clone(),
99 buffer_table,
100 prev_row_table,
101 chunk_size: params.config.developer.chunk_size,
102 time_column_index,
103 fill_columns: fill_columns_with_strategies,
104 gap_interval,
105 high_gap_fill_amplification_threshold: params
106 .config
107 .developer
108 .high_gap_fill_amplification_threshold,
109 });
110
111 Ok((params.info, exec).into())
112 }
113}