risingwave_stream/from_proto/
sink.rs1use std::sync::Arc;
16
17use anyhow::anyhow;
18use risingwave_common::bail;
19use risingwave_common::catalog::{ColumnCatalog, Schema};
20use risingwave_common::secret::LocalSecretManager;
21use risingwave_common::types::DataType;
22use risingwave_connector::match_sink_name_str;
23use risingwave_connector::sink::catalog::{SinkFormatDesc, SinkId, SinkType};
24use risingwave_connector::sink::file_sink::fs::FsSink;
25use risingwave_connector::sink::{
26 CONNECTOR_TYPE_KEY, SINK_TYPE_OPTION, SinkError, SinkMetaClient, SinkParam, SinkWriterParam,
27 build_sink,
28};
29use risingwave_pb::catalog::Table;
30use risingwave_pb::plan_common::PbColumnCatalog;
31use risingwave_pb::stream_plan::{SinkLogStoreType, SinkNode};
32use url::Url;
33
34use super::*;
35use crate::common::log_store_impl::in_mem::BoundedInMemLogStoreFactory;
36use crate::common::log_store_impl::kv_log_store::{
37 KV_LOG_STORE_V2_INFO, KvLogStoreFactory, KvLogStoreMetrics, KvLogStorePkInfo,
38};
39use crate::executor::{SinkExecutor, StreamExecutorError};
40
41pub struct SinkExecutorBuilder;
42
43fn resolve_pk_info(
44 input_schema: &Schema,
45 log_store_table: &Table,
46) -> StreamResult<&'static KvLogStorePkInfo> {
47 let predefined_column_len = log_store_table.columns.len() - input_schema.fields.len();
48
49 #[expect(deprecated)]
50 let info = match predefined_column_len {
51 len if len
52 == crate::common::log_store_impl::kv_log_store::KV_LOG_STORE_V1_INFO
53 .predefined_column_len() =>
54 {
55 Ok(&crate::common::log_store_impl::kv_log_store::KV_LOG_STORE_V1_INFO)
56 }
57 len if len == KV_LOG_STORE_V2_INFO.predefined_column_len() => Ok(&KV_LOG_STORE_V2_INFO),
58 other_len => Err(anyhow!(
59 "invalid log store predefined len {}. log store table: {:?}, input_schema: {:?}",
60 other_len,
61 log_store_table,
62 input_schema
63 )),
64 }?;
65 validate_payload_schema(
66 &log_store_table.columns[predefined_column_len..],
67 input_schema,
68 )?;
69 Ok(info)
70}
71
72fn validate_payload_schema(
73 log_store_payload_schema: &[PbColumnCatalog],
74 input_schema: &Schema,
75) -> StreamResult<()> {
76 if log_store_payload_schema
77 .iter()
78 .zip_eq(input_schema.fields.iter())
79 .all(|(log_store_col, input_field)| {
80 let log_store_col_type = DataType::from(
81 log_store_col
82 .column_desc
83 .as_ref()
84 .unwrap()
85 .column_type
86 .as_ref()
87 .unwrap(),
88 );
89 log_store_col_type.equals_datatype(&input_field.data_type)
90 })
91 {
92 Ok(())
93 } else {
94 Err(anyhow!(
95 "mismatch schema: log store: {:?}, input: {:?}",
96 log_store_payload_schema,
97 input_schema
98 )
99 .into())
100 }
101}
102
103impl ExecutorBuilder for SinkExecutorBuilder {
104 type Node = SinkNode;
105
106 async fn new_boxed_executor(
107 params: ExecutorParams,
108 node: &Self::Node,
109 state_store: impl StateStore,
110 ) -> StreamResult<Executor> {
111 let [input_executor]: [_; 1] = params.input.try_into().unwrap();
112 let input_data_types = input_executor.info().schema.data_types();
113 let chunk_size = params.env.config().developer.chunk_size;
114
115 let sink_desc = node.sink_desc.as_ref().unwrap();
116 let sink_type = SinkType::from_proto(sink_desc.get_sink_type().unwrap());
117 let sink_id: SinkId = sink_desc.get_id().into();
118 let sink_name = sink_desc.get_name().to_owned();
119 let db_name = sink_desc.get_db_name().into();
120 let sink_from_name = sink_desc.get_sink_from_name().into();
121 let properties = sink_desc.get_properties().clone();
122 let secret_refs = sink_desc.get_secret_refs().clone();
123 let downstream_pk = sink_desc
124 .downstream_pk
125 .iter()
126 .map(|i| *i as usize)
127 .collect_vec();
128 let columns = sink_desc
129 .column_catalogs
130 .clone()
131 .into_iter()
132 .map(ColumnCatalog::from)
133 .collect_vec();
134
135 let mut properties_with_secret =
136 LocalSecretManager::global().fill_secrets(properties, secret_refs)?;
137
138 if params.env.config().developer.switch_jdbc_pg_to_native
139 && let Some(connector_type) = properties_with_secret.get(CONNECTOR_TYPE_KEY)
140 && connector_type == "jdbc"
141 && let Some(url) = properties_with_secret.get("jdbc.url")
142 && url.starts_with("jdbc:postgresql:")
143 {
144 let jdbc_url = parse_jdbc_url(url)
145 .map_err(|e| StreamExecutorError::from((SinkError::Config(e), sink_id.sink_id)))?;
146 properties_with_secret.insert("host".to_owned(), jdbc_url.host);
147 properties_with_secret.insert("port".to_owned(), jdbc_url.port.to_string());
148 properties_with_secret.insert("database".to_owned(), jdbc_url.db_name);
149 properties_with_secret.insert("user".to_owned(), jdbc_url.username);
150 properties_with_secret.insert("password".to_owned(), jdbc_url.password);
151 if let Some(table_name) = properties_with_secret.get("table.name") {
152 properties_with_secret.insert("table".to_owned(), table_name.clone());
153 }
154 if let Some(schema_name) = properties_with_secret.get("schema.name") {
155 properties_with_secret.insert("schema".to_owned(), schema_name.clone());
156 }
157 }
159
160 let connector = {
161 let sink_type = properties_with_secret
162 .get(CONNECTOR_TYPE_KEY)
163 .ok_or_else(|| {
164 StreamExecutorError::from((
165 SinkError::Config(anyhow!("missing config: {}", CONNECTOR_TYPE_KEY)),
166 sink_id.sink_id,
167 ))
168 })?;
169
170 let sink_type_str = sink_type.to_string().to_lowercase();
171 match_sink_name_str!(
172 sink_type_str.as_str(),
173 SinkType,
174 Ok(SinkType::SINK_NAME),
175 |other: &str| {
176 Err(StreamExecutorError::from((
177 SinkError::Config(anyhow!("unsupported sink connector {}", other)),
178 sink_id.sink_id,
179 )))
180 }
181 )?
182 };
183 let format_desc = match &sink_desc.format_desc {
184 Some(f) => Some(
186 f.clone()
187 .try_into()
188 .map_err(|e| StreamExecutorError::from((e, sink_id.sink_id)))?,
189 ),
190 None => match properties_with_secret.get(SINK_TYPE_OPTION) {
191 Some(t) => SinkFormatDesc::from_legacy_type(connector, t)
193 .map_err(|e| StreamExecutorError::from((e, sink_id.sink_id)))?,
194 None => None,
196 },
197 };
198
199 let format_desc_with_secret = SinkParam::fill_secret_for_format_desc(format_desc)
200 .map_err(|e| StreamExecutorError::from((e, sink_id.sink_id)))?;
201
202 let sink_param = SinkParam {
203 sink_id,
204 sink_name: sink_name.clone(),
205 properties: properties_with_secret,
206 columns: columns
207 .iter()
208 .filter(|col| !col.is_hidden)
209 .map(|col| col.column_desc.clone())
210 .collect(),
211 downstream_pk,
212 sink_type,
213 format_desc: format_desc_with_secret,
214 db_name,
215 sink_from_name,
216 };
217
218 let sink_write_param = SinkWriterParam {
219 executor_id: params.executor_id,
220 vnode_bitmap: params.vnode_bitmap.clone(),
221 meta_client: params.env.meta_client().map(SinkMetaClient::MetaClient),
222 extra_partition_col_idx: sink_desc.extra_partition_col_idx.map(|v| v as usize),
223
224 actor_id: params.actor_context.id,
225 sink_id,
226 sink_name,
227 connector: connector.to_owned(),
228 streaming_config: params.env.config().as_ref().clone(),
229 };
230
231 let log_store_identity = format!(
232 "sink[{}]-[{}]-executor[{}]",
233 connector, sink_id.sink_id, params.executor_id
234 );
235
236 let sink = build_sink(sink_param.clone())
237 .map_err(|e| StreamExecutorError::from((e, sink_param.sink_id.sink_id)))?;
238
239 let exec = match node.log_store_type() {
240 SinkLogStoreType::InMemoryLogStore | SinkLogStoreType::Unspecified => {
243 let factory = BoundedInMemLogStoreFactory::new(1);
244 SinkExecutor::new(
245 params.actor_context,
246 params.info.clone(),
247 input_executor,
248 sink_write_param,
249 sink,
250 sink_param,
251 columns,
252 factory,
253 chunk_size,
254 input_data_types,
255 node.rate_limit.map(|x| x as _),
256 )
257 .await?
258 .boxed()
259 }
260 SinkLogStoreType::KvLogStore => {
261 let metrics = KvLogStoreMetrics::new(
262 ¶ms.executor_stats,
263 params.actor_context.id,
264 &sink_param,
265 connector,
266 );
267
268 let table = node.table.as_ref().unwrap().clone();
269 let input_schema = input_executor.schema();
270 let pk_info = resolve_pk_info(input_schema, &table)?;
271
272 let align_init_epoch = sink.is_coordinated_sink();
273
274 let factory = KvLogStoreFactory::new(
276 state_store,
277 table,
278 params.vnode_bitmap.clone().map(Arc::new),
279 65536,
280 metrics,
281 log_store_identity,
282 pk_info,
283 align_init_epoch,
284 );
285
286 SinkExecutor::new(
287 params.actor_context,
288 params.info.clone(),
289 input_executor,
290 sink_write_param,
291 sink,
292 sink_param,
293 columns,
294 factory,
295 chunk_size,
296 input_data_types,
297 node.rate_limit.map(|x| x as _),
298 )
299 .await?
300 .boxed()
301 }
302 };
303
304 Ok((params.info, exec).into())
305 }
306}
307
308struct JdbcUrl {
309 host: String,
310 port: u16,
311 db_name: String,
312 username: String,
313 password: String,
314}
315
316fn parse_jdbc_url(url: &str) -> anyhow::Result<JdbcUrl> {
317 if !url.starts_with("jdbc:postgresql") {
318 bail!(
319 "invalid jdbc url, to switch to postgres rust connector, we need to use the url jdbc:postgresql://..."
320 )
321 }
322
323 let url = url.replace("jdbc:", "");
325
326 let url = Url::parse(&url).map_err(|e| anyhow!(e).context("failed to parse jdbc url"))?;
328
329 let scheme = url.scheme();
330 assert_eq!("postgresql", scheme, "jdbc scheme should be postgresql");
331 let host = url
332 .host_str()
333 .ok_or_else(|| anyhow!("missing host in jdbc url"))?;
334 let port = url
335 .port()
336 .ok_or_else(|| anyhow!("missing port in jdbc url"))?;
337 let db_name = url.path();
338 let mut username = None;
339 let mut password = None;
340 for (key, value) in url.query_pairs() {
341 if key == "user" {
342 username = Some(value.to_string());
343 }
344 if key == "password" {
345 password = Some(value.to_string());
346 }
347 }
348 let username = username.ok_or_else(|| anyhow!("missing username in jdbc url"))?;
349 let password = password.ok_or_else(|| anyhow!("missing password in jdbc url"))?;
350
351 Ok(JdbcUrl {
352 host: host.to_owned(),
353 port,
354 db_name: db_name.to_owned(),
355 username: username.to_owned(),
356 password: password.to_owned(),
357 })
358}
359
360#[cfg(test)]
361mod tests {
362 use super::*;
363
364 #[test]
365 fn test_parse_jdbc_url() {
366 let url = "jdbc:postgresql://localhost:5432/test?user=postgres&password=postgres";
367 let jdbc_url = parse_jdbc_url(url).unwrap();
368 assert_eq!(jdbc_url.host, "localhost");
369 assert_eq!(jdbc_url.port, 5432);
370 assert_eq!(jdbc_url.db_name, "/test");
371 assert_eq!(jdbc_url.username, "postgres");
372 assert_eq!(jdbc_url.password, "postgres");
373 }
374}