Skip to main content

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_stream_node_body!(EowcOverWindow(PbEowcOverWindowNode) => EowcOverWindowExecutorBuilder);
30
31impl ExecutorBuilder for EowcOverWindowExecutorBuilder {
32    type Node = PbEowcOverWindowNode;
33
34    async fn new_boxed_executor(
35        params: ExecutorParams,
36        node: &Self::Node,
37        store: impl StateStore,
38    ) -> StreamResult<Executor> {
39        let [input]: [_; 1] = params.input.try_into().unwrap();
40        let calls: Vec<_> = node
41            .get_calls()
42            .iter()
43            .map(WindowFuncCall::from_protobuf)
44            .try_collect()?;
45        let partition_key_indices = node
46            .get_partition_by()
47            .iter()
48            .map(|i| *i as usize)
49            .collect();
50        let order_key_index = node.get_order_by()[0].get_column_index() as usize;
51        let vnodes = Some(Arc::new(
52            params
53                .vnode_bitmap
54                .expect("vnodes not set for EOWC over window"),
55        ));
56        let state_table =
57            StateTableBuilder::new(node.get_state_table()?, store.clone(), vnodes.clone())
58                .with_op_consistency_level(StateTableOpConsistencyLevel::Inconsistent)
59                .enable_preload_all_rows_by_config(&params.config)
60                .build()
61                .await;
62
63        // Build optional intermediate state table for persisting window function states
64        let intermediate_state_table =
65            if let Some(table_pb) = node.intermediate_state_table.as_ref() {
66                Some(
67                    StateTableBuilder::new(table_pb, store, vnodes)
68                        // Use ConsistentOldValue because we update existing rows
69                        .with_op_consistency_level(StateTableOpConsistencyLevel::ConsistentOldValue)
70                        .enable_preload_all_rows_by_config(&params.config)
71                        .build()
72                        .await,
73                )
74            } else {
75                None
76            };
77
78        let exec = EowcOverWindowExecutor::new(EowcOverWindowExecutorArgs {
79            actor_ctx: params.actor_context,
80
81            input,
82
83            schema: params.info.schema.clone(),
84            calls,
85            partition_key_indices,
86            order_key_index,
87            state_table,
88            watermark_epoch: params.watermark_epoch,
89            intermediate_state_table,
90        });
91        Ok((params.info, exec).into())
92    }
93}