1use std::collections::{BTreeMap, HashMap};
16use std::num::NonZeroU64;
17use std::sync::Arc;
18
19use anyhow::anyhow;
20use async_trait::async_trait;
21use bytes::Bytes;
22use mysql_async::Opts;
23use mysql_async::prelude::Queryable;
24use risingwave_common::array::{Op, StreamChunk};
25use risingwave_common::catalog::Schema;
26use risingwave_common::types::DataType;
27use serde::Deserialize;
28use serde_derive::Serialize;
29use serde_json::Value;
30use serde_with::{DisplayFromStr, serde_as};
31use thiserror_ext::AsReport;
32use url::form_urlencoded;
33use with_options::WithOptions;
34
35use super::decouple_checkpoint_log_sink::DEFAULT_COMMIT_CHECKPOINT_INTERVAL_WITH_SINK_DECOUPLE;
36use super::doris_starrocks_connector::{
37 HeaderBuilder, InserterInner, STARROCKS_DELETE_SIGN, STARROCKS_SUCCESS_STATUS,
38 StarrocksTxnRequestBuilder,
39};
40use super::encoder::{JsonEncoder, RowEncoder};
41use super::{
42 SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, SinkError, SinkParam,
43 SinkWriterMetrics,
44};
45use crate::enforce_secret::EnforceSecret;
46use crate::sink::decouple_checkpoint_log_sink::DecoupleCheckpointLogSinkerOf;
47use crate::sink::{Result, Sink, SinkWriter, SinkWriterParam};
48
49pub const STARROCKS_SINK: &str = "starrocks";
50const STARROCK_MYSQL_PREFER_SOCKET: &str = "false";
51const STARROCK_MYSQL_MAX_ALLOWED_PACKET: usize = 1024;
52const STARROCK_MYSQL_WAIT_TIMEOUT: usize = 28800;
53
54const fn _default_stream_load_http_timeout_ms() -> u64 {
55 30 * 1000
56}
57
58const fn default_use_https() -> bool {
59 false
60}
61
62#[serde_as]
63#[derive(Deserialize, Debug, Clone, WithOptions)]
64pub struct StarrocksCommon {
65 #[serde(rename = "starrocks.host")]
67 pub host: String,
68 #[serde(rename = "starrocks.mysqlport", alias = "starrocks.query_port")]
70 pub mysql_port: String,
71 #[serde(rename = "starrocks.httpport", alias = "starrocks.http_port")]
73 pub http_port: String,
74 #[serde(rename = "starrocks.user")]
76 pub user: String,
77 #[serde(rename = "starrocks.password")]
79 pub password: String,
80 #[serde(rename = "starrocks.database")]
82 pub database: String,
83 #[serde(rename = "starrocks.table")]
85 pub table: String,
86
87 #[serde(rename = "starrocks.use_https")]
89 #[serde(default = "default_use_https")]
90 #[serde_as(as = "DisplayFromStr")]
91 pub use_https: bool,
92}
93
94impl EnforceSecret for StarrocksCommon {
95 const ENFORCE_SECRET_PROPERTIES: phf::Set<&'static str> = phf::phf_set! {
96 "starrocks.password", "starrocks.user"
97 };
98}
99
100#[serde_as]
101#[derive(Clone, Debug, Deserialize, WithOptions)]
102pub struct StarrocksConfig {
103 #[serde(flatten)]
104 pub common: StarrocksCommon,
105
106 #[serde(
108 rename = "starrocks.stream_load.http.timeout.ms",
109 default = "_default_stream_load_http_timeout_ms"
110 )]
111 #[serde_as(as = "DisplayFromStr")]
112 pub stream_load_http_timeout_ms: u64,
113
114 #[serde(default = "default_commit_checkpoint_interval")]
120 #[serde_as(as = "DisplayFromStr")]
121 #[with_option(allow_alter_on_fly)]
122 pub commit_checkpoint_interval: u64,
123
124 #[serde(rename = "starrocks.partial_update")]
126 pub partial_update: Option<String>,
127
128 pub r#type: String, }
130
131impl EnforceSecret for StarrocksConfig {
132 fn enforce_one(prop: &str) -> crate::error::ConnectorResult<()> {
133 StarrocksCommon::enforce_one(prop)
134 }
135}
136
137fn default_commit_checkpoint_interval() -> u64 {
138 DEFAULT_COMMIT_CHECKPOINT_INTERVAL_WITH_SINK_DECOUPLE
139}
140
141impl StarrocksConfig {
142 pub fn from_btreemap(properties: BTreeMap<String, String>) -> Result<Self> {
143 let config =
144 serde_json::from_value::<StarrocksConfig>(serde_json::to_value(properties).unwrap())
145 .map_err(|e| SinkError::Config(anyhow!(e)))?;
146 if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT {
147 return Err(SinkError::Config(anyhow!(
148 "`{}` must be {}, or {}",
149 SINK_TYPE_OPTION,
150 SINK_TYPE_APPEND_ONLY,
151 SINK_TYPE_UPSERT
152 )));
153 }
154 if config.commit_checkpoint_interval == 0 {
155 return Err(SinkError::Config(anyhow!(
156 "`commit_checkpoint_interval` must be greater than 0"
157 )));
158 }
159 Ok(config)
160 }
161}
162
163#[derive(Debug)]
164pub struct StarrocksSink {
165 pub config: StarrocksConfig,
166 schema: Schema,
167 pk_indices: Vec<usize>,
168 is_append_only: bool,
169}
170
171impl EnforceSecret for StarrocksSink {
172 fn enforce_secret<'a>(
173 prop_iter: impl Iterator<Item = &'a str>,
174 ) -> crate::error::ConnectorResult<()> {
175 for prop in prop_iter {
176 StarrocksConfig::enforce_one(prop)?;
177 }
178 Ok(())
179 }
180}
181
182impl StarrocksSink {
183 pub fn new(param: SinkParam, config: StarrocksConfig, schema: Schema) -> Result<Self> {
184 let pk_indices = param.downstream_pk.clone();
185 let is_append_only = param.sink_type.is_append_only();
186 Ok(Self {
187 config,
188 schema,
189 pk_indices,
190 is_append_only,
191 })
192 }
193}
194
195impl StarrocksSink {
196 fn check_column_name_and_type(
197 &self,
198 starrocks_columns_desc: HashMap<String, String>,
199 ) -> Result<()> {
200 let rw_fields_name = self.schema.fields();
201 if rw_fields_name.len() > starrocks_columns_desc.len() {
202 return Err(SinkError::Starrocks("The columns of the sink must be equal to or a superset of the target table's columns.".to_owned()));
203 }
204
205 for i in rw_fields_name {
206 let value = starrocks_columns_desc.get(&i.name).ok_or_else(|| {
207 SinkError::Starrocks(format!(
208 "Column name don't find in starrocks, risingwave is {:?} ",
209 i.name
210 ))
211 })?;
212 if !Self::check_and_correct_column_type(&i.data_type, value)? {
213 return Err(SinkError::Starrocks(format!(
214 "Column type don't match, column name is {:?}. starrocks type is {:?} risingwave type is {:?} ",
215 i.name, value, i.data_type
216 )));
217 }
218 }
219 Ok(())
220 }
221
222 fn check_and_correct_column_type(
223 rw_data_type: &DataType,
224 starrocks_data_type: &String,
225 ) -> Result<bool> {
226 match rw_data_type {
227 risingwave_common::types::DataType::Boolean => {
228 Ok(starrocks_data_type.contains("tinyint") | starrocks_data_type.contains("boolean"))
229 }
230 risingwave_common::types::DataType::Int16 => {
231 Ok(starrocks_data_type.contains("smallint"))
232 }
233 risingwave_common::types::DataType::Int32 => Ok(starrocks_data_type.contains("int")),
234 risingwave_common::types::DataType::Int64 => Ok(starrocks_data_type.contains("bigint")),
235 risingwave_common::types::DataType::Float32 => {
236 Ok(starrocks_data_type.contains("float"))
237 }
238 risingwave_common::types::DataType::Float64 => {
239 Ok(starrocks_data_type.contains("double"))
240 }
241 risingwave_common::types::DataType::Decimal => {
242 Ok(starrocks_data_type.contains("decimal"))
243 }
244 risingwave_common::types::DataType::Date => Ok(starrocks_data_type.contains("date")),
245 risingwave_common::types::DataType::Varchar => {
246 Ok(starrocks_data_type.contains("varchar"))
247 }
248 risingwave_common::types::DataType::Time => Err(SinkError::Starrocks(
249 "TIME is not supported for Starrocks sink. Please convert to VARCHAR or other supported types.".to_owned(),
250 )),
251 risingwave_common::types::DataType::Timestamp => {
252 Ok(starrocks_data_type.contains("datetime"))
253 }
254 risingwave_common::types::DataType::Timestamptz => Err(SinkError::Starrocks(
255 "TIMESTAMP WITH TIMEZONE is not supported for Starrocks sink as Starrocks doesn't store time values with timezone information. Please convert to TIMESTAMP first.".to_owned(),
256 )),
257 risingwave_common::types::DataType::Interval => Err(SinkError::Starrocks(
258 "INTERVAL is not supported for Starrocks sink. Please convert to VARCHAR or other supported types.".to_owned(),
259 )),
260 risingwave_common::types::DataType::Struct(_) => Err(SinkError::Starrocks(
261 "STRUCT is not supported for Starrocks sink.".to_owned(),
262 )),
263 risingwave_common::types::DataType::List(list) => {
264 if starrocks_data_type.contains("unknown") {
266 return Ok(true);
267 }
268 let check_result = Self::check_and_correct_column_type(list.as_ref(), starrocks_data_type)?;
269 Ok(check_result && starrocks_data_type.contains("array"))
270 }
271 risingwave_common::types::DataType::Bytea => Err(SinkError::Starrocks(
272 "BYTEA is not supported for Starrocks sink. Please convert to VARCHAR or other supported types.".to_owned(),
273 )),
274 risingwave_common::types::DataType::Jsonb => Ok(starrocks_data_type.contains("json")),
275 risingwave_common::types::DataType::Serial => {
276 Ok(starrocks_data_type.contains("bigint"))
277 }
278 risingwave_common::types::DataType::Int256 => Err(SinkError::Starrocks(
279 "INT256 is not supported for Starrocks sink.".to_owned(),
280 )),
281 risingwave_common::types::DataType::Map(_) => Err(SinkError::Starrocks(
282 "MAP is not supported for Starrocks sink.".to_owned(),
283 )),
284 DataType::Vector(_) => todo!("VECTOR_PLACEHOLDER"),
285 }
286 }
287}
288
289impl Sink for StarrocksSink {
290 type LogSinker = DecoupleCheckpointLogSinkerOf<StarrocksSinkWriter>;
291
292 const SINK_NAME: &'static str = STARROCKS_SINK;
293
294 async fn validate(&self) -> Result<()> {
295 if !self.is_append_only && self.pk_indices.is_empty() {
296 return Err(SinkError::Config(anyhow!(
297 "Primary key not defined for upsert starrocks sink (please define in `primary_key` field)"
298 )));
299 }
300 let mut client = StarrocksSchemaClient::new(
302 self.config.common.host.clone(),
303 self.config.common.mysql_port.clone(),
304 self.config.common.table.clone(),
305 self.config.common.database.clone(),
306 self.config.common.user.clone(),
307 self.config.common.password.clone(),
308 )
309 .await?;
310 let (read_model, pks) = client.get_pk_from_starrocks().await?;
311
312 if !self.is_append_only && read_model.ne("PRIMARY_KEYS") {
313 return Err(SinkError::Config(anyhow!(
314 "If you want to use upsert, please set the keysType of starrocks to PRIMARY_KEY"
315 )));
316 }
317
318 for (index, filed) in self.schema.fields().iter().enumerate() {
319 if self.pk_indices.contains(&index) && !pks.contains(&filed.name) {
320 return Err(SinkError::Starrocks(format!(
321 "Can't find pk {:?} in starrocks",
322 filed.name
323 )));
324 }
325 }
326
327 let starrocks_columns_desc = client.get_columns_from_starrocks().await?;
328
329 self.check_column_name_and_type(starrocks_columns_desc)?;
330 Ok(())
331 }
332
333 fn validate_alter_config(config: &BTreeMap<String, String>) -> Result<()> {
334 StarrocksConfig::from_btreemap(config.clone())?;
335 Ok(())
336 }
337
338 async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result<Self::LogSinker> {
339 let commit_checkpoint_interval =
340 NonZeroU64::new(self.config.commit_checkpoint_interval).expect(
341 "commit_checkpoint_interval should be greater than 0, and it should be checked in config validation",
342 );
343
344 let writer = StarrocksSinkWriter::new(
345 self.config.clone(),
346 self.schema.clone(),
347 self.pk_indices.clone(),
348 self.is_append_only,
349 writer_param.executor_id,
350 )?;
351
352 let metrics = SinkWriterMetrics::new(&writer_param);
353
354 Ok(DecoupleCheckpointLogSinkerOf::new(
355 writer,
356 metrics,
357 commit_checkpoint_interval,
358 ))
359 }
360}
361
362pub struct StarrocksSinkWriter {
363 pub config: StarrocksConfig,
364 #[expect(dead_code)]
365 schema: Schema,
366 #[expect(dead_code)]
367 pk_indices: Vec<usize>,
368 is_append_only: bool,
369 client: Option<StarrocksClient>,
370 txn_client: Arc<StarrocksTxnClient>,
371 row_encoder: JsonEncoder,
372 executor_id: u64,
373 curr_txn_label: Option<String>,
374}
375
376impl TryFrom<SinkParam> for StarrocksSink {
377 type Error = SinkError;
378
379 fn try_from(param: SinkParam) -> std::result::Result<Self, Self::Error> {
380 let schema = param.schema();
381 let config = StarrocksConfig::from_btreemap(param.properties.clone())?;
382 StarrocksSink::new(param, config, schema)
383 }
384}
385
386impl StarrocksSinkWriter {
387 pub fn new(
388 config: StarrocksConfig,
389 schema: Schema,
390 pk_indices: Vec<usize>,
391 is_append_only: bool,
392 executor_id: u64,
393 ) -> Result<Self> {
394 let mut field_names = schema.names_str();
395 if !is_append_only {
396 field_names.push(STARROCKS_DELETE_SIGN);
397 };
398 let field_names = field_names
401 .into_iter()
402 .map(|name| format!("`{}`", name))
403 .collect::<Vec<String>>();
404 let field_names_str = field_names
405 .iter()
406 .map(|name| name.as_str())
407 .collect::<Vec<&str>>();
408
409 let header = HeaderBuilder::new()
410 .add_common_header()
411 .set_user_password(config.common.user.clone(), config.common.password.clone())
412 .add_json_format()
413 .set_partial_update(config.partial_update.clone())
414 .set_columns_name(field_names_str)
415 .set_db(config.common.database.clone())
416 .set_table(config.common.table.clone())
417 .build();
418
419 let url = if config.common.use_https {
420 format!("https://{}:{}", config.common.host, config.common.http_port)
421 } else {
422 format!("http://{}:{}", config.common.host, config.common.http_port)
423 };
424 let txn_request_builder =
425 StarrocksTxnRequestBuilder::new(url, header, config.stream_load_http_timeout_ms)?;
426
427 Ok(Self {
428 config,
429 schema: schema.clone(),
430 pk_indices,
431 is_append_only,
432 client: None,
433 txn_client: Arc::new(StarrocksTxnClient::new(txn_request_builder)),
434 row_encoder: JsonEncoder::new_with_starrocks(schema, None),
435 executor_id,
436 curr_txn_label: None,
437 })
438 }
439
440 async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> {
441 for (op, row) in chunk.rows() {
442 if op != Op::Insert {
443 continue;
444 }
445 let row_json_string = Value::Object(self.row_encoder.encode(row)?).to_string();
446 self.client
447 .as_mut()
448 .ok_or_else(|| SinkError::Starrocks("Can't find starrocks sink insert".to_owned()))?
449 .write(row_json_string.into())
450 .await?;
451 }
452 Ok(())
453 }
454
455 async fn upsert(&mut self, chunk: StreamChunk) -> Result<()> {
456 for (op, row) in chunk.rows() {
457 match op {
458 Op::Insert => {
459 let mut row_json_value = self.row_encoder.encode(row)?;
460 row_json_value.insert(
461 STARROCKS_DELETE_SIGN.to_owned(),
462 Value::String("0".to_owned()),
463 );
464 let row_json_string = serde_json::to_string(&row_json_value).map_err(|e| {
465 SinkError::Starrocks(format!("Json derialize error: {}", e.as_report()))
466 })?;
467 self.client
468 .as_mut()
469 .ok_or_else(|| {
470 SinkError::Starrocks("Can't find starrocks sink insert".to_owned())
471 })?
472 .write(row_json_string.into())
473 .await?;
474 }
475 Op::Delete => {
476 let mut row_json_value = self.row_encoder.encode(row)?;
477 row_json_value.insert(
478 STARROCKS_DELETE_SIGN.to_owned(),
479 Value::String("1".to_owned()),
480 );
481 let row_json_string = serde_json::to_string(&row_json_value).map_err(|e| {
482 SinkError::Starrocks(format!("Json derialize error: {}", e.as_report()))
483 })?;
484 self.client
485 .as_mut()
486 .ok_or_else(|| {
487 SinkError::Starrocks("Can't find starrocks sink insert".to_owned())
488 })?
489 .write(row_json_string.into())
490 .await?;
491 }
492 Op::UpdateDelete => {}
493 Op::UpdateInsert => {
494 let mut row_json_value = self.row_encoder.encode(row)?;
495 row_json_value.insert(
496 STARROCKS_DELETE_SIGN.to_owned(),
497 Value::String("0".to_owned()),
498 );
499 let row_json_string = serde_json::to_string(&row_json_value).map_err(|e| {
500 SinkError::Starrocks(format!("Json derialize error: {}", e.as_report()))
501 })?;
502 self.client
503 .as_mut()
504 .ok_or_else(|| {
505 SinkError::Starrocks("Can't find starrocks sink insert".to_owned())
506 })?
507 .write(row_json_string.into())
508 .await?;
509 }
510 }
511 }
512 Ok(())
513 }
514
515 #[inline(always)]
517 fn new_txn_label(&self) -> String {
518 format!(
519 "rw-txn-{}-{}",
520 self.executor_id,
521 chrono::Utc::now().timestamp_micros()
522 )
523 }
524
525 async fn prepare_and_commit(&self, txn_label: String) -> Result<()> {
526 tracing::debug!(?txn_label, "prepare transaction");
527 let txn_label_res = self.txn_client.prepare(txn_label.clone()).await?;
528 if txn_label != txn_label_res {
529 return Err(SinkError::Starrocks(format!(
530 "label {} returned from prepare transaction {} differs from the current one",
531 txn_label, txn_label_res
532 )));
533 }
534 tracing::debug!(?txn_label, "commit transaction");
535 let txn_label_res = self.txn_client.commit(txn_label.clone()).await?;
536 if txn_label != txn_label_res {
537 return Err(SinkError::Starrocks(format!(
538 "label {} returned from commit transaction {} differs from the current one",
539 txn_label, txn_label_res
540 )));
541 }
542 Ok(())
543 }
544}
545
546impl Drop for StarrocksSinkWriter {
547 fn drop(&mut self) {
548 if let Some(txn_label) = self.curr_txn_label.take() {
549 let txn_client = self.txn_client.clone();
550 tokio::spawn(async move {
551 if let Err(e) = txn_client.rollback(txn_label.clone()).await {
552 tracing::error!(
553 "starrocks rollback transaction error: {:?}, txn label: {}",
554 e.as_report(),
555 txn_label
556 );
557 }
558 });
559 }
560 }
561}
562
563#[async_trait]
564impl SinkWriter for StarrocksSinkWriter {
565 async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> {
566 Ok(())
567 }
568
569 async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> {
570 if self.curr_txn_label.is_none() {
574 let txn_label = self.new_txn_label();
575 tracing::debug!(?txn_label, "begin transaction");
576 let txn_label_res = self.txn_client.begin(txn_label.clone()).await?;
577 if txn_label != txn_label_res {
578 return Err(SinkError::Starrocks(format!(
579 "label {} returned from StarRocks {} differs from generated one",
580 txn_label, txn_label_res
581 )));
582 }
583 self.curr_txn_label = Some(txn_label.clone());
584 }
585 if self.client.is_none() {
586 let txn_label = self.curr_txn_label.clone();
587 self.client = Some(StarrocksClient::new(
588 self.txn_client.load(txn_label.unwrap()).await?,
589 ));
590 }
591 if self.is_append_only {
592 self.append_only(chunk).await
593 } else {
594 self.upsert(chunk).await
595 }
596 }
597
598 async fn barrier(&mut self, is_checkpoint: bool) -> Result<()> {
599 if let Some(client) = self.client.take() {
600 client.finish().await?;
606 }
607
608 if is_checkpoint
609 && let Some(txn_label) = self.curr_txn_label.take()
610 && let Err(err) = self.prepare_and_commit(txn_label.clone()).await
611 {
612 match self.txn_client.rollback(txn_label.clone()).await {
613 Ok(_) => tracing::warn!(
614 ?txn_label,
615 "transaction is successfully rolled back due to commit failure"
616 ),
617 Err(err) => {
618 tracing::warn!(?txn_label, error = ?err.as_report(), "Couldn't roll back transaction after commit failed")
619 }
620 }
621
622 return Err(err);
623 }
624 Ok(())
625 }
626
627 async fn abort(&mut self) -> Result<()> {
628 if let Some(txn_label) = self.curr_txn_label.take() {
629 tracing::debug!(?txn_label, "rollback transaction");
630 self.txn_client.rollback(txn_label).await?;
631 }
632 Ok(())
633 }
634}
635
636pub struct StarrocksSchemaClient {
637 table: String,
638 db: String,
639 conn: mysql_async::Conn,
640}
641
642impl StarrocksSchemaClient {
643 pub async fn new(
644 host: String,
645 port: String,
646 table: String,
647 db: String,
648 user: String,
649 password: String,
650 ) -> Result<Self> {
651 let user = form_urlencoded::byte_serialize(user.as_bytes()).collect::<String>();
654 let password = form_urlencoded::byte_serialize(password.as_bytes()).collect::<String>();
655
656 let conn_uri = format!(
657 "mysql://{}:{}@{}:{}/{}?prefer_socket={}&max_allowed_packet={}&wait_timeout={}",
658 user,
659 password,
660 host,
661 port,
662 db,
663 STARROCK_MYSQL_PREFER_SOCKET,
664 STARROCK_MYSQL_MAX_ALLOWED_PACKET,
665 STARROCK_MYSQL_WAIT_TIMEOUT
666 );
667 let pool = mysql_async::Pool::new(
668 Opts::from_url(&conn_uri)
669 .map_err(|err| SinkError::DorisStarrocksConnect(anyhow!(err)))?,
670 );
671 let conn = pool
672 .get_conn()
673 .await
674 .map_err(|err| SinkError::DorisStarrocksConnect(anyhow!(err)))?;
675
676 Ok(Self { table, db, conn })
677 }
678
679 pub async fn get_columns_from_starrocks(&mut self) -> Result<HashMap<String, String>> {
680 let query = format!(
681 "select column_name, column_type from information_schema.columns where table_name = {:?} and table_schema = {:?};",
682 self.table, self.db
683 );
684 let mut query_map: HashMap<String, String> = HashMap::default();
685 self.conn
686 .query_map(query, |(column_name, column_type)| {
687 query_map.insert(column_name, column_type)
688 })
689 .await
690 .map_err(|err| SinkError::DorisStarrocksConnect(anyhow!(err)))?;
691 Ok(query_map)
692 }
693
694 pub async fn get_pk_from_starrocks(&mut self) -> Result<(String, String)> {
695 let query = format!(
696 "select table_model, primary_key, sort_key from information_schema.tables_config where table_name = {:?} and table_schema = {:?};",
697 self.table, self.db
698 );
699 let table_mode_pk: (String, String) = self
700 .conn
701 .query_map(
702 query,
703 |(table_model, primary_key, sort_key): (String, String, String)| match table_model
704 .as_str()
705 {
706 "AGG_KEYS" => (table_model, sort_key),
710 _ => (table_model, primary_key),
711 },
712 )
713 .await
714 .map_err(|err| SinkError::DorisStarrocksConnect(anyhow!(err)))?
715 .first()
716 .ok_or_else(|| {
717 SinkError::Starrocks(format!(
718 "Can't find schema with table {:?} and database {:?}",
719 self.table, self.db
720 ))
721 })?
722 .clone();
723 Ok(table_mode_pk)
724 }
725}
726
727#[derive(Debug, Serialize, Deserialize)]
728pub struct StarrocksInsertResultResponse {
729 #[serde(rename = "TxnId")]
730 pub txn_id: Option<i64>,
731 #[serde(rename = "Seq")]
732 pub seq: Option<i64>,
733 #[serde(rename = "Label")]
734 pub label: Option<String>,
735 #[serde(rename = "Status")]
736 pub status: String,
737 #[serde(rename = "Message")]
738 pub message: String,
739 #[serde(rename = "NumberTotalRows")]
740 pub number_total_rows: Option<i64>,
741 #[serde(rename = "NumberLoadedRows")]
742 pub number_loaded_rows: Option<i64>,
743 #[serde(rename = "NumberFilteredRows")]
744 pub number_filtered_rows: Option<i32>,
745 #[serde(rename = "NumberUnselectedRows")]
746 pub number_unselected_rows: Option<i32>,
747 #[serde(rename = "LoadBytes")]
748 pub load_bytes: Option<i64>,
749 #[serde(rename = "LoadTimeMs")]
750 pub load_time_ms: Option<i32>,
751 #[serde(rename = "BeginTxnTimeMs")]
752 pub begin_txn_time_ms: Option<i32>,
753 #[serde(rename = "ReadDataTimeMs")]
754 pub read_data_time_ms: Option<i32>,
755 #[serde(rename = "WriteDataTimeMs")]
756 pub write_data_time_ms: Option<i32>,
757 #[serde(rename = "CommitAndPublishTimeMs")]
758 pub commit_and_publish_time_ms: Option<i32>,
759 #[serde(rename = "StreamLoadPlanTimeMs")]
760 pub stream_load_plan_time_ms: Option<i32>,
761 #[serde(rename = "ExistingJobStatus")]
762 pub existing_job_status: Option<String>,
763 #[serde(rename = "ErrorURL")]
764 pub error_url: Option<String>,
765}
766
767pub struct StarrocksClient {
768 insert: InserterInner,
769}
770impl StarrocksClient {
771 pub fn new(insert: InserterInner) -> Self {
772 Self { insert }
773 }
774
775 pub async fn write(&mut self, data: Bytes) -> Result<()> {
776 self.insert.write(data).await?;
777 Ok(())
778 }
779
780 pub async fn finish(self) -> Result<StarrocksInsertResultResponse> {
781 let raw = self.insert.finish().await?;
782 let res: StarrocksInsertResultResponse = serde_json::from_slice(&raw)
783 .map_err(|err| SinkError::DorisStarrocksConnect(anyhow!(err)))?;
784
785 if !STARROCKS_SUCCESS_STATUS.contains(&res.status.as_str()) {
786 return Err(SinkError::DorisStarrocksConnect(anyhow::anyhow!(
787 "Insert error: {}, {}, {:?}",
788 res.status,
789 res.message,
790 res.error_url,
791 )));
792 };
793 Ok(res)
794 }
795}
796
797pub struct StarrocksTxnClient {
798 request_builder: StarrocksTxnRequestBuilder,
799}
800
801impl StarrocksTxnClient {
802 pub fn new(request_builder: StarrocksTxnRequestBuilder) -> Self {
803 Self { request_builder }
804 }
805
806 fn check_response_and_extract_label(&self, res: Bytes) -> Result<String> {
807 let res: StarrocksInsertResultResponse = serde_json::from_slice(&res)
808 .map_err(|err| SinkError::DorisStarrocksConnect(anyhow!(err)))?;
809 if !STARROCKS_SUCCESS_STATUS.contains(&res.status.as_str()) {
810 return Err(SinkError::DorisStarrocksConnect(anyhow::anyhow!(
811 "transaction error: {}, {}, {:?}",
812 res.status,
813 res.message,
814 res.error_url,
815 )));
816 }
817 res.label.ok_or_else(|| {
818 SinkError::DorisStarrocksConnect(anyhow::anyhow!("Can't get label from response"))
819 })
820 }
821
822 pub async fn begin(&self, label: String) -> Result<String> {
823 let res = self
824 .request_builder
825 .build_begin_request_sender(label)?
826 .send()
827 .await?;
828 self.check_response_and_extract_label(res)
829 }
830
831 pub async fn prepare(&self, label: String) -> Result<String> {
832 let res = self
833 .request_builder
834 .build_prepare_request_sender(label)?
835 .send()
836 .await?;
837 self.check_response_and_extract_label(res)
838 }
839
840 pub async fn commit(&self, label: String) -> Result<String> {
841 let res = self
842 .request_builder
843 .build_commit_request_sender(label)?
844 .send()
845 .await?;
846 self.check_response_and_extract_label(res)
847 }
848
849 pub async fn rollback(&self, label: String) -> Result<String> {
850 let res = self
851 .request_builder
852 .build_rollback_request_sender(label)?
853 .send()
854 .await?;
855 self.check_response_and_extract_label(res)
856 }
857
858 pub async fn load(&self, label: String) -> Result<InserterInner> {
859 self.request_builder.build_txn_inserter(label).await
860 }
861}