risingwave_stream/from_proto/
top_n.rs

1// Copyright 2025 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<const APPEND_ONLY: bool> ExecutorBuilder for TopNExecutorBuilder<APPEND_ONLY> {
27    type Node = TopNNode;
28
29    async fn new_boxed_executor(
30        params: ExecutorParams,
31        node: &Self::Node,
32        store: impl StateStore,
33    ) -> StreamResult<Executor> {
34        let [input]: [_; 1] = params.input.try_into().unwrap();
35
36        let table = node.get_table()?;
37        let vnodes = params.vnode_bitmap.map(Arc::new);
38        let state_table = StateTableBuilder::new(table, store, vnodes)
39            .enable_preload_all_rows_by_config(&params.actor_context.streaming_config)
40            .build()
41            .await;
42        let storage_key = table
43            .get_pk()
44            .iter()
45            .map(ColumnOrder::from_protobuf)
46            .collect();
47        let order_by = node
48            .order_by
49            .iter()
50            .map(ColumnOrder::from_protobuf)
51            .collect();
52
53        macro_rules! build {
54            ($excutor:ident, $with_ties:literal) => {
55                Ok($excutor::<_, $with_ties>::new(
56                    input,
57                    params.actor_context,
58                    params.info.schema.clone(),
59                    storage_key,
60                    (node.offset as usize, node.limit as usize),
61                    order_by,
62                    state_table,
63                )?
64                .boxed())
65            };
66        }
67
68        let exec: StreamResult<Box<dyn Execute>> = match (APPEND_ONLY, node.with_ties) {
69            (true, true) => build!(AppendOnlyTopNExecutor, true),
70            (true, false) => build!(AppendOnlyTopNExecutor, false),
71            (false, true) => build!(TopNExecutor, true),
72            (false, false) => build!(TopNExecutor, false),
73        };
74        Ok((params.info, exec?).into())
75    }
76}