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;
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 =
85 StateTable::from_table_catalog(table_l, store.clone(), Some(vnodes.clone())).await;
86
87 let state_table_r =
88 StateTable::from_table_catalog(table_r, store.clone(), Some(vnodes.clone())).await;
89
90 let join_type_proto = node.get_join_type()?;
91 let as_of_desc_proto = node.get_asof_desc()?;
92 let asof_desc = AsOfDesc::from_protobuf(as_of_desc_proto)?;
93 let join_encoding_type = node
94 .get_join_encoding_type()
95 .unwrap_or(JoinEncodingTypeProto::MemoryOptimized);
96
97 let args = AsOfJoinExecutorDispatcherArgs {
98 ctx: params.actor_context,
99 info: params.info.clone(),
100 source_l,
101 source_r,
102 params_l,
103 params_r,
104 null_safe,
105 output_indices,
106 state_table_l,
107 state_table_r,
108 lru_manager: params.watermark_epoch,
109 metrics: params.executor_stats,
110 join_type_proto,
111 join_key_data_types,
112 chunk_size: params.env.config().developer.chunk_size,
113 high_join_amplification_threshold: params
114 .env
115 .config()
116 .developer
117 .high_join_amplification_threshold,
118 asof_desc,
119 join_encoding_type,
120 };
121
122 let exec = args.dispatch()?;
123 Ok((params.info, exec).into())
124 }
125}
126
127struct AsOfJoinExecutorDispatcherArgs<S: StateStore> {
128 ctx: ActorContextRef,
129 info: ExecutorInfo,
130 source_l: Executor,
131 source_r: Executor,
132 params_l: JoinParams,
133 params_r: JoinParams,
134 null_safe: Vec<bool>,
135 output_indices: Vec<usize>,
136 state_table_l: StateTable<S>,
137 state_table_r: StateTable<S>,
138 lru_manager: AtomicU64Ref,
139 metrics: Arc<StreamingMetrics>,
140 join_type_proto: JoinTypeProto,
141 join_key_data_types: Vec<DataType>,
142 chunk_size: usize,
143 high_join_amplification_threshold: usize,
144 asof_desc: AsOfDesc,
145 join_encoding_type: JoinEncodingTypeProto,
146}
147
148impl<S: StateStore> HashKeyDispatcher for AsOfJoinExecutorDispatcherArgs<S> {
149 type Output = StreamResult<Box<dyn Execute>>;
150
151 fn dispatch_impl<K: HashKey>(self) -> Self::Output {
152 macro_rules! build {
154 ($join_type:ident, $join_encoding:ident) => {
155 Ok(
156 AsOfJoinExecutor::<K, S, { AsOfJoinType::$join_type }, $join_encoding>::new(
157 self.ctx,
158 self.info,
159 self.source_l,
160 self.source_r,
161 self.params_l,
162 self.params_r,
163 self.null_safe,
164 self.output_indices,
165 self.state_table_l,
166 self.state_table_r,
167 self.lru_manager,
168 self.metrics,
169 self.chunk_size,
170 self.high_join_amplification_threshold,
171 self.asof_desc,
172 )
173 .boxed(),
174 )
175 };
176 }
177
178 macro_rules! build_match {
179 ($($join_type:ident),*) => {
180 match (self.join_type_proto, self.join_encoding_type) {
181 (JoinTypeProto::Unspecified, _) | (_, JoinEncodingTypeProto::Unspecified) => unreachable!(),
182 $(
183 (JoinTypeProto::$join_type, JoinEncodingTypeProto::MemoryOptimized) => build!($join_type, MemoryEncoding),
184 (JoinTypeProto::$join_type, JoinEncodingTypeProto::CpuOptimized) => build!($join_type, CpuEncoding),
185 )*
186 }
187 };
188 }
189
190 build_match! {
191 Inner,
192 LeftOuter
193 }
194 }
195
196 fn data_types(&self) -> &[DataType] {
197 &self.join_key_data_types
198 }
199}