risingwave_stream/from_proto/
hash_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::cmp::min;
16use std::sync::Arc;
17
18use risingwave_common::hash::{HashKey, HashKeyDispatcher};
19use risingwave_common::types::DataType;
20use risingwave_expr::expr::{
21    InputRefExpression, NonStrictExpression, build_func_non_strict, build_non_strict_from_prost,
22};
23use risingwave_pb::plan_common::JoinType as JoinTypeProto;
24use risingwave_pb::stream_plan::{HashJoinNode, JoinEncodingType as JoinEncodingTypeProto};
25
26use super::*;
27use crate::common::table::state_table::StateTable;
28use crate::executor::hash_join::*;
29use crate::executor::monitor::StreamingMetrics;
30use crate::executor::{ActorContextRef, CpuEncoding, JoinType, MemoryEncoding};
31use crate::task::AtomicU64Ref;
32
33pub struct HashJoinExecutorBuilder;
34
35impl ExecutorBuilder for HashJoinExecutorBuilder {
36    type Node = HashJoinNode;
37
38    async fn new_boxed_executor(
39        params: ExecutorParams,
40        node: &Self::Node,
41        store: impl StateStore,
42    ) -> StreamResult<Executor> {
43        let is_append_only = node.is_append_only;
44        let vnodes = Arc::new(params.vnode_bitmap.expect("vnodes not set for hash join"));
45
46        let [source_l, source_r]: [_; 2] = params.input.try_into().unwrap();
47
48        let table_l = node.get_left_table()?;
49        let degree_table_l = node.get_left_degree_table()?;
50
51        let table_r = node.get_right_table()?;
52        let degree_table_r = node.get_right_degree_table()?;
53
54        let params_l = JoinParams::new(
55            node.get_left_key()
56                .iter()
57                .map(|key| *key as usize)
58                .collect_vec(),
59            node.get_left_deduped_input_pk_indices()
60                .iter()
61                .map(|key| *key as usize)
62                .collect_vec(),
63        );
64        let params_r = JoinParams::new(
65            node.get_right_key()
66                .iter()
67                .map(|key| *key as usize)
68                .collect_vec(),
69            node.get_right_deduped_input_pk_indices()
70                .iter()
71                .map(|key| *key as usize)
72                .collect_vec(),
73        );
74        let null_safe = node.get_null_safe().to_vec();
75        let output_indices = node
76            .get_output_indices()
77            .iter()
78            .map(|&x| x as usize)
79            .collect_vec();
80
81        let condition = match node.get_condition() {
82            Ok(cond_prost) => Some(build_non_strict_from_prost(
83                cond_prost,
84                params.eval_error_report.clone(),
85            )?),
86            Err(_) => None,
87        };
88        trace!("Join non-equi condition: {:?}", condition);
89        let mut inequality_pairs = Vec::with_capacity(node.get_inequality_pairs().len());
90        for inequality_pair in node.get_inequality_pairs() {
91            let key_required_larger = inequality_pair.get_key_required_larger() as usize;
92            let key_required_smaller = inequality_pair.get_key_required_smaller() as usize;
93            inequality_pairs.push((
94                key_required_larger,
95                key_required_smaller,
96                inequality_pair.get_clean_state(),
97                if let Some(delta_expression) = inequality_pair.delta_expression.as_ref() {
98                    let data_type = source_l.schema().fields
99                        [min(key_required_larger, key_required_smaller)]
100                    .data_type();
101                    Some(build_func_non_strict(
102                        delta_expression.delta_type(),
103                        data_type.clone(),
104                        vec![
105                            Box::new(InputRefExpression::new(data_type, 0)),
106                            build_non_strict_from_prost(
107                                delta_expression.delta.as_ref().unwrap(),
108                                params.eval_error_report.clone(),
109                            )?
110                            .into_inner(),
111                        ],
112                        params.eval_error_report.clone(),
113                    )?)
114                } else {
115                    None
116                },
117            ));
118        }
119
120        let join_key_data_types = params_l
121            .join_key_indices
122            .iter()
123            .map(|idx| source_l.schema().fields[*idx].data_type())
124            .collect_vec();
125
126        let state_table_l =
127            StateTable::from_table_catalog(table_l, store.clone(), Some(vnodes.clone())).await;
128        let degree_state_table_l =
129            StateTable::from_table_catalog(degree_table_l, store.clone(), Some(vnodes.clone()))
130                .await;
131
132        let state_table_r =
133            StateTable::from_table_catalog(table_r, store.clone(), Some(vnodes.clone())).await;
134        let degree_state_table_r =
135            StateTable::from_table_catalog(degree_table_r, store, Some(vnodes)).await;
136
137        let join_encoding_type = node
138            .get_join_encoding_type()
139            .unwrap_or(JoinEncodingTypeProto::MemoryOptimized);
140
141        let args = HashJoinExecutorDispatcherArgs {
142            ctx: params.actor_context,
143            info: params.info.clone(),
144            source_l,
145            source_r,
146            params_l,
147            params_r,
148            null_safe,
149            output_indices,
150            cond: condition,
151            inequality_pairs,
152            state_table_l,
153            degree_state_table_l,
154            state_table_r,
155            degree_state_table_r,
156            lru_manager: params.watermark_epoch,
157            is_append_only,
158            metrics: params.executor_stats,
159            join_type_proto: node.get_join_type()?,
160            join_key_data_types,
161            chunk_size: params.env.config().developer.chunk_size,
162            high_join_amplification_threshold: params
163                .env
164                .config()
165                .developer
166                .high_join_amplification_threshold,
167            join_encoding_type,
168        };
169
170        let exec = args.dispatch()?;
171        Ok((params.info, exec).into())
172    }
173}
174
175struct HashJoinExecutorDispatcherArgs<S: StateStore> {
176    ctx: ActorContextRef,
177    info: ExecutorInfo,
178    source_l: Executor,
179    source_r: Executor,
180    params_l: JoinParams,
181    params_r: JoinParams,
182    null_safe: Vec<bool>,
183    output_indices: Vec<usize>,
184    cond: Option<NonStrictExpression>,
185    inequality_pairs: Vec<(usize, usize, bool, Option<NonStrictExpression>)>,
186    state_table_l: StateTable<S>,
187    degree_state_table_l: StateTable<S>,
188    state_table_r: StateTable<S>,
189    degree_state_table_r: StateTable<S>,
190    lru_manager: AtomicU64Ref,
191    is_append_only: bool,
192    metrics: Arc<StreamingMetrics>,
193    join_type_proto: JoinTypeProto,
194    join_key_data_types: Vec<DataType>,
195    chunk_size: usize,
196    high_join_amplification_threshold: usize,
197    join_encoding_type: JoinEncodingTypeProto,
198}
199
200impl<S: StateStore> HashKeyDispatcher for HashJoinExecutorDispatcherArgs<S> {
201    type Output = StreamResult<Box<dyn Execute>>;
202
203    fn dispatch_impl<K: HashKey>(self) -> Self::Output {
204        /// This macro helps to fill the const generic type parameter.
205        macro_rules! build {
206            ($join_type:ident, $join_encoding:ident) => {
207                Ok(
208                    HashJoinExecutor::<K, S, { JoinType::$join_type }, $join_encoding>::new(
209                        self.ctx,
210                        self.info,
211                        self.source_l,
212                        self.source_r,
213                        self.params_l,
214                        self.params_r,
215                        self.null_safe,
216                        self.output_indices,
217                        self.cond,
218                        self.inequality_pairs,
219                        self.state_table_l,
220                        self.degree_state_table_l,
221                        self.state_table_r,
222                        self.degree_state_table_r,
223                        self.lru_manager,
224                        self.is_append_only,
225                        self.metrics,
226                        self.chunk_size,
227                        self.high_join_amplification_threshold,
228                    )
229                    .boxed(),
230                )
231            };
232        }
233
234        macro_rules! build_match {
235            ($($join_type:ident),*) => {
236                match (self.join_type_proto, self.join_encoding_type) {
237                    (JoinTypeProto::AsofInner, _)
238                    | (JoinTypeProto::AsofLeftOuter, _)
239                    | (JoinTypeProto::Unspecified, _)
240                    | (_, JoinEncodingTypeProto::Unspecified ) => unreachable!(),
241                    $(
242                        (JoinTypeProto::$join_type, JoinEncodingTypeProto::MemoryOptimized) => build!($join_type, MemoryEncoding),
243                        (JoinTypeProto::$join_type, JoinEncodingTypeProto::CpuOptimized) => build!($join_type, CpuEncoding),
244                    )*
245                }
246            };
247        }
248        build_match! {
249            Inner,
250            LeftOuter,
251            RightOuter,
252            FullOuter,
253            LeftSemi,
254            LeftAnti,
255            RightSemi,
256            RightAnti
257        }
258    }
259
260    fn data_types(&self) -> &[DataType] {
261        &self.join_key_data_types
262    }
263}