Skip to main content

risingwave_stream/from_proto/
project_set.rs

1// Copyright 2022 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 multimap::MultiMap;
16use risingwave_common::util::iter_util::ZipEqFast;
17use risingwave_pb::stream_plan::ProjectSetNode;
18
19use super::*;
20use crate::executor::project::{ProjectSetExecutor, ProjectSetSelectItem};
21
22pub struct ProjectSetExecutorBuilder;
23
24impl_stream_node_body!(ProjectSet(ProjectSetNode) => ProjectSetExecutorBuilder);
25
26impl ExecutorBuilder for ProjectSetExecutorBuilder {
27    type Node = ProjectSetNode;
28
29    async fn new_boxed_executor(
30        params: ExecutorParams,
31        node: &Self::Node,
32        _store: impl StateStore,
33    ) -> StreamResult<Executor> {
34        let chunk_size = params.config.developer.chunk_size;
35
36        let [input]: [_; 1] = params.input.try_into().unwrap();
37        let select_list: Vec<_> = node
38            .get_select_list()
39            .iter()
40            .map(|proto| {
41                ProjectSetSelectItem::from_prost(
42                    proto,
43                    params.eval_error_report.clone(),
44                    chunk_size,
45                )
46            })
47            .try_collect()?;
48        let watermark_derivations = MultiMap::from_iter(
49            node.get_watermark_input_cols()
50                .iter()
51                .map(|idx| *idx as usize)
52                .zip_eq_fast(
53                    node.get_watermark_expr_indices()
54                        .iter()
55                        .map(|idx| *idx as usize),
56                ),
57        );
58        let nondecreasing_expr_indices = node
59            .get_nondecreasing_exprs()
60            .iter()
61            .map(|idx| *idx as usize)
62            .collect();
63
64        let exec = ProjectSetExecutor::new(
65            params.actor_context,
66            input,
67            select_list,
68            chunk_size,
69            watermark_derivations,
70            nondecreasing_expr_indices,
71            params.eval_error_report,
72        );
73        Ok((params.info, exec).into())
74    }
75}