1use core::fmt::Debug;
16use core::num::NonZeroU64;
17use std::collections::{BTreeMap, HashMap, HashSet};
18
19use anyhow::anyhow;
20use clickhouse::insert::Insert;
21use clickhouse::{Client as ClickHouseClient, Row as ClickHouseRow};
22use itertools::Itertools;
23use phf::{Set, phf_set};
24use risingwave_common::array::{Op, StreamChunk};
25use risingwave_common::catalog::Schema;
26use risingwave_common::row::Row;
27use risingwave_common::types::{DataType, Decimal, ScalarRefImpl, Serial};
28use serde::ser::{SerializeSeq, SerializeStruct};
29use serde::{Deserialize, Serialize};
30use serde_with::{DisplayFromStr, serde_as};
31use thiserror_ext::AsReport;
32use tonic::async_trait;
33use tracing::warn;
34use with_options::WithOptions;
35
36use super::decouple_checkpoint_log_sink::{
37 DecoupleCheckpointLogSinkerOf, default_commit_checkpoint_interval,
38};
39use super::writer::SinkWriter;
40use super::{SinkWriterMetrics, SinkWriterParam};
41use crate::enforce_secret::EnforceSecret;
42use crate::error::ConnectorResult;
43use crate::sink::{
44 Result, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, Sink, SinkError, SinkParam,
45};
46
47const QUERY_ENGINE: &str =
48 "select distinct ?fields from system.tables where database = ? and name = ?";
49const QUERY_COLUMN: &str =
50 "select distinct ?fields from system.columns where database = ? and table = ? order by ?";
51pub const CLICKHOUSE_SINK: &str = "clickhouse";
52
53const ALLOW_EXPERIMENTAL_JSON_TYPE: &str = "allow_experimental_json_type";
54const INPUT_FORMAT_BINARY_READ_JSON_AS_STRING: &str = "input_format_binary_read_json_as_string";
55const OUTPUT_FORMAT_BINARY_WRITE_JSON_AS_STRING: &str = "output_format_binary_write_json_as_string";
56
57#[serde_as]
58#[derive(Deserialize, Debug, Clone, WithOptions)]
59pub struct ClickHouseCommon {
60 #[serde(rename = "clickhouse.url")]
61 pub url: String,
62 #[serde(rename = "clickhouse.user")]
63 pub user: String,
64 #[serde(rename = "clickhouse.password")]
65 pub password: String,
66 #[serde(rename = "clickhouse.database")]
67 pub database: String,
68 #[serde(rename = "clickhouse.table")]
69 pub table: String,
70 #[serde(rename = "clickhouse.delete.column")]
71 pub delete_column: Option<String>,
72 #[serde(default = "default_commit_checkpoint_interval")]
74 #[serde_as(as = "DisplayFromStr")]
75 #[with_option(allow_alter_on_fly)]
76 pub commit_checkpoint_interval: u64,
77}
78
79impl EnforceSecret for ClickHouseCommon {
80 const ENFORCE_SECRET_PROPERTIES: Set<&'static str> = phf_set! {
81 "clickhouse.password", "clickhouse.user"
82 };
83}
84
85#[allow(clippy::enum_variant_names)]
86#[derive(Debug)]
87enum ClickHouseEngine {
88 MergeTree,
89 ReplacingMergeTree(Option<String>),
90 SummingMergeTree,
91 AggregatingMergeTree,
92 CollapsingMergeTree(String),
93 VersionedCollapsingMergeTree(String),
94 GraphiteMergeTree,
95 ReplicatedMergeTree,
96 ReplicatedReplacingMergeTree(Option<String>),
97 ReplicatedSummingMergeTree,
98 ReplicatedAggregatingMergeTree,
99 ReplicatedCollapsingMergeTree(String),
100 ReplicatedVersionedCollapsingMergeTree(String),
101 ReplicatedGraphiteMergeTree,
102 SharedMergeTree,
103 SharedReplacingMergeTree(Option<String>),
104 SharedSummingMergeTree,
105 SharedAggregatingMergeTree,
106 SharedCollapsingMergeTree(String),
107 SharedVersionedCollapsingMergeTree(String),
108 SharedGraphiteMergeTree,
109 Null,
110}
111impl ClickHouseEngine {
112 pub fn is_collapsing_engine(&self) -> bool {
113 matches!(
114 self,
115 ClickHouseEngine::CollapsingMergeTree(_)
116 | ClickHouseEngine::VersionedCollapsingMergeTree(_)
117 | ClickHouseEngine::ReplicatedCollapsingMergeTree(_)
118 | ClickHouseEngine::ReplicatedVersionedCollapsingMergeTree(_)
119 | ClickHouseEngine::SharedCollapsingMergeTree(_)
120 | ClickHouseEngine::SharedVersionedCollapsingMergeTree(_)
121 )
122 }
123
124 pub fn is_delete_replacing_engine(&self) -> bool {
125 match self {
126 ClickHouseEngine::ReplacingMergeTree(delete_col) => delete_col.is_some(),
127 ClickHouseEngine::ReplicatedReplacingMergeTree(delete_col) => delete_col.is_some(),
128 ClickHouseEngine::SharedReplacingMergeTree(delete_col) => delete_col.is_some(),
129 _ => false,
130 }
131 }
132
133 pub fn get_delete_col(&self) -> Option<String> {
134 match self {
135 ClickHouseEngine::ReplacingMergeTree(Some(delete_col)) => Some(delete_col.clone()),
136 ClickHouseEngine::ReplicatedReplacingMergeTree(Some(delete_col)) => {
137 Some(delete_col.clone())
138 }
139 ClickHouseEngine::SharedReplacingMergeTree(Some(delete_col)) => {
140 Some(delete_col.clone())
141 }
142 _ => None,
143 }
144 }
145
146 pub fn get_sign_name(&self) -> Option<String> {
147 match self {
148 ClickHouseEngine::CollapsingMergeTree(sign_name) => Some(sign_name.clone()),
149 ClickHouseEngine::VersionedCollapsingMergeTree(sign_name) => Some(sign_name.clone()),
150 ClickHouseEngine::ReplicatedCollapsingMergeTree(sign_name) => Some(sign_name.clone()),
151 ClickHouseEngine::ReplicatedVersionedCollapsingMergeTree(sign_name) => {
152 Some(sign_name.clone())
153 }
154 ClickHouseEngine::SharedCollapsingMergeTree(sign_name) => Some(sign_name.clone()),
155 ClickHouseEngine::SharedVersionedCollapsingMergeTree(sign_name) => {
156 Some(sign_name.clone())
157 }
158 _ => None,
159 }
160 }
161
162 pub fn is_shared_tree(&self) -> bool {
163 matches!(
164 self,
165 ClickHouseEngine::SharedMergeTree
166 | ClickHouseEngine::SharedReplacingMergeTree(_)
167 | ClickHouseEngine::SharedSummingMergeTree
168 | ClickHouseEngine::SharedAggregatingMergeTree
169 | ClickHouseEngine::SharedCollapsingMergeTree(_)
170 | ClickHouseEngine::SharedVersionedCollapsingMergeTree(_)
171 | ClickHouseEngine::SharedGraphiteMergeTree
172 )
173 }
174
175 pub fn from_query_engine(
176 engine_name: &ClickhouseQueryEngine,
177 config: &ClickHouseConfig,
178 ) -> Result<Self> {
179 match engine_name.engine.as_str() {
180 "MergeTree" => Ok(ClickHouseEngine::MergeTree),
181 "Null" => Ok(ClickHouseEngine::Null),
182 "ReplacingMergeTree" => {
183 let delete_column = config.common.delete_column.clone();
184 Ok(ClickHouseEngine::ReplacingMergeTree(delete_column))
185 }
186 "SummingMergeTree" => Ok(ClickHouseEngine::SummingMergeTree),
187 "AggregatingMergeTree" => Ok(ClickHouseEngine::AggregatingMergeTree),
188 "VersionedCollapsingMergeTree" => {
190 let sign_name = engine_name
191 .create_table_query
192 .split("VersionedCollapsingMergeTree(")
193 .last()
194 .ok_or_else(|| SinkError::ClickHouse("must have last".to_owned()))?
195 .split(',')
196 .next()
197 .ok_or_else(|| SinkError::ClickHouse("must have next".to_owned()))?
198 .trim()
199 .to_owned();
200 Ok(ClickHouseEngine::VersionedCollapsingMergeTree(sign_name))
201 }
202 "CollapsingMergeTree" => {
204 let sign_name = engine_name
205 .create_table_query
206 .split("CollapsingMergeTree(")
207 .last()
208 .ok_or_else(|| SinkError::ClickHouse("must have last".to_owned()))?
209 .split(')')
210 .next()
211 .ok_or_else(|| SinkError::ClickHouse("must have next".to_owned()))?
212 .trim()
213 .to_owned();
214 Ok(ClickHouseEngine::CollapsingMergeTree(sign_name))
215 }
216 "GraphiteMergeTree" => Ok(ClickHouseEngine::GraphiteMergeTree),
217 "ReplicatedMergeTree" => Ok(ClickHouseEngine::ReplicatedMergeTree),
218 "ReplicatedReplacingMergeTree" => {
219 let delete_column = config.common.delete_column.clone();
220 Ok(ClickHouseEngine::ReplicatedReplacingMergeTree(
221 delete_column,
222 ))
223 }
224 "ReplicatedSummingMergeTree" => Ok(ClickHouseEngine::ReplicatedSummingMergeTree),
225 "ReplicatedAggregatingMergeTree" => {
226 Ok(ClickHouseEngine::ReplicatedAggregatingMergeTree)
227 }
228 "ReplicatedVersionedCollapsingMergeTree" => {
230 let sign_name = engine_name
231 .create_table_query
232 .split("ReplicatedVersionedCollapsingMergeTree(")
233 .last()
234 .ok_or_else(|| SinkError::ClickHouse("must have last".to_owned()))?
235 .split(',')
236 .rev()
237 .nth(1)
238 .ok_or_else(|| SinkError::ClickHouse("must have index 1".to_owned()))?
239 .trim()
240 .to_owned();
241 Ok(ClickHouseEngine::ReplicatedVersionedCollapsingMergeTree(
242 sign_name,
243 ))
244 }
245 "ReplicatedCollapsingMergeTree" => {
247 let sign_name = engine_name
248 .create_table_query
249 .split("ReplicatedCollapsingMergeTree(")
250 .last()
251 .ok_or_else(|| SinkError::ClickHouse("must have last".to_owned()))?
252 .split(')')
253 .next()
254 .ok_or_else(|| SinkError::ClickHouse("must have next".to_owned()))?
255 .split(',')
256 .next_back()
257 .ok_or_else(|| SinkError::ClickHouse("must have last".to_owned()))?
258 .trim()
259 .to_owned();
260 Ok(ClickHouseEngine::ReplicatedCollapsingMergeTree(sign_name))
261 }
262 "ReplicatedGraphiteMergeTree" => Ok(ClickHouseEngine::ReplicatedGraphiteMergeTree),
263 "SharedMergeTree" => Ok(ClickHouseEngine::SharedMergeTree),
264 "SharedReplacingMergeTree" => {
265 let delete_column = config.common.delete_column.clone();
266 Ok(ClickHouseEngine::SharedReplacingMergeTree(delete_column))
267 }
268 "SharedSummingMergeTree" => Ok(ClickHouseEngine::SharedSummingMergeTree),
269 "SharedAggregatingMergeTree" => Ok(ClickHouseEngine::SharedAggregatingMergeTree),
270 "SharedVersionedCollapsingMergeTree" => {
272 let sign_name = engine_name
273 .create_table_query
274 .split("SharedVersionedCollapsingMergeTree(")
275 .last()
276 .ok_or_else(|| SinkError::ClickHouse("must have last".to_owned()))?
277 .split(',')
278 .rev()
279 .nth(1)
280 .ok_or_else(|| SinkError::ClickHouse("must have index 1".to_owned()))?
281 .trim()
282 .to_owned();
283 Ok(ClickHouseEngine::SharedVersionedCollapsingMergeTree(
284 sign_name,
285 ))
286 }
287 "SharedCollapsingMergeTree" => {
289 let sign_name = engine_name
290 .create_table_query
291 .split("SharedCollapsingMergeTree(")
292 .last()
293 .ok_or_else(|| SinkError::ClickHouse("must have last".to_owned()))?
294 .split(')')
295 .next()
296 .ok_or_else(|| SinkError::ClickHouse("must have next".to_owned()))?
297 .split(',')
298 .next_back()
299 .ok_or_else(|| SinkError::ClickHouse("must have last".to_owned()))?
300 .trim()
301 .to_owned();
302 Ok(ClickHouseEngine::SharedCollapsingMergeTree(sign_name))
303 }
304 "SharedGraphiteMergeTree" => Ok(ClickHouseEngine::SharedGraphiteMergeTree),
305 _ => Err(SinkError::ClickHouse(format!(
306 "Cannot find clickhouse engine {:?}",
307 engine_name.engine
308 ))),
309 }
310 }
311}
312
313impl ClickHouseCommon {
314 pub(crate) fn build_client(&self) -> ConnectorResult<ClickHouseClient> {
315 let client = ClickHouseClient::default() .with_url(&self.url)
317 .with_user(&self.user)
318 .with_password(&self.password)
319 .with_database(&self.database)
320 .with_option(ALLOW_EXPERIMENTAL_JSON_TYPE, "1")
321 .with_option(INPUT_FORMAT_BINARY_READ_JSON_AS_STRING, "1")
322 .with_option(OUTPUT_FORMAT_BINARY_WRITE_JSON_AS_STRING, "1");
323 Ok(client)
324 }
325}
326
327#[serde_as]
328#[derive(Clone, Debug, Deserialize, WithOptions)]
329pub struct ClickHouseConfig {
330 #[serde(flatten)]
331 pub common: ClickHouseCommon,
332
333 pub r#type: String, }
335
336impl EnforceSecret for ClickHouseConfig {
337 fn enforce_one(prop: &str) -> crate::error::ConnectorResult<()> {
338 ClickHouseCommon::enforce_one(prop)
339 }
340
341 fn enforce_secret<'a>(
342 prop_iter: impl Iterator<Item = &'a str>,
343 ) -> crate::error::ConnectorResult<()> {
344 for prop in prop_iter {
345 ClickHouseCommon::enforce_one(prop)?;
346 }
347 Ok(())
348 }
349}
350
351#[derive(Clone, Debug)]
352pub struct ClickHouseSink {
353 pub config: ClickHouseConfig,
354 schema: Schema,
355 pk_indices: Vec<usize>,
356 is_append_only: bool,
357}
358
359impl EnforceSecret for ClickHouseSink {
360 fn enforce_secret<'a>(
361 prop_iter: impl Iterator<Item = &'a str>,
362 ) -> crate::error::ConnectorResult<()> {
363 for prop in prop_iter {
364 ClickHouseConfig::enforce_one(prop)?;
365 }
366 Ok(())
367 }
368}
369
370impl ClickHouseConfig {
371 pub fn from_btreemap(properties: BTreeMap<String, String>) -> Result<Self> {
372 let config =
373 serde_json::from_value::<ClickHouseConfig>(serde_json::to_value(properties).unwrap())
374 .map_err(|e| SinkError::Config(anyhow!(e)))?;
375 if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT {
376 return Err(SinkError::Config(anyhow!(
377 "`{}` must be {}, or {}",
378 SINK_TYPE_OPTION,
379 SINK_TYPE_APPEND_ONLY,
380 SINK_TYPE_UPSERT
381 )));
382 }
383 Ok(config)
384 }
385}
386
387impl TryFrom<SinkParam> for ClickHouseSink {
388 type Error = SinkError;
389
390 fn try_from(param: SinkParam) -> std::result::Result<Self, Self::Error> {
391 let schema = param.schema();
392 let config = ClickHouseConfig::from_btreemap(param.properties)?;
393 Ok(Self {
394 config,
395 schema,
396 pk_indices: param.downstream_pk,
397 is_append_only: param.sink_type.is_append_only(),
398 })
399 }
400}
401
402impl ClickHouseSink {
403 fn check_column_name_and_type(&self, clickhouse_columns_desc: &[SystemColumn]) -> Result<()> {
405 let rw_fields_name = build_fields_name_type_from_schema(&self.schema)?;
406 let clickhouse_columns_desc: HashMap<String, SystemColumn> = clickhouse_columns_desc
407 .iter()
408 .map(|s| (s.name.clone(), s.clone()))
409 .collect();
410
411 if rw_fields_name.len().gt(&clickhouse_columns_desc.len()) {
412 return Err(SinkError::ClickHouse("The columns of the sink must be equal to or a superset of the target table's columns.".to_owned()));
413 }
414
415 for i in rw_fields_name {
416 let value = clickhouse_columns_desc.get(&i.0).ok_or_else(|| {
417 SinkError::ClickHouse(format!(
418 "Column name don't find in clickhouse, risingwave is {:?} ",
419 i.0
420 ))
421 })?;
422
423 Self::check_and_correct_column_type(&i.1, value)?;
424 }
425 Ok(())
426 }
427
428 fn check_pk_match(&self, clickhouse_columns_desc: &[SystemColumn]) -> Result<()> {
430 let mut clickhouse_pks: HashSet<String> = clickhouse_columns_desc
431 .iter()
432 .filter(|s| s.is_in_primary_key == 1)
433 .map(|s| s.name.clone())
434 .collect();
435
436 for (_, field) in self
437 .schema
438 .fields()
439 .iter()
440 .enumerate()
441 .filter(|(index, _)| self.pk_indices.contains(index))
442 {
443 if !clickhouse_pks.remove(&field.name) {
444 return Err(SinkError::ClickHouse(
445 "Clicklhouse and RisingWave pk is not match".to_owned(),
446 ));
447 }
448 }
449
450 if !clickhouse_pks.is_empty() {
451 return Err(SinkError::ClickHouse(
452 "Clicklhouse and RisingWave pk is not match".to_owned(),
453 ));
454 }
455 Ok(())
456 }
457
458 fn check_and_correct_column_type(
460 fields_type: &DataType,
461 ck_column: &SystemColumn,
462 ) -> Result<()> {
463 let is_match = match fields_type {
465 risingwave_common::types::DataType::Boolean => Ok(ck_column.r#type.contains("Bool")),
466 risingwave_common::types::DataType::Int16 => Ok(ck_column.r#type.contains("UInt16")
467 | ck_column.r#type.contains("Int16")
468 | ck_column.r#type.contains("Enum16")),
471 risingwave_common::types::DataType::Int32 => {
472 Ok(ck_column.r#type.contains("UInt32") | ck_column.r#type.contains("Int32"))
473 }
474 risingwave_common::types::DataType::Int64 => {
475 Ok(ck_column.r#type.contains("UInt64") | ck_column.r#type.contains("Int64"))
476 }
477 risingwave_common::types::DataType::Float32 => Ok(ck_column.r#type.contains("Float32")),
478 risingwave_common::types::DataType::Float64 => Ok(ck_column.r#type.contains("Float64")),
479 risingwave_common::types::DataType::Decimal => Ok(ck_column.r#type.contains("Decimal")),
480 risingwave_common::types::DataType::Date => Ok(ck_column.r#type.contains("Date32")),
481 risingwave_common::types::DataType::Varchar => Ok(ck_column.r#type.contains("String")),
482 risingwave_common::types::DataType::Time => Err(SinkError::ClickHouse(
483 "TIME is not supported for ClickHouse sink. Please convert to VARCHAR or other supported types.".to_owned(),
484 )),
485 risingwave_common::types::DataType::Timestamp => Err(SinkError::ClickHouse(
486 "clickhouse does not have a type corresponding to naive timestamp".to_owned(),
487 )),
488 risingwave_common::types::DataType::Timestamptz => {
489 Ok(ck_column.r#type.contains("DateTime64"))
490 }
491 risingwave_common::types::DataType::Interval => Err(SinkError::ClickHouse(
492 "INTERVAL is not supported for ClickHouse sink. Please convert to VARCHAR or other supported types.".to_owned(),
493 )),
494 risingwave_common::types::DataType::Struct(_) => Err(SinkError::ClickHouse(
495 "struct needs to be converted into a list".to_owned(),
496 )),
497 risingwave_common::types::DataType::List(list) => {
498 Self::check_and_correct_column_type(list.as_ref(), ck_column)?;
499 Ok(ck_column.r#type.contains("Array"))
500 }
501 risingwave_common::types::DataType::Bytea => Err(SinkError::ClickHouse(
502 "BYTEA is not supported for ClickHouse sink. Please convert to VARCHAR or other supported types.".to_owned(),
503 )),
504 risingwave_common::types::DataType::Jsonb => Ok(ck_column.r#type.contains("JSON")),
505 risingwave_common::types::DataType::Serial => {
506 Ok(ck_column.r#type.contains("UInt64") | ck_column.r#type.contains("Int64"))
507 }
508 risingwave_common::types::DataType::Int256 => Err(SinkError::ClickHouse(
509 "INT256 is not supported for ClickHouse sink.".to_owned(),
510 )),
511 risingwave_common::types::DataType::Map(_) => Err(SinkError::ClickHouse(
512 "MAP is not supported for ClickHouse sink.".to_owned(),
513 )),
514 DataType::Vector(_) => Err(SinkError::ClickHouse(
515 "VECTOR is not supported for ClickHouse sink.".to_owned(),
516 )),
517 };
518 if !is_match? {
519 return Err(SinkError::ClickHouse(format!(
520 "Column type mismatch for column {:?}: RisingWave type is {:?}, ClickHouse type is {:?}",
521 ck_column.name, fields_type, ck_column.r#type
522 )));
523 }
524
525 Ok(())
526 }
527}
528
529impl Sink for ClickHouseSink {
530 type LogSinker = DecoupleCheckpointLogSinkerOf<ClickHouseSinkWriter>;
531
532 const SINK_NAME: &'static str = CLICKHOUSE_SINK;
533
534 async fn validate(&self) -> Result<()> {
535 if !self.is_append_only && self.pk_indices.is_empty() {
537 return Err(SinkError::Config(anyhow!(
538 "Primary key not defined for upsert clickhouse sink (please define in `primary_key` field)"
539 )));
540 }
541
542 let client = self.config.common.build_client()?;
544
545 let (clickhouse_column, clickhouse_engine) =
546 query_column_engine_from_ck(client, &self.config).await?;
547 if clickhouse_engine.is_shared_tree() {
548 risingwave_common::license::Feature::ClickHouseSharedEngine
549 .check_available()
550 .map_err(|e| anyhow::anyhow!(e))?;
551 }
552
553 if !self.is_append_only
554 && !clickhouse_engine.is_collapsing_engine()
555 && !clickhouse_engine.is_delete_replacing_engine()
556 {
557 return match clickhouse_engine {
558 ClickHouseEngine::ReplicatedReplacingMergeTree(None) | ClickHouseEngine::ReplacingMergeTree(None) | ClickHouseEngine::SharedReplacingMergeTree(None) => {
559 Err(SinkError::ClickHouse("To enable upsert with a `ReplacingMergeTree`, you must set a `clickhouse.delete.column` to the UInt8 column in ClickHouse used to signify deletes. See https://clickhouse.com/docs/en/engines/table-engines/mergetree-family/replacingmergetree#is_deleted for more information".to_owned()))
560 }
561 _ => Err(SinkError::ClickHouse("If you want to use upsert, please use either `VersionedCollapsingMergeTree`, `CollapsingMergeTree` or the `ReplacingMergeTree` in ClickHouse".to_owned()))
562 };
563 }
564
565 self.check_column_name_and_type(&clickhouse_column)?;
566 if !self.is_append_only {
567 self.check_pk_match(&clickhouse_column)?;
568 }
569
570 if self.config.common.commit_checkpoint_interval == 0 {
571 return Err(SinkError::Config(anyhow!(
572 "`commit_checkpoint_interval` must be greater than 0"
573 )));
574 }
575 Ok(())
576 }
577
578 fn validate_alter_config(config: &BTreeMap<String, String>) -> Result<()> {
579 ClickHouseConfig::from_btreemap(config.clone())?;
580 Ok(())
581 }
582
583 async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result<Self::LogSinker> {
584 let writer = ClickHouseSinkWriter::new(
585 self.config.clone(),
586 self.schema.clone(),
587 self.pk_indices.clone(),
588 self.is_append_only,
589 )
590 .await?;
591 let commit_checkpoint_interval =
592 NonZeroU64::new(self.config.common.commit_checkpoint_interval).expect(
593 "commit_checkpoint_interval should be greater than 0, and it should be checked in config validation",
594 );
595
596 Ok(DecoupleCheckpointLogSinkerOf::new(
597 writer,
598 SinkWriterMetrics::new(&writer_param),
599 commit_checkpoint_interval,
600 ))
601 }
602}
603pub struct ClickHouseSinkWriter {
604 pub config: ClickHouseConfig,
605 #[expect(dead_code)]
606 schema: Schema,
607 #[expect(dead_code)]
608 pk_indices: Vec<usize>,
609 client: ClickHouseClient,
610 #[expect(dead_code)]
611 is_append_only: bool,
612 column_correct_vec: Vec<ClickHouseSchemaFeature>,
614 rw_fields_name_after_calibration: Vec<String>,
615 clickhouse_engine: ClickHouseEngine,
616 inserter: Option<Insert<ClickHouseColumn>>,
617}
618#[derive(Debug)]
619struct ClickHouseSchemaFeature {
620 can_null: bool,
621 accuracy_time: u8,
623
624 accuracy_decimal: (u8, u8),
625}
626
627impl ClickHouseSinkWriter {
628 pub async fn new(
629 config: ClickHouseConfig,
630 schema: Schema,
631 pk_indices: Vec<usize>,
632 is_append_only: bool,
633 ) -> Result<Self> {
634 let client = config.common.build_client()?;
635
636 let (clickhouse_column, clickhouse_engine) =
637 query_column_engine_from_ck(client.clone(), &config).await?;
638
639 let column_correct_vec: Result<Vec<ClickHouseSchemaFeature>> = clickhouse_column
640 .iter()
641 .map(Self::build_column_correct_vec)
642 .collect();
643 let mut rw_fields_name_after_calibration = build_fields_name_type_from_schema(&schema)?
644 .iter()
645 .map(|(a, _)| a.clone())
646 .collect_vec();
647
648 if let Some(sign) = clickhouse_engine.get_sign_name() {
649 rw_fields_name_after_calibration.push(sign);
650 }
651 if let Some(delete_col) = clickhouse_engine.get_delete_col() {
652 rw_fields_name_after_calibration.push(delete_col);
653 }
654 Ok(Self {
655 config,
656 schema,
657 pk_indices,
658 client,
659 is_append_only,
660 column_correct_vec: column_correct_vec?,
661 rw_fields_name_after_calibration,
662 clickhouse_engine,
663 inserter: None,
664 })
665 }
666
667 fn build_column_correct_vec(ck_column: &SystemColumn) -> Result<ClickHouseSchemaFeature> {
670 let can_null = ck_column.r#type.contains("Nullable");
671 let accuracy_time = if ck_column.r#type.contains("DateTime64(") {
673 ck_column
674 .r#type
675 .split("DateTime64(")
676 .last()
677 .ok_or_else(|| SinkError::ClickHouse("must have last".to_owned()))?
678 .split(')')
679 .next()
680 .ok_or_else(|| SinkError::ClickHouse("must have next".to_owned()))?
681 .split(',')
682 .next()
683 .ok_or_else(|| SinkError::ClickHouse("must have next".to_owned()))?
684 .parse::<u8>()
685 .map_err(|e| SinkError::ClickHouse(e.to_report_string()))?
686 } else {
687 0_u8
688 };
689 let accuracy_decimal = if ck_column.r#type.contains("Decimal(") {
690 let decimal_all = ck_column
691 .r#type
692 .split("Decimal(")
693 .last()
694 .ok_or_else(|| SinkError::ClickHouse("must have last".to_owned()))?
695 .split(')')
696 .next()
697 .ok_or_else(|| SinkError::ClickHouse("must have next".to_owned()))?
698 .split(", ")
699 .collect_vec();
700 let length = decimal_all
701 .first()
702 .ok_or_else(|| SinkError::ClickHouse("must have next".to_owned()))?
703 .parse::<u8>()
704 .map_err(|e| SinkError::ClickHouse(e.to_report_string()))?;
705
706 if length > 38 {
707 return Err(SinkError::ClickHouse(
708 "RW don't support Decimal256".to_owned(),
709 ));
710 }
711
712 let scale = decimal_all
713 .last()
714 .ok_or_else(|| SinkError::ClickHouse("must have next".to_owned()))?
715 .parse::<u8>()
716 .map_err(|e| SinkError::ClickHouse(e.to_report_string()))?;
717 (length, scale)
718 } else {
719 (0_u8, 0_u8)
720 };
721 Ok(ClickHouseSchemaFeature {
722 can_null,
723 accuracy_time,
724 accuracy_decimal,
725 })
726 }
727
728 async fn write(&mut self, chunk: StreamChunk) -> Result<()> {
729 if self.inserter.is_none() {
730 self.inserter = Some(self.client.insert_with_fields_name(
731 &self.config.common.table,
732 self.rw_fields_name_after_calibration.clone(),
733 )?);
734 }
735 for (op, row) in chunk.rows() {
736 let mut clickhouse_filed_vec = vec![];
737 for (index, data) in row.iter().enumerate() {
738 clickhouse_filed_vec.extend(ClickHouseFieldWithNull::from_scalar_ref(
739 data,
740 &self.column_correct_vec,
741 index,
742 )?);
743 }
744 match op {
745 Op::Insert | Op::UpdateInsert => {
746 if self.clickhouse_engine.is_collapsing_engine() {
747 clickhouse_filed_vec.push(ClickHouseFieldWithNull::WithoutSome(
748 ClickHouseField::Int8(1),
749 ));
750 }
751 if self.clickhouse_engine.is_delete_replacing_engine() {
752 clickhouse_filed_vec.push(ClickHouseFieldWithNull::WithoutSome(
753 ClickHouseField::Int8(0),
754 ))
755 }
756 }
757 Op::Delete | Op::UpdateDelete => {
758 if !self.clickhouse_engine.is_collapsing_engine()
759 && !self.clickhouse_engine.is_delete_replacing_engine()
760 {
761 return Err(SinkError::ClickHouse(
762 "Clickhouse engine don't support upsert".to_owned(),
763 ));
764 }
765 if self.clickhouse_engine.is_collapsing_engine() {
766 clickhouse_filed_vec.push(ClickHouseFieldWithNull::WithoutSome(
767 ClickHouseField::Int8(-1),
768 ));
769 }
770 if self.clickhouse_engine.is_delete_replacing_engine() {
771 clickhouse_filed_vec.push(ClickHouseFieldWithNull::WithoutSome(
772 ClickHouseField::Int8(1),
773 ))
774 }
775 }
776 }
777 let clickhouse_column = ClickHouseColumn {
778 row: clickhouse_filed_vec,
779 };
780 self.inserter
781 .as_mut()
782 .unwrap()
783 .write(&clickhouse_column)
784 .await?;
785 }
786 Ok(())
787 }
788}
789
790#[async_trait]
791impl SinkWriter for ClickHouseSinkWriter {
792 async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> {
793 self.write(chunk).await
794 }
795
796 async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> {
797 Ok(())
798 }
799
800 async fn abort(&mut self) -> Result<()> {
801 Ok(())
802 }
803
804 async fn barrier(&mut self, is_checkpoint: bool) -> Result<()> {
805 if is_checkpoint && let Some(inserter) = self.inserter.take() {
806 inserter.end().await?;
807 }
808 Ok(())
809 }
810}
811
812#[derive(ClickHouseRow, Deserialize, Clone)]
813struct SystemColumn {
814 name: String,
815 r#type: String,
816 is_in_primary_key: u8,
817}
818
819#[derive(ClickHouseRow, Deserialize)]
820struct ClickhouseQueryEngine {
821 #[expect(dead_code)]
822 name: String,
823 engine: String,
824 create_table_query: String,
825}
826
827async fn query_column_engine_from_ck(
828 client: ClickHouseClient,
829 config: &ClickHouseConfig,
830) -> Result<(Vec<SystemColumn>, ClickHouseEngine)> {
831 let query_engine = QUERY_ENGINE;
832 let query_column = QUERY_COLUMN;
833
834 let clickhouse_engine = client
835 .query(query_engine)
836 .bind(config.common.database.clone())
837 .bind(config.common.table.clone())
838 .fetch_all::<ClickhouseQueryEngine>()
839 .await?;
840 let mut clickhouse_column = client
841 .query(query_column)
842 .bind(config.common.database.clone())
843 .bind(config.common.table.clone())
844 .bind("position")
845 .fetch_all::<SystemColumn>()
846 .await?;
847 if clickhouse_engine.is_empty() || clickhouse_column.is_empty() {
848 return Err(SinkError::ClickHouse(format!(
849 "table {:?}.{:?} is not find in clickhouse",
850 config.common.database, config.common.table
851 )));
852 }
853
854 let clickhouse_engine =
855 ClickHouseEngine::from_query_engine(clickhouse_engine.first().unwrap(), config)?;
856
857 if let Some(sign) = &clickhouse_engine.get_sign_name() {
858 clickhouse_column.retain(|a| sign.ne(&a.name))
859 }
860
861 if let Some(delete_col) = &clickhouse_engine.get_delete_col() {
862 clickhouse_column.retain(|a| delete_col.ne(&a.name))
863 }
864
865 Ok((clickhouse_column, clickhouse_engine))
866}
867
868#[derive(ClickHouseRow, Debug)]
870struct ClickHouseColumn {
871 row: Vec<ClickHouseFieldWithNull>,
872}
873
874#[derive(Debug)]
876enum ClickHouseField {
877 Int16(i16),
878 Int32(i32),
879 Int64(i64),
880 Serial(Serial),
881 Float32(f32),
882 Float64(f64),
883 String(String),
884 Bool(bool),
885 List(Vec<ClickHouseFieldWithNull>),
886 Int8(i8),
887 Decimal(ClickHouseDecimal),
888}
889#[derive(Debug)]
890enum ClickHouseDecimal {
891 Decimal32(i32),
892 Decimal64(i64),
893 Decimal128(i128),
894}
895impl Serialize for ClickHouseDecimal {
896 fn serialize<S>(&self, serializer: S) -> std::result::Result<S::Ok, S::Error>
897 where
898 S: serde::Serializer,
899 {
900 match self {
901 ClickHouseDecimal::Decimal32(v) => serializer.serialize_i32(*v),
902 ClickHouseDecimal::Decimal64(v) => serializer.serialize_i64(*v),
903 ClickHouseDecimal::Decimal128(v) => serializer.serialize_i128(*v),
904 }
905 }
906}
907
908#[derive(Debug)]
910enum ClickHouseFieldWithNull {
911 WithSome(ClickHouseField),
912 WithoutSome(ClickHouseField),
913 None,
914}
915
916impl ClickHouseFieldWithNull {
917 pub fn from_scalar_ref(
918 data: Option<ScalarRefImpl<'_>>,
919 clickhouse_schema_feature_vec: &Vec<ClickHouseSchemaFeature>,
920 clickhouse_schema_feature_index: usize,
921 ) -> Result<Vec<ClickHouseFieldWithNull>> {
922 let clickhouse_schema_feature = clickhouse_schema_feature_vec
923 .get(clickhouse_schema_feature_index)
924 .ok_or_else(|| SinkError::ClickHouse(format!("No column found from clickhouse table schema, index is {clickhouse_schema_feature_index}")))?;
925 if data.is_none() {
926 if !clickhouse_schema_feature.can_null {
927 return Err(SinkError::ClickHouse(
928 "Cannot insert null value into non-nullable ClickHouse column".to_owned(),
929 ));
930 } else {
931 return Ok(vec![ClickHouseFieldWithNull::None]);
932 }
933 }
934 let data = match data.unwrap() {
935 ScalarRefImpl::Int16(v) => ClickHouseField::Int16(v),
936 ScalarRefImpl::Int32(v) => ClickHouseField::Int32(v),
937 ScalarRefImpl::Int64(v) => ClickHouseField::Int64(v),
938 ScalarRefImpl::Int256(_) => {
939 return Err(SinkError::ClickHouse(
940 "INT256 is not supported for ClickHouse sink.".to_owned(),
941 ));
942 }
943 ScalarRefImpl::Serial(v) => ClickHouseField::Serial(v),
944 ScalarRefImpl::Float32(v) => ClickHouseField::Float32(v.into_inner()),
945 ScalarRefImpl::Float64(v) => ClickHouseField::Float64(v.into_inner()),
946 ScalarRefImpl::Utf8(v) => ClickHouseField::String(v.to_owned()),
947 ScalarRefImpl::Bool(v) => ClickHouseField::Bool(v),
948 ScalarRefImpl::Decimal(d) => {
949 let d = if let Decimal::Normalized(d) = d {
950 let scale =
951 clickhouse_schema_feature.accuracy_decimal.1 as i32 - d.scale() as i32;
952 if scale < 0 {
953 d.mantissa() / 10_i128.pow(scale.unsigned_abs())
954 } else {
955 d.mantissa() * 10_i128.pow(scale as u32)
956 }
957 } else if clickhouse_schema_feature.can_null {
958 warn!("Inf, -Inf, Nan in RW decimal is converted into clickhouse null!");
959 return Ok(vec![ClickHouseFieldWithNull::None]);
960 } else {
961 warn!("Inf, -Inf, Nan in RW decimal is converted into clickhouse 0!");
962 0_i128
963 };
964 if clickhouse_schema_feature.accuracy_decimal.0 <= 9 {
965 ClickHouseField::Decimal(ClickHouseDecimal::Decimal32(d as i32))
966 } else if clickhouse_schema_feature.accuracy_decimal.0 <= 18 {
967 ClickHouseField::Decimal(ClickHouseDecimal::Decimal64(d as i64))
968 } else {
969 ClickHouseField::Decimal(ClickHouseDecimal::Decimal128(d))
970 }
971 }
972 ScalarRefImpl::Interval(_) => {
973 return Err(SinkError::ClickHouse(
974 "INTERVAL is not supported for ClickHouse sink. Please convert to VARCHAR or other supported types.".to_owned(),
975 ));
976 }
977 ScalarRefImpl::Date(v) => {
978 let days = v.get_nums_days_unix_epoch();
979 ClickHouseField::Int32(days)
980 }
981 ScalarRefImpl::Time(_) => {
982 return Err(SinkError::ClickHouse(
983 "TIME is not supported for ClickHouse sink. Please convert to VARCHAR or other supported types.".to_owned(),
984 ));
985 }
986 ScalarRefImpl::Timestamp(_) => {
987 return Err(SinkError::ClickHouse(
988 "clickhouse does not have a type corresponding to naive timestamp".to_owned(),
989 ));
990 }
991 ScalarRefImpl::Timestamptz(v) => {
992 let micros = v.timestamp_micros();
993 let ticks = match clickhouse_schema_feature.accuracy_time <= 6 {
994 true => {
995 micros / 10_i64.pow((6 - clickhouse_schema_feature.accuracy_time).into())
996 }
997 false => micros
998 .checked_mul(
999 10_i64.pow((clickhouse_schema_feature.accuracy_time - 6).into()),
1000 )
1001 .ok_or_else(|| SinkError::ClickHouse("DateTime64 overflow".to_owned()))?,
1002 };
1003 ClickHouseField::Int64(ticks)
1004 }
1005 ScalarRefImpl::Jsonb(v) => {
1006 let json_str = v.to_string();
1007 ClickHouseField::String(json_str)
1008 }
1009 ScalarRefImpl::Struct(v) => {
1010 let mut struct_vec = vec![];
1011 for (index, field) in v.iter_fields_ref().enumerate() {
1012 let a = Self::from_scalar_ref(
1013 field,
1014 clickhouse_schema_feature_vec,
1015 clickhouse_schema_feature_index + index,
1016 )?;
1017 struct_vec.push(ClickHouseFieldWithNull::WithoutSome(ClickHouseField::List(
1018 a,
1019 )));
1020 }
1021 return Ok(struct_vec);
1022 }
1023 ScalarRefImpl::List(v) => {
1024 let mut vec = vec![];
1025 for i in v.iter() {
1026 vec.extend(Self::from_scalar_ref(
1027 i,
1028 clickhouse_schema_feature_vec,
1029 clickhouse_schema_feature_index,
1030 )?)
1031 }
1032 return Ok(vec![ClickHouseFieldWithNull::WithoutSome(
1033 ClickHouseField::List(vec),
1034 )]);
1035 }
1036 ScalarRefImpl::Bytea(_) => {
1037 return Err(SinkError::ClickHouse(
1038 "BYTEA is not supported for ClickHouse sink. Please convert to VARCHAR or other supported types.".to_owned(),
1039 ));
1040 }
1041 ScalarRefImpl::Map(_) => {
1042 return Err(SinkError::ClickHouse(
1043 "MAP is not supported for ClickHouse sink.".to_owned(),
1044 ));
1045 }
1046 ScalarRefImpl::Vector(_) => {
1047 return Err(SinkError::ClickHouse(
1048 "VECTOR is not supported for ClickHouse sink.".to_owned(),
1049 ));
1050 }
1051 };
1052 let data = if clickhouse_schema_feature.can_null {
1053 vec![ClickHouseFieldWithNull::WithSome(data)]
1054 } else {
1055 vec![ClickHouseFieldWithNull::WithoutSome(data)]
1056 };
1057 Ok(data)
1058 }
1059}
1060
1061impl Serialize for ClickHouseField {
1062 fn serialize<S>(&self, serializer: S) -> std::result::Result<S::Ok, S::Error>
1063 where
1064 S: serde::Serializer,
1065 {
1066 match self {
1067 ClickHouseField::Int16(v) => serializer.serialize_i16(*v),
1068 ClickHouseField::Int32(v) => serializer.serialize_i32(*v),
1069 ClickHouseField::Int64(v) => serializer.serialize_i64(*v),
1070 ClickHouseField::Serial(v) => v.serialize(serializer),
1071 ClickHouseField::Float32(v) => serializer.serialize_f32(*v),
1072 ClickHouseField::Float64(v) => serializer.serialize_f64(*v),
1073 ClickHouseField::String(v) => serializer.serialize_str(v),
1074 ClickHouseField::Bool(v) => serializer.serialize_bool(*v),
1075 ClickHouseField::List(v) => {
1076 let mut s = serializer.serialize_seq(Some(v.len()))?;
1077 for i in v {
1078 s.serialize_element(i)?;
1079 }
1080 s.end()
1081 }
1082 ClickHouseField::Decimal(v) => v.serialize(serializer),
1083 ClickHouseField::Int8(v) => serializer.serialize_i8(*v),
1084 }
1085 }
1086}
1087impl Serialize for ClickHouseFieldWithNull {
1088 fn serialize<S>(&self, serializer: S) -> std::result::Result<S::Ok, S::Error>
1089 where
1090 S: serde::Serializer,
1091 {
1092 match self {
1093 ClickHouseFieldWithNull::WithSome(v) => serializer.serialize_some(v),
1094 ClickHouseFieldWithNull::WithoutSome(v) => v.serialize(serializer),
1095 ClickHouseFieldWithNull::None => serializer.serialize_none(),
1096 }
1097 }
1098}
1099impl Serialize for ClickHouseColumn {
1100 fn serialize<S>(&self, serializer: S) -> std::result::Result<S::Ok, S::Error>
1101 where
1102 S: serde::Serializer,
1103 {
1104 let mut s = serializer.serialize_struct("useless", self.row.len())?;
1105 for data in &self.row {
1106 s.serialize_field("useless", &data)?
1107 }
1108 s.end()
1109 }
1110}
1111
1112pub fn build_fields_name_type_from_schema(schema: &Schema) -> Result<Vec<(String, DataType)>> {
1115 let mut vec = vec![];
1116 for field in schema.fields() {
1117 if let DataType::Struct(st) = &field.data_type {
1118 for (name, data_type) in st.iter() {
1119 if matches!(data_type, DataType::Struct(_)) {
1120 return Err(SinkError::ClickHouse(
1121 "Only one level of nesting is supported for struct".to_owned(),
1122 ));
1123 } else {
1124 vec.push((
1125 format!("{}.{}", field.name, name),
1126 DataType::List(Box::new(data_type.clone())),
1127 ))
1128 }
1129 }
1130 } else {
1131 vec.push((field.name.clone(), field.data_type()));
1132 }
1133 }
1134 Ok(vec)
1135}