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