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