risingwave_stream/from_proto/
eowc_over_window.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
15use 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 =
55            StateTableBuilder::new(node.get_state_table()?, store.clone(), vnodes.clone())
56                .with_op_consistency_level(StateTableOpConsistencyLevel::Inconsistent)
57                .enable_preload_all_rows_by_config(&params.config)
58                .build()
59                .await;
60
61        // Build optional intermediate state table for persisting window function states
62        let intermediate_state_table =
63            if let Some(table_pb) = node.intermediate_state_table.as_ref() {
64                Some(
65                    StateTableBuilder::new(table_pb, store, vnodes)
66                        // Use ConsistentOldValue because we update existing rows
67                        .with_op_consistency_level(StateTableOpConsistencyLevel::ConsistentOldValue)
68                        .enable_preload_all_rows_by_config(&params.config)
69                        .build()
70                        .await,
71                )
72            } else {
73                None
74            };
75
76        let exec = EowcOverWindowExecutor::new(EowcOverWindowExecutorArgs {
77            actor_ctx: params.actor_context,
78
79            input,
80
81            schema: params.info.schema.clone(),
82            calls,
83            partition_key_indices,
84            order_key_index,
85            state_table,
86            watermark_epoch: params.watermark_epoch,
87            intermediate_state_table,
88        });
89        Ok((params.info, exec).into())
90    }
91}