Skip to main content

risingwave_stream/from_proto/
top_n.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 std::sync::Arc;
16
17use risingwave_common::util::sort_util::ColumnOrder;
18use risingwave_pb::stream_plan::TopNNode;
19
20use super::*;
21use crate::common::table::state_table::StateTableBuilder;
22use crate::executor::{AppendOnlyTopNExecutor, TopNExecutor};
23
24pub struct TopNExecutorBuilder<const APPEND_ONLY: bool>;
25
26impl_stream_node_body!(TopN(TopNNode) => TopNExecutorBuilder<false>);
27impl_stream_node_body!(AppendOnlyTopN(TopNNode) => TopNExecutorBuilder<true>);
28
29impl<const APPEND_ONLY: bool> ExecutorBuilder for TopNExecutorBuilder<APPEND_ONLY> {
30    type Node = TopNNode;
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
39        let table = node.get_table()?;
40        let vnodes = params.vnode_bitmap.map(Arc::new);
41        let state_table = StateTableBuilder::new(table, store, vnodes)
42            .enable_preload_all_rows_by_config(&params.config)
43            .build()
44            .await;
45        let storage_key = table
46            .get_pk()
47            .iter()
48            .map(ColumnOrder::from_protobuf)
49            .collect();
50        let order_by = node
51            .order_by
52            .iter()
53            .map(ColumnOrder::from_protobuf)
54            .collect();
55
56        macro_rules! build {
57            ($executor:ident, $with_ties:literal) => {
58                Ok($executor::<_, $with_ties>::new(
59                    input,
60                    params.actor_context,
61                    params.info.schema.clone(),
62                    storage_key,
63                    (node.offset as usize, node.limit as usize),
64                    order_by,
65                    state_table,
66                )?
67                .boxed())
68            };
69        }
70
71        let exec: StreamResult<Box<dyn Execute>> = match (APPEND_ONLY, node.with_ties) {
72            (true, true) => build!(AppendOnlyTopNExecutor, true),
73            (true, false) => build!(AppendOnlyTopNExecutor, false),
74            (false, true) => build!(TopNExecutor, true),
75            (false, false) => build!(TopNExecutor, false),
76        };
77        Ok((params.info, exec?).into())
78    }
79}