risingwave_stream/from_proto/
group_top_n.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::sync::Arc;

use risingwave_common::catalog::Schema;
use risingwave_common::hash::{HashKey, HashKeyDispatcher};
use risingwave_common::types::DataType;
use risingwave_common::util::sort_util::ColumnOrder;
use risingwave_pb::stream_plan::GroupTopNNode;

use super::*;
use crate::common::table::state_table::StateTable;
use crate::executor::{ActorContextRef, AppendOnlyGroupTopNExecutor, GroupTopNExecutor};
use crate::task::AtomicU64Ref;

pub struct GroupTopNExecutorBuilder<const APPEND_ONLY: bool>;

impl<const APPEND_ONLY: bool> ExecutorBuilder for GroupTopNExecutorBuilder<APPEND_ONLY> {
    type Node = GroupTopNNode;

    async fn new_boxed_executor(
        params: ExecutorParams,
        node: &Self::Node,
        store: impl StateStore,
    ) -> StreamResult<Executor> {
        let group_by: Vec<usize> = node
            .get_group_key()
            .iter()
            .map(|idx| *idx as usize)
            .collect();
        let table = node.get_table()?;
        let vnodes = params.vnode_bitmap.map(Arc::new);
        let state_table = StateTable::from_table_catalog(table, store, vnodes).await;
        let storage_key = table
            .get_pk()
            .iter()
            .map(ColumnOrder::from_protobuf)
            .collect();
        let [input]: [_; 1] = params.input.try_into().unwrap();
        let group_key_types = group_by
            .iter()
            .map(|i| input.schema()[*i].data_type())
            .collect();
        let order_by = node
            .order_by
            .iter()
            .map(ColumnOrder::from_protobuf)
            .collect();

        let args = GroupTopNExecutorDispatcherArgs {
            input,
            ctx: params.actor_context,
            schema: params.info.schema.clone(),
            storage_key,
            offset_and_limit: (node.offset as usize, node.limit as usize),
            order_by,
            group_by,
            state_table,
            watermark_epoch: params.watermark_epoch,
            group_key_types,

            with_ties: node.with_ties,
            append_only: APPEND_ONLY,
        };
        Ok((params.info, args.dispatch()?).into())
    }
}

struct GroupTopNExecutorDispatcherArgs<S: StateStore> {
    input: Executor,
    ctx: ActorContextRef,
    schema: Schema,
    storage_key: Vec<ColumnOrder>,
    offset_and_limit: (usize, usize),
    order_by: Vec<ColumnOrder>,
    group_by: Vec<usize>,
    state_table: StateTable<S>,
    watermark_epoch: AtomicU64Ref,
    group_key_types: Vec<DataType>,

    with_ties: bool,
    append_only: bool,
}

impl<S: StateStore> HashKeyDispatcher for GroupTopNExecutorDispatcherArgs<S> {
    type Output = StreamResult<Box<dyn Execute>>;

    fn dispatch_impl<K: HashKey>(self) -> Self::Output {
        macro_rules! build {
            ($excutor:ident, $with_ties:literal) => {
                Ok($excutor::<K, S, $with_ties>::new(
                    self.input,
                    self.ctx,
                    self.schema,
                    self.storage_key,
                    self.offset_and_limit,
                    self.order_by,
                    self.group_by,
                    self.state_table,
                    self.watermark_epoch,
                )?
                .boxed())
            };
        }
        match (self.append_only, self.with_ties) {
            (true, true) => build!(AppendOnlyGroupTopNExecutor, true),
            (true, false) => build!(AppendOnlyGroupTopNExecutor, false),
            (false, true) => build!(GroupTopNExecutor, true),
            (false, false) => build!(GroupTopNExecutor, false),
        }
    }

    fn data_types(&self) -> &[DataType] {
        &self.group_key_types
    }
}