risingwave_stream/from_proto/
asof_join.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::hash::{HashKey, HashKeyDispatcher};
18use risingwave_common::types::DataType;
19use risingwave_pb::plan_common::AsOfJoinType as JoinTypeProto;
20use risingwave_pb::stream_plan::AsOfJoinNode;
21
22use super::*;
23use crate::common::table::state_table::StateTable;
24use crate::executor::asof_join::*;
25use crate::executor::monitor::StreamingMetrics;
26use crate::executor::{ActorContextRef, AsOfDesc, AsOfJoinType, JoinType};
27use crate::task::AtomicU64Ref;
28
29pub struct AsOfJoinExecutorBuilder;
30
31impl ExecutorBuilder for AsOfJoinExecutorBuilder {
32    type Node = AsOfJoinNode;
33
34    async fn new_boxed_executor(
35        params: ExecutorParams,
36        node: &Self::Node,
37        store: impl StateStore,
38    ) -> StreamResult<Executor> {
39        // This assert is to make sure AsOf join can use `JoinChunkBuilder` as Hash join.
40        assert_eq!(AsOfJoinType::Inner, JoinType::Inner);
41        assert_eq!(AsOfJoinType::LeftOuter, JoinType::LeftOuter);
42        let vnodes = Arc::new(params.vnode_bitmap.expect("vnodes not set for AsOf join"));
43
44        let [source_l, source_r]: [_; 2] = params.input.try_into().unwrap();
45
46        let table_l = node.get_left_table()?;
47        let table_r = node.get_right_table()?;
48
49        let params_l = JoinParams::new(
50            node.get_left_key()
51                .iter()
52                .map(|key| *key as usize)
53                .collect_vec(),
54            node.get_left_deduped_input_pk_indices()
55                .iter()
56                .map(|key| *key as usize)
57                .collect_vec(),
58        );
59        let params_r = JoinParams::new(
60            node.get_right_key()
61                .iter()
62                .map(|key| *key as usize)
63                .collect_vec(),
64            node.get_right_deduped_input_pk_indices()
65                .iter()
66                .map(|key| *key as usize)
67                .collect_vec(),
68        );
69        let null_safe = node.get_null_safe().to_vec();
70        let output_indices = node
71            .get_output_indices()
72            .iter()
73            .map(|&x| x as usize)
74            .collect_vec();
75
76        let join_key_data_types = params_l
77            .join_key_indices
78            .iter()
79            .map(|idx| source_l.schema().fields[*idx].data_type())
80            .collect_vec();
81
82        let state_table_l =
83            StateTable::from_table_catalog(table_l, store.clone(), Some(vnodes.clone())).await;
84
85        let state_table_r =
86            StateTable::from_table_catalog(table_r, store.clone(), Some(vnodes.clone())).await;
87
88        let join_type_proto = node.get_join_type()?;
89        let as_of_desc_proto = node.get_asof_desc()?;
90        let asof_desc = AsOfDesc::from_protobuf(as_of_desc_proto)?;
91
92        let args = AsOfJoinExecutorDispatcherArgs {
93            ctx: params.actor_context,
94            info: params.info.clone(),
95            source_l,
96            source_r,
97            params_l,
98            params_r,
99            null_safe,
100            output_indices,
101            state_table_l,
102            state_table_r,
103            lru_manager: params.watermark_epoch,
104            metrics: params.executor_stats,
105            join_type_proto,
106            join_key_data_types,
107            chunk_size: params.env.config().developer.chunk_size,
108            high_join_amplification_threshold: params
109                .env
110                .config()
111                .developer
112                .high_join_amplification_threshold,
113            asof_desc,
114        };
115
116        let exec = args.dispatch()?;
117        Ok((params.info, exec).into())
118    }
119}
120
121struct AsOfJoinExecutorDispatcherArgs<S: StateStore> {
122    ctx: ActorContextRef,
123    info: ExecutorInfo,
124    source_l: Executor,
125    source_r: Executor,
126    params_l: JoinParams,
127    params_r: JoinParams,
128    null_safe: Vec<bool>,
129    output_indices: Vec<usize>,
130    state_table_l: StateTable<S>,
131    state_table_r: StateTable<S>,
132    lru_manager: AtomicU64Ref,
133    metrics: Arc<StreamingMetrics>,
134    join_type_proto: JoinTypeProto,
135    join_key_data_types: Vec<DataType>,
136    chunk_size: usize,
137    high_join_amplification_threshold: usize,
138    asof_desc: AsOfDesc,
139}
140
141impl<S: StateStore> HashKeyDispatcher for AsOfJoinExecutorDispatcherArgs<S> {
142    type Output = StreamResult<Box<dyn Execute>>;
143
144    fn dispatch_impl<K: HashKey>(self) -> Self::Output {
145        /// This macro helps to fill the const generic type parameter.
146        macro_rules! build {
147            ($join_type:ident) => {
148                Ok(AsOfJoinExecutor::<K, S, { AsOfJoinType::$join_type }>::new(
149                    self.ctx,
150                    self.info,
151                    self.source_l,
152                    self.source_r,
153                    self.params_l,
154                    self.params_r,
155                    self.null_safe,
156                    self.output_indices,
157                    self.state_table_l,
158                    self.state_table_r,
159                    self.lru_manager,
160                    self.metrics,
161                    self.chunk_size,
162                    self.high_join_amplification_threshold,
163                    self.asof_desc,
164                )
165                .boxed())
166            };
167        }
168        match self.join_type_proto {
169            JoinTypeProto::Unspecified => unreachable!(),
170            JoinTypeProto::Inner => build!(Inner),
171            JoinTypeProto::LeftOuter => build!(LeftOuter),
172        }
173    }
174
175    fn data_types(&self) -> &[DataType] {
176        &self.join_key_data_types
177    }
178}