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