risingwave_stream/from_proto/
hash_join.rs1use 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;
25
26use super::*;
27use crate::common::table::state_table::StateTable;
28use crate::executor::hash_join::*;
29use crate::executor::monitor::StreamingMetrics;
30use crate::executor::{ActorContextRef, JoinType};
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 args = HashJoinExecutorDispatcherArgs {
138 ctx: params.actor_context,
139 info: params.info.clone(),
140 source_l,
141 source_r,
142 params_l,
143 params_r,
144 null_safe,
145 output_indices,
146 cond: condition,
147 inequality_pairs,
148 state_table_l,
149 degree_state_table_l,
150 state_table_r,
151 degree_state_table_r,
152 lru_manager: params.watermark_epoch,
153 is_append_only,
154 metrics: params.executor_stats,
155 join_type_proto: node.get_join_type()?,
156 join_key_data_types,
157 chunk_size: params.env.config().developer.chunk_size,
158 high_join_amplification_threshold: params
159 .env
160 .config()
161 .developer
162 .high_join_amplification_threshold,
163 };
164
165 let exec = args.dispatch()?;
166 Ok((params.info, exec).into())
167 }
168}
169
170struct HashJoinExecutorDispatcherArgs<S: StateStore> {
171 ctx: ActorContextRef,
172 info: ExecutorInfo,
173 source_l: Executor,
174 source_r: Executor,
175 params_l: JoinParams,
176 params_r: JoinParams,
177 null_safe: Vec<bool>,
178 output_indices: Vec<usize>,
179 cond: Option<NonStrictExpression>,
180 inequality_pairs: Vec<(usize, usize, bool, Option<NonStrictExpression>)>,
181 state_table_l: StateTable<S>,
182 degree_state_table_l: StateTable<S>,
183 state_table_r: StateTable<S>,
184 degree_state_table_r: StateTable<S>,
185 lru_manager: AtomicU64Ref,
186 is_append_only: bool,
187 metrics: Arc<StreamingMetrics>,
188 join_type_proto: JoinTypeProto,
189 join_key_data_types: Vec<DataType>,
190 chunk_size: usize,
191 high_join_amplification_threshold: usize,
192}
193
194impl<S: StateStore> HashKeyDispatcher for HashJoinExecutorDispatcherArgs<S> {
195 type Output = StreamResult<Box<dyn Execute>>;
196
197 fn dispatch_impl<K: HashKey>(self) -> Self::Output {
198 macro_rules! build {
200 ($join_type:ident) => {
201 Ok(HashJoinExecutor::<K, S, { JoinType::$join_type }>::new(
202 self.ctx,
203 self.info,
204 self.source_l,
205 self.source_r,
206 self.params_l,
207 self.params_r,
208 self.null_safe,
209 self.output_indices,
210 self.cond,
211 self.inequality_pairs,
212 self.state_table_l,
213 self.degree_state_table_l,
214 self.state_table_r,
215 self.degree_state_table_r,
216 self.lru_manager,
217 self.is_append_only,
218 self.metrics,
219 self.chunk_size,
220 self.high_join_amplification_threshold,
221 )
222 .boxed())
223 };
224 }
225 match self.join_type_proto {
226 JoinTypeProto::AsofInner
227 | JoinTypeProto::AsofLeftOuter
228 | JoinTypeProto::Unspecified => unreachable!(),
229 JoinTypeProto::Inner => build!(Inner),
230 JoinTypeProto::LeftOuter => build!(LeftOuter),
231 JoinTypeProto::RightOuter => build!(RightOuter),
232 JoinTypeProto::FullOuter => build!(FullOuter),
233 JoinTypeProto::LeftSemi => build!(LeftSemi),
234 JoinTypeProto::LeftAnti => build!(LeftAnti),
235 JoinTypeProto::RightSemi => build!(RightSemi),
236 JoinTypeProto::RightAnti => build!(RightAnti),
237 }
238 }
239
240 fn data_types(&self) -> &[DataType] {
241 &self.join_key_data_types
242 }
243}