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::StateTable;
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 = StateTable::from_table_catalog(table, store, vnodes).await;
39        let storage_key = table
40            .get_pk()
41            .iter()
42            .map(ColumnOrder::from_protobuf)
43            .collect();
44        let order_by = node
45            .order_by
46            .iter()
47            .map(ColumnOrder::from_protobuf)
48            .collect();
49
50        macro_rules! build {
51            ($excutor:ident, $with_ties:literal) => {
52                Ok($excutor::<_, $with_ties>::new(
53                    input,
54                    params.actor_context,
55                    params.info.schema.clone(),
56                    storage_key,
57                    (node.offset as usize, node.limit as usize),
58                    order_by,
59                    state_table,
60                )?
61                .boxed())
62            };
63        }
64
65        let exec: StreamResult<Box<dyn Execute>> = match (APPEND_ONLY, node.with_ties) {
66            (true, true) => build!(AppendOnlyTopNExecutor, true),
67            (true, false) => build!(AppendOnlyTopNExecutor, false),
68            (false, true) => build!(TopNExecutor, true),
69            (false, false) => build!(TopNExecutor, false),
70        };
71        Ok((params.info, exec?).into())
72    }
73}