risingwave_stream/from_proto/
asof_join.rs1use 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, JoinEncodingType as JoinEncodingTypeProto};
21
22use super::*;
23use crate::common::table::state_table::{StateTable, StateTableBuilder};
24use crate::executor::asof_join::*;
25use crate::executor::monitor::StreamingMetrics;
26use crate::executor::{
27 ActorContextRef, AsOfDesc, AsOfJoinType, CpuEncoding, JoinType, MemoryEncoding,
28};
29use crate::task::AtomicU64Ref;
30
31pub struct AsOfJoinExecutorBuilder;
32
33impl ExecutorBuilder for AsOfJoinExecutorBuilder {
34 type Node = AsOfJoinNode;
35
36 async fn new_boxed_executor(
37 params: ExecutorParams,
38 node: &Self::Node,
39 store: impl StateStore,
40 ) -> StreamResult<Executor> {
41 assert_eq!(AsOfJoinType::Inner, JoinType::Inner);
43 assert_eq!(AsOfJoinType::LeftOuter, JoinType::LeftOuter);
44 let vnodes = Arc::new(params.vnode_bitmap.expect("vnodes not set for AsOf 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 table_r = node.get_right_table()?;
50
51 let params_l = JoinParams::new(
52 node.get_left_key()
53 .iter()
54 .map(|key| *key as usize)
55 .collect_vec(),
56 node.get_left_deduped_input_pk_indices()
57 .iter()
58 .map(|key| *key as usize)
59 .collect_vec(),
60 );
61 let params_r = JoinParams::new(
62 node.get_right_key()
63 .iter()
64 .map(|key| *key as usize)
65 .collect_vec(),
66 node.get_right_deduped_input_pk_indices()
67 .iter()
68 .map(|key| *key as usize)
69 .collect_vec(),
70 );
71 let null_safe = node.get_null_safe().to_vec();
72 let output_indices = node
73 .get_output_indices()
74 .iter()
75 .map(|&x| x as usize)
76 .collect_vec();
77
78 let join_key_data_types = params_l
79 .join_key_indices
80 .iter()
81 .map(|idx| source_l.schema().fields[*idx].data_type())
82 .collect_vec();
83
84 let state_table_l = StateTableBuilder::new(table_l, store.clone(), Some(vnodes.clone()))
85 .enable_preload_all_rows_by_config(¶ms.actor_context.streaming_config)
86 .build()
87 .await;
88
89 let state_table_r = StateTableBuilder::new(table_r, store.clone(), Some(vnodes.clone()))
90 .enable_preload_all_rows_by_config(¶ms.actor_context.streaming_config)
91 .build()
92 .await;
93
94 let join_type_proto = node.get_join_type()?;
95 let as_of_desc_proto = node.get_asof_desc()?;
96 let asof_desc = AsOfDesc::from_protobuf(as_of_desc_proto)?;
97 let join_encoding_type = node
98 .get_join_encoding_type()
99 .unwrap_or(JoinEncodingTypeProto::MemoryOptimized);
100
101 let args = AsOfJoinExecutorDispatcherArgs {
102 ctx: params.actor_context,
103 info: params.info.clone(),
104 source_l,
105 source_r,
106 params_l,
107 params_r,
108 null_safe,
109 output_indices,
110 state_table_l,
111 state_table_r,
112 lru_manager: params.watermark_epoch,
113 metrics: params.executor_stats,
114 join_type_proto,
115 join_key_data_types,
116 chunk_size: params.env.config().developer.chunk_size,
117 high_join_amplification_threshold: params
118 .env
119 .config()
120 .developer
121 .high_join_amplification_threshold,
122 asof_desc,
123 join_encoding_type,
124 };
125
126 let exec = args.dispatch()?;
127 Ok((params.info, exec).into())
128 }
129}
130
131struct AsOfJoinExecutorDispatcherArgs<S: StateStore> {
132 ctx: ActorContextRef,
133 info: ExecutorInfo,
134 source_l: Executor,
135 source_r: Executor,
136 params_l: JoinParams,
137 params_r: JoinParams,
138 null_safe: Vec<bool>,
139 output_indices: Vec<usize>,
140 state_table_l: StateTable<S>,
141 state_table_r: StateTable<S>,
142 lru_manager: AtomicU64Ref,
143 metrics: Arc<StreamingMetrics>,
144 join_type_proto: JoinTypeProto,
145 join_key_data_types: Vec<DataType>,
146 chunk_size: usize,
147 high_join_amplification_threshold: usize,
148 asof_desc: AsOfDesc,
149 join_encoding_type: JoinEncodingTypeProto,
150}
151
152impl<S: StateStore> HashKeyDispatcher for AsOfJoinExecutorDispatcherArgs<S> {
153 type Output = StreamResult<Box<dyn Execute>>;
154
155 fn dispatch_impl<K: HashKey>(self) -> Self::Output {
156 macro_rules! build {
158 ($join_type:ident, $join_encoding:ident) => {
159 Ok(
160 AsOfJoinExecutor::<K, S, { AsOfJoinType::$join_type }, $join_encoding>::new(
161 self.ctx,
162 self.info,
163 self.source_l,
164 self.source_r,
165 self.params_l,
166 self.params_r,
167 self.null_safe,
168 self.output_indices,
169 self.state_table_l,
170 self.state_table_r,
171 self.lru_manager,
172 self.metrics,
173 self.chunk_size,
174 self.high_join_amplification_threshold,
175 self.asof_desc,
176 )
177 .boxed(),
178 )
179 };
180 }
181
182 macro_rules! build_match {
183 ($($join_type:ident),*) => {
184 match (self.join_type_proto, self.join_encoding_type) {
185 (JoinTypeProto::Unspecified, _) | (_, JoinEncodingTypeProto::Unspecified) => unreachable!(),
186 $(
187 (JoinTypeProto::$join_type, JoinEncodingTypeProto::MemoryOptimized) => build!($join_type, MemoryEncoding),
188 (JoinTypeProto::$join_type, JoinEncodingTypeProto::CpuOptimized) => build!($join_type, CpuEncoding),
189 )*
190 }
191 };
192 }
193
194 build_match! {
195 Inner,
196 LeftOuter
197 }
198 }
199
200 fn data_types(&self) -> &[DataType] {
201 &self.join_key_data_types
202 }
203}