From fd7e485a9af902c3b18f49e76e87445bdd79f447 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 26 Sep 2023 14:58:04 +0800 Subject: [PATCH 01/31] chore: bump risingwave version in docker compose to 1.2.0 (#12530) Signed-off-by: Bugen Zhao --- docker/docker-compose.yml | 8 ++++---- docs/memory-profiling.md | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index c7f2db4cb737d..328ebe9b7bacb 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -39,7 +39,7 @@ services: timeout: 5s retries: 5 compute-node-0: - image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.0.0}" + image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}" command: - compute-node - "--listen-addr" @@ -126,7 +126,7 @@ services: timeout: 5s retries: 5 frontend-node-0: - image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.0.0}" + image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}" command: - frontend-node - "--listen-addr" @@ -185,7 +185,7 @@ services: timeout: 5s retries: 5 meta-node-0: - image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.0.0}" + image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}" command: - meta-node - "--listen-addr" @@ -301,7 +301,7 @@ services: timeout: 5s retries: 5 connector-node: - image: ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.0.0} + image: ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0} entrypoint: "/risingwave/bin/connector-node/start-service.sh" ports: - 50051 diff --git a/docs/memory-profiling.md b/docs/memory-profiling.md index fec324770f0c8..759807ba1e9ed 100644 --- a/docs/memory-profiling.md +++ b/docs/memory-profiling.md @@ -158,7 +158,7 @@ cp ./target/release/examples/addr2line Find a Linux machine and use `docker` command to start an environment with the specific RisingWave version. Here, `-v $(pwd):/dumps` mounts current directory to `/dumps` folder inside the container, so that you don't need to copy the files in and out. ```bash -docker run -it --rm --entrypoint /bin/bash -v $(pwd):/dumps ghcr.io/risingwavelabs/risingwave:v1.0.0 +docker run -it --rm --entrypoint /bin/bash -v $(pwd):/dumps ghcr.io/risingwavelabs/risingwave:latest ``` From 7e700e700f19891fd588082e612e8c58d55c41ea Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 26 Sep 2023 15:04:02 +0800 Subject: [PATCH 02/31] refactor(sink): decouple sink formatter and payload writer (#12515) --- src/connector/src/sink/doris.rs | 26 +-- src/connector/src/sink/encoder/json.rs | 23 +-- .../src/sink/formatter/debezium_json.rs | 51 +++--- src/connector/src/sink/formatter/mod.rs | 71 +++++++- src/connector/src/sink/kafka.rs | 166 +++++------------- src/connector/src/sink/kinesis.rs | 96 +++------- src/connector/src/sink/mod.rs | 2 +- src/connector/src/sink/nats.rs | 5 +- src/connector/src/sink/pulsar.rs | 85 ++++----- src/connector/src/sink/remote.rs | 7 +- src/connector/src/sink/utils.rs | 30 +--- 11 files changed, 248 insertions(+), 314 deletions(-) diff --git a/src/connector/src/sink/doris.rs b/src/connector/src/sink/doris.rs index c19365fcc51c4..82e6454663f48 100644 --- a/src/connector/src/sink/doris.rs +++ b/src/connector/src/sink/doris.rs @@ -26,9 +26,9 @@ use serde_json::Value; use serde_with::serde_as; use super::doris_connector::{DorisField, DorisInsert, DorisInsertClient, DORIS_DELETE_SIGN}; -use super::utils::doris_rows_to_json; use super::{SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; use crate::common::DorisCommon; +use crate::sink::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkWriter, SinkWriterParam}; pub const DORIS_SINK: &str = "doris"; @@ -192,7 +192,7 @@ pub struct DorisSinkWriter { client: DorisInsertClient, is_append_only: bool, insert: Option, - decimal_map: HashMap, + row_encoder: JsonEncoder, } impl DorisSinkWriter { @@ -232,12 +232,17 @@ impl DorisSinkWriter { let insert = Some(doris_insert_client.build().await?); Ok(Self { config, - schema, + schema: schema.clone(), pk_indices, client: doris_insert_client, is_append_only, insert, - decimal_map, + row_encoder: JsonEncoder::new_with_doris( + schema, + None, + TimestampHandlingMode::String, + decimal_map, + ), }) } @@ -246,9 +251,7 @@ impl DorisSinkWriter { if op != Op::Insert { continue; } - let row_json_string = - Value::Object(doris_rows_to_json(row, &self.schema, &self.decimal_map)?) - .to_string(); + let row_json_string = Value::Object(self.row_encoder.encode(row)?).to_string(); self.insert .as_mut() .ok_or_else(|| SinkError::Doris("Can't find doris sink insert".to_string()))? @@ -262,8 +265,7 @@ impl DorisSinkWriter { for (op, row) in chunk.rows() { match op { Op::Insert => { - let mut row_json_value = - doris_rows_to_json(row, &self.schema, &self.decimal_map)?; + let mut row_json_value = self.row_encoder.encode(row)?; row_json_value.insert( DORIS_DELETE_SIGN.to_string(), Value::String("0".to_string()), @@ -279,8 +281,7 @@ impl DorisSinkWriter { .await?; } Op::Delete => { - let mut row_json_value = - doris_rows_to_json(row, &self.schema, &self.decimal_map)?; + let mut row_json_value = self.row_encoder.encode(row)?; row_json_value.insert( DORIS_DELETE_SIGN.to_string(), Value::String("1".to_string()), @@ -297,8 +298,7 @@ impl DorisSinkWriter { } Op::UpdateDelete => {} Op::UpdateInsert => { - let mut row_json_value = - doris_rows_to_json(row, &self.schema, &self.decimal_map)?; + let mut row_json_value = self.row_encoder.encode(row)?; row_json_value.insert( DORIS_DELETE_SIGN.to_string(), Value::String("0".to_string()), diff --git a/src/connector/src/sink/encoder/json.rs b/src/connector/src/sink/encoder/json.rs index 9264df0d12a97..6add09b2cb86e 100644 --- a/src/connector/src/sink/encoder/json.rs +++ b/src/connector/src/sink/encoder/json.rs @@ -27,17 +27,18 @@ use serde_json::{json, Map, Value}; use super::{CustomJsonType, Result, RowEncoder, SerTo, TimestampHandlingMode}; use crate::sink::SinkError; -pub struct JsonEncoder<'a> { - schema: &'a Schema, - col_indices: Option<&'a [usize]>, + +pub struct JsonEncoder { + schema: Schema, + col_indices: Option>, timestamp_handling_mode: TimestampHandlingMode, custom_json_type: CustomJsonType, } -impl<'a> JsonEncoder<'a> { +impl JsonEncoder { pub fn new( - schema: &'a Schema, - col_indices: Option<&'a [usize]>, + schema: Schema, + col_indices: Option>, timestamp_handling_mode: TimestampHandlingMode, ) -> Self { Self { @@ -49,8 +50,8 @@ impl<'a> JsonEncoder<'a> { } pub fn new_with_doris( - schema: &'a Schema, - col_indices: Option<&'a [usize]>, + schema: Schema, + col_indices: Option>, timestamp_handling_mode: TimestampHandlingMode, map: HashMap, ) -> Self { @@ -63,15 +64,15 @@ impl<'a> JsonEncoder<'a> { } } -impl<'a> RowEncoder for JsonEncoder<'a> { +impl RowEncoder for JsonEncoder { type Output = Map; fn schema(&self) -> &Schema { - self.schema + &self.schema } fn col_indices(&self) -> Option<&[usize]> { - self.col_indices + self.col_indices.as_ref().map(Vec::as_ref) } fn encode_cols( diff --git a/src/connector/src/sink/formatter/debezium_json.rs b/src/connector/src/sink/formatter/debezium_json.rs index f5b48e836c64d..f002d17e75dfe 100644 --- a/src/connector/src/sink/formatter/debezium_json.rs +++ b/src/connector/src/sink/formatter/debezium_json.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::time::{SystemTime, UNIX_EPOCH}; + use risingwave_common::array::Op; use risingwave_common::catalog::{Field, Schema}; use serde_json::{json, Map, Value}; @@ -39,36 +41,43 @@ fn concat_debezium_name_field(db_name: &str, sink_from_name: &str, value: &str) DEBEZIUM_NAME_FIELD_PREFIX.to_owned() + "." + db_name + "." + sink_from_name + "." + value } -pub struct DebeziumJsonFormatter<'a> { - schema: &'a Schema, - pk_indices: &'a [usize], - db_name: &'a str, - sink_from_name: &'a str, +pub struct DebeziumJsonFormatter { + schema: Schema, + pk_indices: Vec, + db_name: String, + sink_from_name: String, opts: DebeziumAdapterOpts, - ts_ms: u64, + key_encoder: JsonEncoder, + val_encoder: JsonEncoder, } -impl<'a> DebeziumJsonFormatter<'a> { +impl DebeziumJsonFormatter { pub fn new( - schema: &'a Schema, - pk_indices: &'a [usize], - db_name: &'a str, - sink_from_name: &'a str, + schema: Schema, + pk_indices: Vec, + db_name: String, + sink_from_name: String, opts: DebeziumAdapterOpts, - ts_ms: u64, ) -> Self { + let key_encoder = JsonEncoder::new( + schema.clone(), + Some(pk_indices.clone()), + TimestampHandlingMode::Milli, + ); + let val_encoder = JsonEncoder::new(schema.clone(), None, TimestampHandlingMode::Milli); Self { schema, pk_indices, db_name, sink_from_name, opts, - ts_ms, + key_encoder, + val_encoder, } } } -impl<'a> SinkFormatter for DebeziumJsonFormatter<'a> { +impl SinkFormatter for DebeziumJsonFormatter { type K = Value; type V = Value; @@ -83,8 +92,13 @@ impl<'a> SinkFormatter for DebeziumJsonFormatter<'a> { db_name, sink_from_name, opts, - ts_ms, + key_encoder, + val_encoder, } = self; + let ts_ms = SystemTime::now() + .duration_since(UNIX_EPOCH) + .unwrap() + .as_millis() as u64; let source_field = json!({ "db": db_name, "table": sink_from_name, @@ -92,10 +106,6 @@ impl<'a> SinkFormatter for DebeziumJsonFormatter<'a> { let mut update_cache: Option> = None; - let key_encoder = - JsonEncoder::new(schema, Some(pk_indices), TimestampHandlingMode::Milli); - let val_encoder = JsonEncoder::new(schema, None, TimestampHandlingMode::Milli); - for (op, row) in chunk.rows() { let event_key_object: Option = Some(json!({ "schema": json!({ @@ -338,7 +348,8 @@ mod tests { }, ]); - let json_chunk = chunk_to_json(chunk, &schema).unwrap(); + let encoder = JsonEncoder::new(schema.clone(), None, TimestampHandlingMode::Milli); + let json_chunk = chunk_to_json(chunk, &encoder).unwrap(); let schema_json = schema_to_json(&schema, "test_db", "test_table"); assert_eq!(schema_json, serde_json::from_str::("{\"fields\":[{\"field\":\"before\",\"fields\":[{\"field\":\"v1\",\"optional\":true,\"type\":\"int32\"},{\"field\":\"v2\",\"optional\":true,\"type\":\"float\"},{\"field\":\"v3\",\"optional\":true,\"type\":\"string\"}],\"name\":\"RisingWave.test_db.test_table.Key\",\"optional\":true,\"type\":\"struct\"},{\"field\":\"after\",\"fields\":[{\"field\":\"v1\",\"optional\":true,\"type\":\"int32\"},{\"field\":\"v2\",\"optional\":true,\"type\":\"float\"},{\"field\":\"v3\",\"optional\":true,\"type\":\"string\"}],\"name\":\"RisingWave.test_db.test_table.Key\",\"optional\":true,\"type\":\"struct\"},{\"field\":\"source\",\"fields\":[{\"field\":\"db\",\"optional\":false,\"type\":\"string\"},{\"field\":\"table\",\"optional\":true,\"type\":\"string\"}],\"name\":\"RisingWave.test_db.test_table.Source\",\"optional\":false,\"type\":\"struct\"},{\"field\":\"op\",\"optional\":false,\"type\":\"string\"},{\"field\":\"ts_ms\",\"optional\":false,\"type\":\"int64\"}],\"name\":\"RisingWave.test_db.test_table.Envelope\",\"optional\":false,\"type\":\"struct\"}").unwrap()); assert_eq!( diff --git a/src/connector/src/sink/formatter/mod.rs b/src/connector/src/sink/formatter/mod.rs index 585752b03327c..89e8369d93819 100644 --- a/src/connector/src/sink/formatter/mod.rs +++ b/src/connector/src/sink/formatter/mod.rs @@ -12,9 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use anyhow::anyhow; use risingwave_common::array::StreamChunk; -use crate::sink::Result; +use crate::sink::{Result, SinkError, SINK_TYPE_DEBEZIUM, SINK_TYPE_UPSERT}; mod append_only; mod debezium_json; @@ -22,8 +23,11 @@ mod upsert; pub use append_only::AppendOnlyFormatter; pub use debezium_json::{DebeziumAdapterOpts, DebeziumJsonFormatter}; +use risingwave_common::catalog::Schema; pub use upsert::UpsertFormatter; +use crate::sink::encoder::{JsonEncoder, TimestampHandlingMode}; + /// Transforms a `StreamChunk` into a sequence of key-value pairs according a specific format, /// for example append-only, upsert or debezium. pub trait SinkFormatter { @@ -50,3 +54,68 @@ macro_rules! tri { } }; } + +#[expect(clippy::enum_variant_names)] +pub enum SinkFormatterImpl { + AppendOnlyJson(AppendOnlyFormatter), + UpsertJson(UpsertFormatter), + DebeziumJson(DebeziumJsonFormatter), +} + +impl SinkFormatterImpl { + pub fn new( + formatter_type: &str, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + db_name: String, + sink_from_name: String, + ) -> Result { + if is_append_only { + let key_encoder = JsonEncoder::new( + schema.clone(), + Some(pk_indices), + TimestampHandlingMode::Milli, + ); + let val_encoder = JsonEncoder::new(schema, None, TimestampHandlingMode::Milli); + + let formatter = AppendOnlyFormatter::new(key_encoder, val_encoder); + Ok(SinkFormatterImpl::AppendOnlyJson(formatter)) + } else if formatter_type == SINK_TYPE_DEBEZIUM { + Ok(SinkFormatterImpl::DebeziumJson(DebeziumJsonFormatter::new( + schema, + pk_indices, + db_name, + sink_from_name, + DebeziumAdapterOpts::default(), + ))) + } else if formatter_type == SINK_TYPE_UPSERT { + let key_encoder = JsonEncoder::new( + schema.clone(), + Some(pk_indices), + TimestampHandlingMode::Milli, + ); + let val_encoder = JsonEncoder::new(schema, None, TimestampHandlingMode::Milli); + + // Initialize the upsert_stream + let formatter = UpsertFormatter::new(key_encoder, val_encoder); + Ok(SinkFormatterImpl::UpsertJson(formatter)) + } else { + Err(SinkError::Config(anyhow!( + "unsupported upsert sink type {}", + formatter_type + ))) + } + } +} + +#[macro_export] +macro_rules! dispatch_sink_formatter_impl { + ($impl:expr, $name:ident, $body:expr) => { + match $impl { + SinkFormatterImpl::AppendOnlyJson($name) => $body, + SinkFormatterImpl::UpsertJson($name) => $body, + SinkFormatterImpl::DebeziumJson($name) => $body, + } + }; +} diff --git a/src/connector/src/sink/kafka.rs b/src/connector/src/sink/kafka.rs index 4ae886d15ce0c..85894792cd7ee 100644 --- a/src/connector/src/sink/kafka.rs +++ b/src/connector/src/sink/kafka.rs @@ -15,7 +15,7 @@ use std::collections::{HashMap, VecDeque}; use std::fmt::Debug; use std::sync::Arc; -use std::time::{Duration, SystemTime, UNIX_EPOCH}; +use std::time::Duration; use anyhow::anyhow; use futures::future::try_join_all; @@ -31,15 +31,12 @@ use serde_derive::{Deserialize, Serialize}; use serde_with::{serde_as, DisplayFromStr}; use strum_macros::{Display, EnumString}; -use super::encoder::{JsonEncoder, TimestampHandlingMode}; -use super::formatter::{ - AppendOnlyFormatter, DebeziumAdapterOpts, DebeziumJsonFormatter, UpsertFormatter, -}; use super::{ FormattedSink, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_DEBEZIUM, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; use crate::common::KafkaCommon; +use crate::sink::formatter::SinkFormatterImpl; use crate::sink::{ DummySinkCommitCoordinator, Result, SinkWriterParam, SinkWriterV1, SinkWriterV1Adapter, }; @@ -47,6 +44,7 @@ use crate::source::kafka::{KafkaProperties, KafkaSplitEnumerator, PrivateLinkPro use crate::source::{SourceEnumeratorContext, SplitEnumerator}; use crate::{ deserialize_bool_from_string, deserialize_duration_from_string, deserialize_u32_from_string, + dispatch_sink_formatter_impl, }; pub const KAFKA_SINK: &str = "kafka"; @@ -301,16 +299,18 @@ impl Sink for KafkaSink { type Coordinator = DummySinkCommitCoordinator; type Writer = SinkWriterV1Adapter; - async fn new_writer(&self, writer_param: SinkWriterParam) -> Result { + async fn new_writer(&self, _writer_param: SinkWriterParam) -> Result { Ok(SinkWriterV1Adapter::new( KafkaSinkWriter::new( self.config.clone(), - self.schema.clone(), - self.pk_indices.clone(), - self.is_append_only, - self.db_name.clone(), - self.sink_from_name.clone(), - format!("sink-{:?}", writer_param.executor_id), + SinkFormatterImpl::new( + &self.config.r#type, + self.schema.clone(), + self.pk_indices.clone(), + self.is_append_only, + self.db_name.clone(), + self.sink_from_name.clone(), + )?, ) .await?, )) @@ -338,41 +338,25 @@ impl Sink for KafkaSink { } } -#[derive(Debug, Clone, PartialEq, enum_as_inner::EnumAsInner)] -enum KafkaSinkState { - Init, - // State running with epoch. - Running(u64), -} - /// The delivery buffer queue size /// When the `DeliveryFuture` the current `future_delivery_buffer` /// is buffering is greater than this size, then enforcing commit once const KAFKA_WRITER_MAX_QUEUE_SIZE: usize = 65536; +struct KafkaPayloadWriter { + inner: FutureProducer, + future_delivery_buffer: VecDeque, + config: KafkaConfig, +} + pub struct KafkaSinkWriter { pub config: KafkaConfig, - pub inner: FutureProducer, - identifier: String, - state: KafkaSinkState, - schema: Schema, - pk_indices: Vec, - is_append_only: bool, - future_delivery_buffer: VecDeque, - db_name: String, - sink_from_name: String, + payload_writer: KafkaPayloadWriter, + formatter: SinkFormatterImpl, } impl KafkaSinkWriter { - pub async fn new( - mut config: KafkaConfig, - schema: Schema, - pk_indices: Vec, - is_append_only: bool, - db_name: String, - sink_from_name: String, - identifier: String, - ) -> Result { + pub async fn new(mut config: KafkaConfig, formatter: SinkFormatterImpl) -> Result { let inner: FutureProducer = { let mut c = ClientConfig::new(); @@ -400,18 +384,17 @@ impl KafkaSinkWriter { Ok(KafkaSinkWriter { config: config.clone(), - inner, - identifier, - state: KafkaSinkState::Init, - schema, - pk_indices, - is_append_only, - future_delivery_buffer: VecDeque::new(), - db_name, - sink_from_name, + payload_writer: KafkaPayloadWriter { + inner, + future_delivery_buffer: VecDeque::new(), + config, + }, + formatter, }) } +} +impl KafkaPayloadWriter { /// The actual `send_result` function, will be called when the `KafkaSinkWriter` needs to sink /// messages async fn send_result<'a, K, P>( @@ -537,57 +520,9 @@ impl KafkaSinkWriter { Ok(()) } - - async fn debezium_update(&mut self, chunk: StreamChunk, ts_ms: u64) -> Result<()> { - // TODO: Remove the clones here, only to satisfy borrow checker at present - let schema = self.schema.clone(); - let pk_indices = self.pk_indices.clone(); - let db_name = self.db_name.clone(); - let sink_from_name = self.sink_from_name.clone(); - - // Initialize the dbz_stream - let f = DebeziumJsonFormatter::new( - &schema, - &pk_indices, - &db_name, - &sink_from_name, - DebeziumAdapterOpts::default(), - ts_ms, - ); - - self.write_chunk(chunk, f).await - } - - async fn upsert(&mut self, chunk: StreamChunk) -> Result<()> { - // TODO: Remove the clones here, only to satisfy borrow checker at present - let schema = self.schema.clone(); - let pk_indices = self.pk_indices.clone(); - let key_encoder = - JsonEncoder::new(&schema, Some(&pk_indices), TimestampHandlingMode::Milli); - let val_encoder = JsonEncoder::new(&schema, None, TimestampHandlingMode::Milli); - - // Initialize the upsert_stream - let f = UpsertFormatter::new(key_encoder, val_encoder); - - self.write_chunk(chunk, f).await - } - - async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { - // TODO: Remove the clones here, only to satisfy borrow checker at present - let schema = self.schema.clone(); - let pk_indices = self.pk_indices.clone(); - let key_encoder = - JsonEncoder::new(&schema, Some(&pk_indices), TimestampHandlingMode::Milli); - let val_encoder = JsonEncoder::new(&schema, None, TimestampHandlingMode::Milli); - - // Initialize the append_only_stream - let f = AppendOnlyFormatter::new(key_encoder, val_encoder); - - self.write_chunk(chunk, f).await - } } -impl FormattedSink for KafkaSinkWriter { +impl FormattedSink for KafkaPayloadWriter { type K = Vec; type V = Vec; @@ -599,25 +534,9 @@ impl FormattedSink for KafkaSinkWriter { #[async_trait::async_trait] impl SinkWriterV1 for KafkaSinkWriter { async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { - if self.is_append_only { - // Append-only - self.append_only(chunk).await - } else { - // Debezium - if self.config.r#type == SINK_TYPE_DEBEZIUM { - self.debezium_update( - chunk, - SystemTime::now() - .duration_since(UNIX_EPOCH) - .unwrap() - .as_millis() as u64, - ) - .await - } else { - // Upsert - self.upsert(chunk).await - } - } + dispatch_sink_formatter_impl!(&self.formatter, formatter, { + self.payload_writer.write_chunk(chunk, formatter).await + }) } /// --------------------------------------------------------------------------------------- @@ -632,7 +551,7 @@ impl SinkWriterV1 for KafkaSinkWriter { async fn commit(&mut self) -> Result<()> { // Group delivery (await the `FutureRecord`) here - self.commit_inner().await?; + self.payload_writer.commit_inner().await?; Ok(()) } @@ -648,6 +567,8 @@ mod test { use risingwave_common::types::DataType; use super::*; + use crate::sink::encoder::{JsonEncoder, TimestampHandlingMode}; + use crate::sink::formatter::AppendOnlyFormatter; #[test] fn parse_rdkafka_props() { @@ -822,12 +743,14 @@ mod test { // Create the actual sink writer to Kafka let mut sink = KafkaSinkWriter::new( kafka_config.clone(), - schema, - pk_indices, - true, - "test_sink_1".to_string(), - "test_db".into(), - "test_table".into(), + SinkFormatterImpl::AppendOnlyJson(AppendOnlyFormatter::new( + JsonEncoder::new( + schema.clone(), + Some(pk_indices), + TimestampHandlingMode::Milli, + ), + JsonEncoder::new(schema, None, TimestampHandlingMode::Milli), + )), ) .await .unwrap(); @@ -838,6 +761,7 @@ mod test { println!("epoch: {}", i); for j in 0..100 { match sink + .payload_writer .send_result( FutureRecord::to(kafka_config.common.topic.as_str()) .payload(format!("value-{}", j).as_bytes()) diff --git a/src/connector/src/sink/kinesis.rs b/src/connector/src/sink/kinesis.rs index 7c43abcbe82a7..114070ca1e440 100644 --- a/src/connector/src/sink/kinesis.rs +++ b/src/connector/src/sink/kinesis.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::collections::HashMap; -use std::time::{SystemTime, UNIX_EPOCH}; use anyhow::anyhow; use aws_sdk_kinesis::error::DisplayErrorContext; @@ -27,12 +26,10 @@ use serde_with::serde_as; use tokio_retry::strategy::{jitter, ExponentialBackoff}; use tokio_retry::Retry; -use super::formatter::{ - AppendOnlyFormatter, DebeziumAdapterOpts, DebeziumJsonFormatter, UpsertFormatter, -}; use super::{FormattedSink, SinkParam}; use crate::common::KinesisCommon; -use crate::sink::encoder::{JsonEncoder, TimestampHandlingMode}; +use crate::dispatch_sink_formatter_impl; +use crate::sink::formatter::SinkFormatterImpl; use crate::sink::{ DummySinkCommitCoordinator, Result, Sink, SinkError, SinkWriter, SinkWriterParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_DEBEZIUM, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, @@ -134,15 +131,15 @@ impl KinesisSinkConfig { } } -#[derive(Debug)] pub struct KinesisSinkWriter { pub config: KinesisSinkConfig, - schema: Schema, - pk_indices: Vec, + formatter: SinkFormatterImpl, + payload_writer: KinesisSinkPayloadWriter, +} + +struct KinesisSinkPayloadWriter { client: KinesisClient, - is_append_only: bool, - db_name: String, - sink_from_name: String, + config: KinesisSinkConfig, } impl KinesisSinkWriter { @@ -154,22 +151,27 @@ impl KinesisSinkWriter { db_name: String, sink_from_name: String, ) -> Result { + let formatter = SinkFormatterImpl::new( + &config.r#type, + schema, + pk_indices, + is_append_only, + db_name, + sink_from_name, + )?; let client = config .common .build_client() .await .map_err(SinkError::Kinesis)?; Ok(Self { - config, - schema, - pk_indices, - client, - is_append_only, - db_name, - sink_from_name, + config: config.clone(), + formatter, + payload_writer: KinesisSinkPayloadWriter { client, config }, }) } - +} +impl KinesisSinkPayloadWriter { async fn put_record(&self, key: &str, payload: Vec) -> Result { let payload = Blob::new(payload); // todo: switch to put_records() for batching @@ -199,44 +201,9 @@ impl KinesisSinkWriter { )) }) } - - async fn upsert(mut self: &Self, chunk: StreamChunk) -> Result<()> { - let key_encoder = JsonEncoder::new( - &self.schema, - Some(&self.pk_indices), - TimestampHandlingMode::Milli, - ); - let val_encoder = JsonEncoder::new(&self.schema, None, TimestampHandlingMode::Milli); - let f = UpsertFormatter::new(key_encoder, val_encoder); - self.write_chunk(chunk, f).await - } - - async fn append_only(mut self: &Self, chunk: StreamChunk) -> Result<()> { - let key_encoder = JsonEncoder::new( - &self.schema, - Some(&self.pk_indices), - TimestampHandlingMode::Milli, - ); - let val_encoder = JsonEncoder::new(&self.schema, None, TimestampHandlingMode::Milli); - let f = AppendOnlyFormatter::new(key_encoder, val_encoder); - self.write_chunk(chunk, f).await - } - - async fn debezium_update(mut self: &Self, chunk: StreamChunk, ts_ms: u64) -> Result<()> { - let f = DebeziumJsonFormatter::new( - &self.schema, - &self.pk_indices, - &self.db_name, - &self.sink_from_name, - DebeziumAdapterOpts::default(), - ts_ms, - ); - - self.write_chunk(chunk, f).await - } } -impl FormattedSink for &KinesisSinkWriter { +impl FormattedSink for KinesisSinkPayloadWriter { type K = String; type V = Vec; @@ -250,22 +217,9 @@ impl FormattedSink for &KinesisSinkWriter { #[async_trait::async_trait] impl SinkWriter for KinesisSinkWriter { async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { - if self.is_append_only { - self.append_only(chunk).await - } else if self.config.r#type == SINK_TYPE_DEBEZIUM { - self.debezium_update( - chunk, - SystemTime::now() - .duration_since(UNIX_EPOCH) - .unwrap() - .as_millis() as u64, - ) - .await - } else if self.config.r#type == SINK_TYPE_UPSERT { - self.upsert(chunk).await - } else { - unreachable!() - } + dispatch_sink_formatter_impl!(&self.formatter, formatter, { + self.payload_writer.write_chunk(chunk, formatter).await + }) } async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index f1f12ed323d7b..449f4f541f0e4 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -230,7 +230,7 @@ pub trait FormattedSink { async fn write_chunk( &mut self, chunk: StreamChunk, - formatter: F, + formatter: &F, ) -> Result<()> where F::K: SerTo, diff --git a/src/connector/src/sink/nats.rs b/src/connector/src/sink/nats.rs index e04235b6f586e..57b9e572c0615 100644 --- a/src/connector/src/sink/nats.rs +++ b/src/connector/src/sink/nats.rs @@ -27,6 +27,7 @@ use tokio_retry::Retry; use super::utils::chunk_to_json; use super::{DummySinkCommitCoordinator, SinkWriter, SinkWriterParam}; use crate::common::NatsCommon; +use crate::sink::encoder::{JsonEncoder, TimestampHandlingMode}; use crate::sink::{Result, Sink, SinkError, SINK_TYPE_APPEND_ONLY}; pub const NATS_SINK: &str = "nats"; @@ -52,6 +53,7 @@ pub struct NatsSinkWriter { pub config: NatsConfig, context: Context, schema: Schema, + json_encoder: JsonEncoder, } /// Basic data types for use with the nats interface @@ -118,6 +120,7 @@ impl NatsSinkWriter { config: config.clone(), context, schema: schema.clone(), + json_encoder: JsonEncoder::new(schema, None, TimestampHandlingMode::Milli), }) } @@ -125,7 +128,7 @@ impl NatsSinkWriter { Retry::spawn( ExponentialBackoff::from_millis(100).map(jitter).take(3), || async { - let data = chunk_to_json(chunk.clone(), &self.schema).unwrap(); + let data = chunk_to_json(chunk.clone(), &self.json_encoder).unwrap(); for item in data { self.context .publish(self.config.common.subject.clone(), item.into()) diff --git a/src/connector/src/sink/pulsar.rs b/src/connector/src/sink/pulsar.rs index d1cbc0b5a527c..8dc478f7dd82d 100644 --- a/src/connector/src/sink/pulsar.rs +++ b/src/connector/src/sink/pulsar.rs @@ -27,15 +27,14 @@ use risingwave_common::catalog::Schema; use serde::Deserialize; use serde_with::{serde_as, DisplayFromStr}; -use super::encoder::{JsonEncoder, TimestampHandlingMode}; -use super::formatter::{AppendOnlyFormatter, UpsertFormatter}; use super::{ FormattedSink, Sink, SinkError, SinkParam, SinkWriter, SinkWriterParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; use crate::common::PulsarCommon; -use crate::deserialize_duration_from_string; +use crate::sink::formatter::SinkFormatterImpl; use crate::sink::{DummySinkCommitCoordinator, Result}; +use crate::{deserialize_duration_from_string, dispatch_sink_formatter_impl}; pub const PULSAR_SINK: &str = "pulsar"; @@ -142,6 +141,8 @@ pub struct PulsarSink { schema: Schema, downstream_pk: Vec, is_append_only: bool, + db_name: String, + sink_from_name: String, } impl PulsarSink { @@ -151,6 +152,8 @@ impl PulsarSink { schema: param.schema(), downstream_pk: param.downstream_pk, is_append_only: param.sink_type.is_append_only(), + db_name: param.db_name, + sink_from_name: param.sink_from_name, } } } @@ -166,6 +169,8 @@ impl Sink for PulsarSink { self.schema.clone(), self.downstream_pk.clone(), self.is_append_only, + self.db_name.clone(), + self.sink_from_name.clone(), ) .await } @@ -188,12 +193,14 @@ impl Sink for PulsarSink { } pub struct PulsarSinkWriter { + payload_writer: PulsarPayloadWriter, + formatter: SinkFormatterImpl, +} + +struct PulsarPayloadWriter { pulsar: Pulsar, producer: Producer, config: PulsarConfig, - schema: Schema, - downstream_pk: Vec, - is_append_only: bool, send_future_buffer: VecDeque, } @@ -203,20 +210,32 @@ impl PulsarSinkWriter { schema: Schema, downstream_pk: Vec, is_append_only: bool, + db_name: String, + sink_from_name: String, ) -> Result { - let pulsar = config.common.build_client().await?; - let producer = build_pulsar_producer(&pulsar, &config).await?; - Ok(Self { - pulsar, - producer, - config, + let formatter = SinkFormatterImpl::new( + &config.r#type, schema, downstream_pk, is_append_only, - send_future_buffer: VecDeque::new(), + db_name, + sink_from_name, + )?; + let pulsar = config.common.build_client().await?; + let producer = build_pulsar_producer(&pulsar, &config).await?; + Ok(Self { + formatter, + payload_writer: PulsarPayloadWriter { + pulsar, + producer, + config, + send_future_buffer: VecDeque::new(), + }, }) } +} +impl PulsarPayloadWriter { async fn send_message(&mut self, message: Message) -> Result<()> { let mut success_flag = false; let mut connection_err = None; @@ -276,34 +295,6 @@ impl PulsarSinkWriter { Ok(()) } - async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { - // TODO: Remove the clones here, only to satisfy borrow checker at present - let schema = self.schema.clone(); - let downstream_pk = self.downstream_pk.clone(); - let key_encoder = - JsonEncoder::new(&schema, Some(&downstream_pk), TimestampHandlingMode::Milli); - let val_encoder = JsonEncoder::new(&schema, None, TimestampHandlingMode::Milli); - - // Initialize the append_only_stream - let f = AppendOnlyFormatter::new(key_encoder, val_encoder); - - self.write_chunk(chunk, f).await - } - - async fn upsert(&mut self, chunk: StreamChunk) -> Result<()> { - // TODO: Remove the clones here, only to satisfy borrow checker at present - let schema = self.schema.clone(); - let downstream_pk = self.downstream_pk.clone(); - let key_encoder = - JsonEncoder::new(&schema, Some(&downstream_pk), TimestampHandlingMode::Milli); - let val_encoder = JsonEncoder::new(&schema, None, TimestampHandlingMode::Milli); - - // Initialize the upsert_stream - let f = UpsertFormatter::new(key_encoder, val_encoder); - - self.write_chunk(chunk, f).await - } - async fn commit_inner(&mut self) -> Result<()> { self.producer .send_batch() @@ -320,7 +311,7 @@ impl PulsarSinkWriter { } } -impl FormattedSink for PulsarSinkWriter { +impl FormattedSink for PulsarPayloadWriter { type K = String; type V = Vec; @@ -332,11 +323,9 @@ impl FormattedSink for PulsarSinkWriter { #[async_trait] impl SinkWriter for PulsarSinkWriter { async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { - if self.is_append_only { - self.append_only(chunk).await - } else { - self.upsert(chunk).await - } + dispatch_sink_formatter_impl!(&self.formatter, formatter, { + self.payload_writer.write_chunk(chunk, formatter).await + }) } async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { @@ -345,7 +334,7 @@ impl SinkWriter for PulsarSinkWriter { async fn barrier(&mut self, is_checkpoint: bool) -> Result { if is_checkpoint { - self.commit_inner().await?; + self.payload_writer.commit_inner().await?; } Ok(()) diff --git a/src/connector/src/sink/remote.rs b/src/connector/src/sink/remote.rs index 004f8ac46d128..f6fa615e44c53 100644 --- a/src/connector/src/sink/remote.rs +++ b/src/connector/src/sink/remote.rs @@ -245,7 +245,6 @@ impl Sink for CoordinatedRemoteSink { pub type RemoteSinkWriter = RemoteSinkWriterInner<()>; pub type CoordinatedRemoteSinkWriter = RemoteSinkWriterInner>; -#[derive(Debug)] pub struct RemoteSinkWriterInner { pub connector_type: String, properties: HashMap, @@ -254,6 +253,7 @@ pub struct RemoteSinkWriterInner { schema: Schema, payload_format: SinkPayloadFormat, stream_handle: SinkWriterStreamHandle, + json_encoder: JsonEncoder, _phantom: PhantomData, } @@ -291,6 +291,7 @@ impl RemoteSinkWriterInner { schema: param.schema(), stream_handle, payload_format: connector_params.sink_payload_format, + json_encoder: JsonEncoder::new(param.schema(), None, TimestampHandlingMode::String), _phantom: PhantomData, }) } @@ -331,6 +332,7 @@ impl RemoteSinkWriterInner { properties, epoch: None, batch_id: 0, + json_encoder: JsonEncoder::new(schema.clone(), None, TimestampHandlingMode::String), schema, stream_handle, payload_format: SinkPayloadFormat::Json, @@ -387,9 +389,8 @@ where let payload = match self.payload_format { SinkPayloadFormat::Json => { let mut row_ops = Vec::with_capacity(chunk.cardinality()); - let enc = JsonEncoder::new(&self.schema, None, TimestampHandlingMode::String); for (op, row_ref) in chunk.rows() { - let map = enc.encode(row_ref)?; + let map = self.json_encoder.encode(row_ref)?; let row_op = RowOp { op_type: op.to_protobuf() as i32, line: serde_json::to_string(&map) diff --git a/src/connector/src/sink/utils.rs b/src/connector/src/sink/utils.rs index da9e92b6ae0aa..967c3fc43ba30 100644 --- a/src/connector/src/sink/utils.rs +++ b/src/connector/src/sink/utils.rs @@ -1,5 +1,3 @@ -use std::collections::HashMap; - // Copyright 2023 RisingWave Labs // // Licensed under the Apache License, Version 2.0 (the "License"); @@ -13,16 +11,15 @@ use std::collections::HashMap; // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::array::{RowRef, StreamChunk}; -use risingwave_common::catalog::Schema; -use serde_json::{Map, Value}; -use super::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; +use risingwave_common::array::StreamChunk; +use serde_json::Value; + +use super::encoder::{JsonEncoder, RowEncoder}; use crate::sink::Result; -pub fn chunk_to_json(chunk: StreamChunk, schema: &Schema) -> Result> { - let encoder = JsonEncoder::new(schema, None, TimestampHandlingMode::Milli); - let mut records: Vec = Vec::with_capacity(chunk.cardinality()); +pub fn chunk_to_json(chunk: StreamChunk, encoder: &JsonEncoder) -> Result> { + let mut records: Vec = Vec::with_capacity(chunk.capacity()); for (_, row) in chunk.rows() { let record = Value::Object(encoder.encode(row)?); records.push(record.to_string()); @@ -30,18 +27,3 @@ pub fn chunk_to_json(chunk: StreamChunk, schema: &Schema) -> Result> Ok(records) } - -pub fn doris_rows_to_json( - row: RowRef<'_>, - schema: &Schema, - decimal_map: &HashMap, -) -> Result> { - let encoder = JsonEncoder::new_with_doris( - schema, - None, - TimestampHandlingMode::String, - decimal_map.clone(), - ); - let map = encoder.encode(row)?; - Ok(map) -} From 5eeef12e02e5d245a7ff37a7492bdfee2e90e1e7 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 26 Sep 2023 15:07:59 +0800 Subject: [PATCH 03/31] chore(java-binding): refine stream chunk benchmark (#12504) --- .../java/binding/ArrayListBenchmark.java | 15 ++------ .../java/binding/StreamchunkBenchmark.java | 36 ++++++++++++------- src/java_binding/make-java-binding.toml | 9 ++--- 3 files changed, 29 insertions(+), 31 deletions(-) diff --git a/java/java-binding-benchmark/src/main/java/com/risingwave/java/binding/ArrayListBenchmark.java b/java/java-binding-benchmark/src/main/java/com/risingwave/java/binding/ArrayListBenchmark.java index 6540033371d34..c05cf23d2c582 100644 --- a/java/java-binding-benchmark/src/main/java/com/risingwave/java/binding/ArrayListBenchmark.java +++ b/java/java-binding-benchmark/src/main/java/com/risingwave/java/binding/ArrayListBenchmark.java @@ -20,8 +20,8 @@ import java.util.concurrent.TimeUnit; import org.openjdk.jmh.annotations.*; -@Warmup(iterations = 10, time = 1, timeUnit = TimeUnit.MILLISECONDS) -@Measurement(iterations = 20, time = 1, timeUnit = TimeUnit.MILLISECONDS) +@Warmup(iterations = 2, time = 1, timeUnit = TimeUnit.MILLISECONDS, batchSize = 10) +@Measurement(iterations = 5, time = 1, timeUnit = TimeUnit.MILLISECONDS, batchSize = 10) @Fork(value = 1) @BenchmarkMode(org.openjdk.jmh.annotations.Mode.AverageTime) @OutputTimeUnit(TimeUnit.MICROSECONDS) @@ -30,8 +30,6 @@ public class ArrayListBenchmark { @Param({"100", "1000", "10000"}) static int loopTime; - ArrayList> data = new ArrayList<>(); - public ArrayList getRow(int index) { short v1 = (short) index; int v2 = (int) index; @@ -61,17 +59,10 @@ public void getValue(ArrayList rowData) { Integer mayNull = (Integer) rowData.get(6); } - @Setup - public void setup() { - for (int i = 0; i < loopTime; i++) { - data.add(getRow(i)); - } - } - @Benchmark public void arrayListTest() { for (int i = 0; i < loopTime; i++) { - getValue(data.get(i)); + getValue(getRow(i)); } } } diff --git a/java/java-binding-benchmark/src/main/java/com/risingwave/java/binding/StreamchunkBenchmark.java b/java/java-binding-benchmark/src/main/java/com/risingwave/java/binding/StreamchunkBenchmark.java index 8741044f7b34e..628d1405c8d81 100644 --- a/java/java-binding-benchmark/src/main/java/com/risingwave/java/binding/StreamchunkBenchmark.java +++ b/java/java-binding-benchmark/src/main/java/com/risingwave/java/binding/StreamchunkBenchmark.java @@ -16,31 +16,37 @@ package com.risingwave.java.binding; +import java.util.ArrayList; +import java.util.Iterator; import java.util.concurrent.TimeUnit; import org.openjdk.jmh.annotations.*; -@Warmup(iterations = 10, time = 1, timeUnit = TimeUnit.MILLISECONDS) -@Measurement(iterations = 20, time = 1, timeUnit = TimeUnit.MILLISECONDS) +@Warmup(iterations = 2, time = 1, timeUnit = TimeUnit.MILLISECONDS, batchSize = 10) +@Measurement(iterations = 5, time = 1, timeUnit = TimeUnit.MILLISECONDS, batchSize = 10) @Fork(value = 1) @BenchmarkMode(org.openjdk.jmh.annotations.Mode.AverageTime) -@OutputTimeUnit(TimeUnit.MICROSECONDS) +@OutputTimeUnit(TimeUnit.MILLISECONDS) @State(org.openjdk.jmh.annotations.Scope.Benchmark) public class StreamchunkBenchmark { @Param({"100", "1000", "10000"}) - static int loopTime; + int loopTime; - String str; - StreamChunkIterator iter; + Iterator iterOfIter; - @Setup(Level.Invocation) + @Setup(Level.Iteration) public void setup() { - str = "i i I f F B i"; - for (int i = 0; i < loopTime; i++) { - String b = i % 2 == 0 ? "f" : "t"; - String n = i % 2 == 0 ? "." : "1"; - str += String.format("\n + %d %d %d %d.0 %d.0 %s %s", i, i, i, i, i, b, n); + var iterList = new ArrayList(); + for (int iterI = 0; iterI < 10; iterI++) { + String str = "i i I f F B i"; + for (int i = 0; i < loopTime; i++) { + String b = i % 2 == 0 ? "f" : "t"; + String n = i % 2 == 0 ? "." : "1"; + str += String.format("\n + %d %d %d %d.0 %d.0 %s %s", i, i, i, i, i, b, n); + } + var iter = new StreamChunkIterator(str); + iterList.add(iter); } - iter = new StreamChunkIterator(str); + iterOfIter = iterList.iterator(); } public void getValue(StreamChunkRow row) { @@ -55,6 +61,10 @@ public void getValue(StreamChunkRow row) { @Benchmark public void streamchunkTest() { + if (!iterOfIter.hasNext()) { + throw new RuntimeException("too few prepared iter"); + } + var iter = iterOfIter.next(); int count = 0; while (true) { try (StreamChunkRow row = iter.next()) { diff --git a/src/java_binding/make-java-binding.toml b/src/java_binding/make-java-binding.toml index 957cec9c762f5..af76a18ba9b45 100644 --- a/src/java_binding/make-java-binding.toml +++ b/src/java_binding/make-java-binding.toml @@ -15,7 +15,7 @@ script = ''' #!/usr/bin/env bash set -ex cd java -mvn install --no-transfer-progress --pl java-binding-integration-test --am -DskipTests=true -Dmaven.javadoc.skip=true +mvn install --no-transfer-progress --pl java-binding-integration-test --am -DskipTests=true -Dmaven.javadoc.skip mvn dependency:copy-dependencies --no-transfer-progress --pl java-binding-integration-test ''' @@ -98,9 +98,6 @@ cd ${RISINGWAVE_ROOT}/java [tasks.run-java-binding-stream-chunk-benchmark] category = "RiseDev - Java Binding" description = "Run the java binding stream chunk benchmark" -dependencies = [ - "build-java-binding", -] script = ''' #!/usr/bin/env bash set -ex @@ -109,10 +106,10 @@ RISINGWAVE_ROOT=$(git rev-parse --show-toplevel) cd ${RISINGWAVE_ROOT}/java -mvn install --pl java-binding-benchmark --am -DskipTests=true -Dmaven.javadoc.skip=true +mvn install --pl java-binding-benchmark --am -DskipTests=true -Dmaven.javadoc.skip mvn dependency:copy-dependencies --pl java-binding-benchmark -java -cp "java-binding-benchmark/target/dependency/*:java-binding-benchmark/target/java-binding-benchmark-1.0-SNAPSHOT.jar" \ +java -cp "java-binding-benchmark/target/dependency/*:java-binding-benchmark/target/java-binding-benchmark-0.1.0-SNAPSHOT.jar" \ com.risingwave.java.binding.BenchmarkRunner ''' From b7c8c9d3d21cfebc3d66844fd190ad4f33bf9c82 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Tue, 26 Sep 2023 15:30:42 +0800 Subject: [PATCH 04/31] feat(meta): add create_type to `Table` (#12529) --- proto/catalog.proto | 10 +++++++ proto/ddl_service.proto | 7 ----- src/frontend/src/catalog/catalog_service.rs | 14 ++++------ src/frontend/src/catalog/table_catalog.rs | 4 ++- src/frontend/src/handler/create_mv.rs | 14 +++++----- src/frontend/src/test_utils.rs | 6 ++--- src/meta/src/rpc/ddl_controller.rs | 26 +++++++------------ src/meta/src/rpc/service/ddl_service.rs | 11 ++++---- src/rpc_client/src/meta_client.rs | 2 -- src/storage/src/filter_key_extractor.rs | 3 ++- .../src/delete_range_runner.rs | 3 ++- 11 files changed, 46 insertions(+), 54 deletions(-) diff --git a/proto/catalog.proto b/proto/catalog.proto index 07aff3baee22f..f200d10d234ff 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -39,6 +39,14 @@ enum StreamJobStatus { CREATED = 2; } +// How the stream job was created will determine +// whether they are persisted. +enum CreateType { + CREATE_TYPE_UNSPECIFIED = 0; + BACKGROUND = 1; + FOREGROUND = 2; +} + message StreamSourceInfo { // deprecated plan_common.RowFormatType row_format = 1; @@ -262,6 +270,8 @@ message Table { // Used to filter created / creating tables in meta. StreamJobStatus stream_job_status = 31; + CreateType create_type = 32; + // Per-table catalog version, used by schema change. `None` for internal tables and tests. // Not to be confused with the global catalog version for notification service. TableVersion version = 100; diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 68a74d421462c..27c9f2ee82f83 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -97,16 +97,9 @@ message DropSinkResponse { uint64 version = 2; } -enum StreamJobExecutionMode { - STREAM_JOB_EXECUTION_MODE_UNSPECIFIED = 0; - BACKGROUND = 1; - FOREGROUND = 2; -} - message CreateMaterializedViewRequest { catalog.Table materialized_view = 1; stream_plan.StreamFragmentGraph fragment_graph = 2; - StreamJobExecutionMode stream_job_execution_mode = 3; } message CreateMaterializedViewResponse { diff --git a/src/frontend/src/catalog/catalog_service.rs b/src/frontend/src/catalog/catalog_service.rs index bc2648fe105bd..8eb6b9e3e4485 100644 --- a/src/frontend/src/catalog/catalog_service.rs +++ b/src/frontend/src/catalog/catalog_service.rs @@ -21,10 +21,10 @@ use risingwave_common::error::ErrorCode::InternalError; use risingwave_common::error::{Result, RwError}; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::catalog::{ - PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, + PbCreateType, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, }; use risingwave_pb::ddl_service::alter_relation_name_request::Relation; -use risingwave_pb::ddl_service::{create_connection_request, StreamJobExecutionMode}; +use risingwave_pb::ddl_service::create_connection_request; use risingwave_pb::stream_plan::StreamFragmentGraph; use risingwave_rpc_client::MetaClient; use tokio::sync::watch::Receiver; @@ -70,7 +70,6 @@ pub trait CatalogWriter: Send + Sync { &self, table: PbTable, graph: StreamFragmentGraph, - stream_job_execution_mode: StreamJobExecutionMode, ) -> Result<()>; async fn create_table( @@ -191,16 +190,13 @@ impl CatalogWriter for CatalogWriterImpl { &self, table: PbTable, graph: StreamFragmentGraph, - stream_job_execution_mode: StreamJobExecutionMode, ) -> Result<()> { + let create_type = table.get_create_type().unwrap_or(PbCreateType::Foreground); let (_, version) = self .meta_client - .create_materialized_view(table, graph, stream_job_execution_mode) + .create_materialized_view(table, graph) .await?; - if matches!( - stream_job_execution_mode, - StreamJobExecutionMode::Foreground | StreamJobExecutionMode::Unspecified - ) { + if matches!(create_type, PbCreateType::Foreground) { self.wait_version(version).await? } Ok(()) diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 2b8ef546c9be9..778b43c0598f5 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -24,7 +24,7 @@ use risingwave_common::error::{ErrorCode, RwError}; use risingwave_common::util::epoch::Epoch; use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::catalog::table::{OptionalAssociatedSourceId, PbTableType, PbTableVersion}; -use risingwave_pb::catalog::{PbStreamJobStatus, PbTable}; +use risingwave_pb::catalog::{PbCreateType, PbStreamJobStatus, PbTable}; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; use risingwave_pb::plan_common::DefaultColumnDesc; @@ -402,6 +402,7 @@ impl TableCatalog { created_at_epoch: self.created_at_epoch.map(|epoch| epoch.0), cleaned_by_watermark: self.cleaned_by_watermark, stream_job_status: PbStreamJobStatus::Creating.into(), + create_type: PbCreateType::Foreground.into(), } } @@ -607,6 +608,7 @@ mod tests { created_at_epoch: None, cleaned_by_watermark: false, stream_job_status: PbStreamJobStatus::Creating.into(), + create_type: PbCreateType::Foreground.into(), } .into(); diff --git a/src/frontend/src/handler/create_mv.rs b/src/frontend/src/handler/create_mv.rs index 3983291e2845f..3fa9129f39743 100644 --- a/src/frontend/src/handler/create_mv.rs +++ b/src/frontend/src/handler/create_mv.rs @@ -15,8 +15,7 @@ use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::error::{ErrorCode, Result}; -use risingwave_pb::catalog::PbTable; -use risingwave_pb::ddl_service::StreamJobExecutionMode; +use risingwave_pb::catalog::{CreateType, PbTable}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_pb::user::grant_privilege::Action; use risingwave_sqlparser::ast::{EmitMode, Ident, ObjectName, Query}; @@ -164,7 +163,7 @@ pub async fn handle_create_mv( Ok(_) => {} }; - let (table, graph) = { + let (mut table, graph) = { let context = OptimizerContext::from_handler_args(handler_args); let has_order_by = !query.order_by.is_empty(); @@ -202,16 +201,17 @@ It only indicates the physical clustering of the data, which may improve the per )); let run_in_background = session.config().get_background_ddl(); - let stream_job_execution_mode = if run_in_background { - StreamJobExecutionMode::Background + let create_type = if run_in_background { + CreateType::Background } else { - StreamJobExecutionMode::Foreground + CreateType::Foreground }; + table.create_type = create_type.into(); let session = session.clone(); let catalog_writer = session.catalog_writer()?; catalog_writer - .create_materialized_view(table, graph, stream_job_execution_mode) + .create_materialized_view(table, graph) .await?; Ok(PgResponse::empty_result( diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 1823dcec91281..20eb252fc5053 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -34,7 +34,7 @@ use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::{ PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, Table, }; -use risingwave_pb::ddl_service::{create_connection_request, DdlProgress, StreamJobExecutionMode}; +use risingwave_pb::ddl_service::{create_connection_request, DdlProgress}; use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::{ BranchedObject, CompactionGroupInfo, HummockSnapshot, HummockVersion, HummockVersionDelta, @@ -235,7 +235,6 @@ impl CatalogWriter for MockCatalogWriter { &self, mut table: PbTable, _graph: StreamFragmentGraph, - _stream_job_execution_mode: StreamJobExecutionMode, ) -> Result<()> { table.id = self.gen_id(); self.catalog.write().create_table(&table); @@ -261,8 +260,7 @@ impl CatalogWriter for MockCatalogWriter { table.optional_associated_source_id = Some(OptionalAssociatedSourceId::AssociatedSourceId(source_id)); } - self.create_materialized_view(table, graph, StreamJobExecutionMode::Foreground) - .await?; + self.create_materialized_view(table, graph).await?; Ok(()) } diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index d4b324ea6ddc6..c65efc1726825 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -23,10 +23,10 @@ use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::epoch::Epoch; use risingwave_pb::catalog::connection::private_link_service::PbPrivateLinkProvider; use risingwave_pb::catalog::{ - connection, Connection, Database, Function, Schema, Source, Table, View, + connection, Connection, CreateType, Database, Function, Schema, Source, Table, View, }; use risingwave_pb::ddl_service::alter_relation_name_request::Relation; -use risingwave_pb::ddl_service::{DdlProgress, StreamJobExecutionMode}; +use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::stream_plan::StreamFragmentGraph as StreamFragmentGraphProto; use tokio::sync::Semaphore; use tracing::log::warn; @@ -93,11 +93,7 @@ pub enum DdlCommand { DropFunction(FunctionId), CreateView(View), DropView(ViewId, DropMode), - CreateStreamingJob( - StreamingJob, - StreamFragmentGraphProto, - StreamJobExecutionMode, - ), + CreateStreamingJob(StreamingJob, StreamFragmentGraphProto, CreateType), DropStreamingJob(StreamingJobId, DropMode), ReplaceTable(StreamingJob, StreamFragmentGraphProto, ColIndexMapping), AlterRelationName(Relation, String), @@ -240,12 +236,8 @@ impl DdlController { DdlCommand::DropView(view_id, drop_mode) => { ctrl.drop_view(view_id, drop_mode).await } - DdlCommand::CreateStreamingJob( - stream_job, - fragment_graph, - stream_job_execution_mode, - ) => { - ctrl.create_streaming_job(stream_job, fragment_graph, stream_job_execution_mode) + DdlCommand::CreateStreamingJob(stream_job, fragment_graph, create_type) => { + ctrl.create_streaming_job(stream_job, fragment_graph, create_type) .await } DdlCommand::DropStreamingJob(job_id, drop_mode) => { @@ -414,7 +406,7 @@ impl DdlController { &self, mut stream_job: StreamingJob, fragment_graph: StreamFragmentGraphProto, - stream_job_execution_mode: StreamJobExecutionMode, + create_type: CreateType, ) -> MetaResult { let _permit = self .creating_streaming_job_permits @@ -462,12 +454,12 @@ impl DdlController { } }; - match stream_job_execution_mode { - StreamJobExecutionMode::Foreground | StreamJobExecutionMode::Unspecified => { + match create_type { + CreateType::Foreground | CreateType::Unspecified => { self.create_streaming_job_inner(stream_job, table_fragments, ctx, internal_tables) .await } - StreamJobExecutionMode::Background => { + CreateType::Background => { let ctrl = self.clone(); let definition = stream_job.definition(); let fut = async move { diff --git a/src/meta/src/rpc/service/ddl_service.rs b/src/meta/src/rpc/service/ddl_service.rs index e22deadf947e8..935d398aeacb0 100644 --- a/src/meta/src/rpc/service/ddl_service.rs +++ b/src/meta/src/rpc/service/ddl_service.rs @@ -25,7 +25,7 @@ use risingwave_pb::catalog::connection::private_link_service::{ use risingwave_pb::catalog::connection::PbPrivateLinkService; use risingwave_pb::catalog::source::OptionalAssociatedTableId; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; -use risingwave_pb::catalog::{connection, Connection, PbSource, PbTable}; +use risingwave_pb::catalog::{connection, Connection, CreateType, PbSource, PbTable}; use risingwave_pb::ddl_service::ddl_service_server::DdlService; use risingwave_pb::ddl_service::drop_table_request::PbSourceId; use risingwave_pb::ddl_service::*; @@ -232,7 +232,7 @@ impl DdlService for DdlServiceImpl { .run_command(DdlCommand::CreateStreamingJob( stream_job, fragment_graph, - StreamJobExecutionMode::Foreground, + CreateType::Foreground, )) .await?; @@ -276,6 +276,7 @@ impl DdlService for DdlServiceImpl { let req = request.into_inner(); let mview = req.get_materialized_view()?.clone(); + let create_type = mview.get_create_type().unwrap_or(CreateType::Foreground); let fragment_graph = req.get_fragment_graph()?.clone(); let mut stream_job = StreamingJob::MaterializedView(mview); @@ -287,7 +288,7 @@ impl DdlService for DdlServiceImpl { .run_command(DdlCommand::CreateStreamingJob( stream_job, fragment_graph, - req.stream_job_execution_mode(), + create_type, )) .await?; @@ -342,7 +343,7 @@ impl DdlService for DdlServiceImpl { .run_command(DdlCommand::CreateStreamingJob( stream_job, fragment_graph, - StreamJobExecutionMode::Foreground, + CreateType::Foreground, )) .await?; @@ -438,7 +439,7 @@ impl DdlService for DdlServiceImpl { .run_command(DdlCommand::CreateStreamingJob( stream_job, fragment_graph, - StreamJobExecutionMode::Foreground, + CreateType::Foreground, )) .await?; diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 2b695f9c045b0..357cd4cf37f1d 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -332,12 +332,10 @@ impl MetaClient { &self, table: PbTable, graph: StreamFragmentGraph, - stream_job_execution_mode: StreamJobExecutionMode, ) -> Result<(TableId, CatalogVersion)> { let request = CreateMaterializedViewRequest { materialized_view: Some(table), fragment_graph: Some(graph), - stream_job_execution_mode: stream_job_execution_mode as i32, }; let resp = self.inner.create_materialized_view(request).await?; // TODO: handle error in `resp.status` here diff --git a/src/storage/src/filter_key_extractor.rs b/src/storage/src/filter_key_extractor.rs index c3b65bc26992d..b5a79a6f6b42f 100644 --- a/src/storage/src/filter_key_extractor.rs +++ b/src/storage/src/filter_key_extractor.rs @@ -448,7 +448,7 @@ mod tests { use risingwave_common::util::sort_util::OrderType; use risingwave_hummock_sdk::key::TABLE_PREFIX_LEN; use risingwave_pb::catalog::table::TableType; - use risingwave_pb::catalog::{PbStreamJobStatus, PbTable}; + use risingwave_pb::catalog::{PbCreateType, PbStreamJobStatus, PbTable}; use risingwave_pb::common::{PbColumnOrder, PbDirection, PbNullsAre, PbOrderType}; use risingwave_pb::plan_common::PbColumnCatalog; @@ -550,6 +550,7 @@ mod tests { created_at_epoch: None, cleaned_by_watermark: false, stream_job_status: PbStreamJobStatus::Created.into(), + create_type: PbCreateType::Foreground.into(), } } diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index 486b819d3ee7e..abeae5e418af2 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -37,7 +37,7 @@ use risingwave_meta::hummock::test_utils::setup_compute_env_with_config; use risingwave_meta::hummock::MockHummockMetaClient; use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; use risingwave_object_store::object::parse_remote_object_store; -use risingwave_pb::catalog::{PbStreamJobStatus, PbTable}; +use risingwave_pb::catalog::{PbCreateType, PbStreamJobStatus, PbTable}; use risingwave_pb::hummock::{CompactionConfig, CompactionGroupInfo}; use risingwave_pb::meta::SystemParams; use risingwave_rpc_client::HummockMetaClient; @@ -152,6 +152,7 @@ async fn compaction_test( created_at_epoch: None, cleaned_by_watermark: false, stream_job_status: PbStreamJobStatus::Created.into(), + create_type: PbCreateType::Foreground.into(), }; let mut delete_range_table = delete_key_table.clone(); delete_range_table.id = 2; From f38554ef84ad3aa795671e4f9f8ca0a5d49712bb Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 26 Sep 2023 16:13:42 +0800 Subject: [PATCH 05/31] refactor(sink): refine sink trait and macro (#12478) --- e2e_test/sink/append_only_sink.slt | 2 +- src/connector/src/sink/boxed.rs | 35 +-- src/connector/src/sink/clickhouse.rs | 29 +- src/connector/src/sink/doris.rs | 26 +- src/connector/src/sink/iceberg.rs | 28 +- src/connector/src/sink/kafka.rs | 17 +- src/connector/src/sink/kinesis.rs | 17 +- src/connector/src/sink/mod.rs | 284 ++++++++---------- src/connector/src/sink/nats.rs | 21 +- src/connector/src/sink/pulsar.rs | 17 +- src/connector/src/sink/redis.rs | 14 +- src/connector/src/sink/remote.rs | 191 ++++++------ src/connector/src/sink/test_sink.rs | 64 +++- src/meta/src/stream/sink.rs | 2 +- .../tests/integration_tests/sink/basic.rs | 40 +-- 15 files changed, 395 insertions(+), 392 deletions(-) diff --git a/e2e_test/sink/append_only_sink.slt b/e2e_test/sink/append_only_sink.slt index 5ace195ec48ce..9a6aeabb88ae3 100644 --- a/e2e_test/sink/append_only_sink.slt +++ b/e2e_test/sink/append_only_sink.slt @@ -22,7 +22,7 @@ create sink invalid_sink_type from t with (connector = 'blackhole', type = 'inva statement error `force_append_only` must be true or false create sink invalid_force_append_only from t with (connector = 'blackhole', force_append_only = 'invalid'); -statement error invalid connector type: invalid +statement error db error: ERROR: QueryError: internal error: Sink error: config error: unsupported sink connector invalid create sink invalid_connector from t with (connector = 'invalid'); statement ok diff --git a/src/connector/src/sink/boxed.rs b/src/connector/src/sink/boxed.rs index d966b1b6dea0f..8c4dcdb4b0506 100644 --- a/src/connector/src/sink/boxed.rs +++ b/src/connector/src/sink/boxed.rs @@ -12,28 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::fmt::{Debug, Formatter}; -use std::ops::{Deref, DerefMut}; +use std::ops::DerefMut; use std::sync::Arc; use async_trait::async_trait; use risingwave_common::array::StreamChunk; use risingwave_common::buffer::Bitmap; use risingwave_pb::connector_service::SinkMetadata; -use risingwave_rpc_client::ConnectorClient; -use crate::sink::{Sink, SinkCommitCoordinator, SinkWriter, SinkWriterParam}; +use crate::sink::{SinkCommitCoordinator, SinkWriter}; pub type BoxWriter = Box + Send + 'static>; pub type BoxCoordinator = Box; -pub type BoxSink = - Box, Coordinator = BoxCoordinator> + Send + Sync + 'static>; - -impl Debug for BoxSink { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - f.write_str("BoxSink") - } -} #[async_trait] impl SinkWriter for BoxWriter { @@ -70,24 +60,3 @@ impl SinkCommitCoordinator for BoxCoordinator { self.deref_mut().commit(epoch, metadata).await } } - -#[async_trait] -impl Sink for BoxSink { - type Coordinator = BoxCoordinator; - type Writer = BoxWriter<()>; - - async fn validate(&self) -> crate::sink::Result<()> { - self.deref().validate().await - } - - async fn new_writer(&self, writer_param: SinkWriterParam) -> crate::sink::Result { - self.deref().new_writer(writer_param).await - } - - async fn new_coordinator( - &self, - connector_client: Option, - ) -> crate::sink::Result { - self.deref().new_coordinator(connector_client).await - } -} diff --git a/src/connector/src/sink/clickhouse.rs b/src/connector/src/sink/clickhouse.rs index ef4cc8b993a9b..08287b789eaf9 100644 --- a/src/connector/src/sink/clickhouse.rs +++ b/src/connector/src/sink/clickhouse.rs @@ -30,7 +30,8 @@ use serde_with::serde_as; use super::{DummySinkCommitCoordinator, SinkWriterParam}; use crate::common::ClickHouseCommon; use crate::sink::{ - Result, Sink, SinkError, SinkWriter, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, + Result, Sink, SinkError, SinkParam, SinkWriter, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, + SINK_TYPE_UPSERT, }; pub const CLICKHOUSE_SINK: &str = "clickhouse"; @@ -70,21 +71,22 @@ impl ClickHouseConfig { } } -impl ClickHouseSink { - pub fn new( - config: ClickHouseConfig, - schema: Schema, - pk_indices: Vec, - is_append_only: bool, - ) -> Result { +impl TryFrom for ClickHouseSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let schema = param.schema(); + let config = ClickHouseConfig::from_hashmap(param.properties)?; Ok(Self { config, schema, - pk_indices, - is_append_only, + pk_indices: param.downstream_pk, + is_append_only: param.sink_type.is_append_only(), }) } +} +impl ClickHouseSink { /// Check that the column names and types of risingwave and clickhouse are identical fn check_column_name_and_type(&self, clickhouse_columns_desc: &[SystemColumn]) -> Result<()> { let rw_fields_name = build_fields_name_type_from_schema(&self.schema)?; @@ -205,11 +207,12 @@ impl ClickHouseSink { Ok(()) } } -#[async_trait::async_trait] impl Sink for ClickHouseSink { type Coordinator = DummySinkCommitCoordinator; type Writer = ClickHouseSinkWriter; + const SINK_NAME: &'static str = CLICKHOUSE_SINK; + async fn validate(&self) -> Result<()> { // For upsert clickhouse sink, the primary key must be defined. if !self.is_append_only && self.pk_indices.is_empty() { @@ -241,13 +244,13 @@ impl Sink for ClickHouseSink { } async fn new_writer(&self, _writer_env: SinkWriterParam) -> Result { - Ok(ClickHouseSinkWriter::new( + ClickHouseSinkWriter::new( self.config.clone(), self.schema.clone(), self.pk_indices.clone(), self.is_append_only, ) - .await?) + .await } } pub struct ClickHouseSinkWriter { diff --git a/src/connector/src/sink/doris.rs b/src/connector/src/sink/doris.rs index 82e6454663f48..3e226bcb37d46 100644 --- a/src/connector/src/sink/doris.rs +++ b/src/connector/src/sink/doris.rs @@ -29,7 +29,9 @@ use super::doris_connector::{DorisField, DorisInsert, DorisInsertClient, DORIS_D use super::{SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; use crate::common::DorisCommon; use crate::sink::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; -use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkWriter, SinkWriterParam}; +use crate::sink::{ + DummySinkCommitCoordinator, Result, Sink, SinkParam, SinkWriter, SinkWriterParam, +}; pub const DORIS_SINK: &str = "doris"; #[serde_as] @@ -151,19 +153,20 @@ impl DorisSink { } } -#[async_trait] impl Sink for DorisSink { type Coordinator = DummySinkCommitCoordinator; type Writer = DorisSinkWriter; + const SINK_NAME: &'static str = DORIS_SINK; + async fn new_writer(&self, _writer_env: SinkWriterParam) -> Result { - Ok(DorisSinkWriter::new( + DorisSinkWriter::new( self.config.clone(), self.schema.clone(), self.pk_indices.clone(), self.is_append_only, ) - .await?) + .await } async fn validate(&self) -> Result<()> { @@ -195,6 +198,21 @@ pub struct DorisSinkWriter { row_encoder: JsonEncoder, } +impl TryFrom for DorisSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let schema = param.schema(); + let config = DorisConfig::from_hashmap(param.properties)?; + DorisSink::new( + config, + schema, + param.downstream_pk, + param.sink_type.is_append_only(), + ) + } +} + impl DorisSinkWriter { pub async fn new( config: DorisConfig, diff --git a/src/connector/src/sink/iceberg.rs b/src/connector/src/sink/iceberg.rs index 41aded5ba26cf..25c5aa7765114 100644 --- a/src/connector/src/sink/iceberg.rs +++ b/src/connector/src/sink/iceberg.rs @@ -41,15 +41,21 @@ use super::{ }; use crate::deserialize_bool_from_string; use crate::sink::coordinate::CoordinatedSinkWriter; -use crate::sink::remote::{CoordinatedRemoteSink, RemoteConfig}; +use crate::sink::remote::{CoordinatedRemoteSink, RemoteSinkTrait}; use crate::sink::{Result, SinkCommitCoordinator, SinkParam}; /// This iceberg sink is WIP. When it ready, we will change this name to "iceberg". pub const ICEBERG_SINK: &str = "iceberg"; pub const REMOTE_ICEBERG_SINK: &str = "iceberg_java"; -pub type RemoteIcebergSink = CoordinatedRemoteSink; -pub type RemoteIcebergConfig = RemoteConfig; +#[derive(Debug)] +pub struct RemoteIceberg; + +impl RemoteSinkTrait for RemoteIceberg { + const SINK_NAME: &'static str = REMOTE_ICEBERG_SINK; +} + +pub type RemoteIcebergSink = CoordinatedRemoteSink; #[derive(Debug, Clone, Deserialize)] #[serde(deny_unknown_fields)] @@ -192,6 +198,15 @@ pub struct IcebergSink { param: SinkParam, } +impl TryFrom for IcebergSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let config = IcebergConfig::from_hashmap(param.properties.clone())?; + IcebergSink::new(config, param) + } +} + impl Debug for IcebergSink { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("IcebergSink") @@ -240,11 +255,12 @@ impl IcebergSink { } } -#[async_trait::async_trait] impl Sink for IcebergSink { type Coordinator = IcebergSinkCommitter; type Writer = CoordinatedSinkWriter; + const SINK_NAME: &'static str = ICEBERG_SINK; + async fn validate(&self) -> Result<()> { let _ = self.create_table().await?; Ok(()) @@ -261,7 +277,7 @@ impl Sink for IcebergSink { .map_err(|err| SinkError::Iceberg(anyhow!(err)))?, table, }; - Ok(CoordinatedSinkWriter::new( + CoordinatedSinkWriter::new( writer_param .meta_client .expect("should have meta client") @@ -275,7 +291,7 @@ impl Sink for IcebergSink { })?, inner, ) - .await?) + .await } async fn new_coordinator( diff --git a/src/connector/src/sink/kafka.rs b/src/connector/src/sink/kafka.rs index 85894792cd7ee..0cad48bbb57f6 100644 --- a/src/connector/src/sink/kafka.rs +++ b/src/connector/src/sink/kafka.rs @@ -281,24 +281,29 @@ pub struct KafkaSink { sink_from_name: String, } -impl KafkaSink { - pub fn new(config: KafkaConfig, param: SinkParam) -> Self { - Self { +impl TryFrom for KafkaSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let schema = param.schema(); + let config = KafkaConfig::from_hashmap(param.properties)?; + Ok(Self { config, - schema: param.schema(), + schema, pk_indices: param.downstream_pk, is_append_only: param.sink_type.is_append_only(), db_name: param.db_name, sink_from_name: param.sink_from_name, - } + }) } } -#[async_trait::async_trait] impl Sink for KafkaSink { type Coordinator = DummySinkCommitCoordinator; type Writer = SinkWriterV1Adapter; + const SINK_NAME: &'static str = KAFKA_SINK; + async fn new_writer(&self, _writer_param: SinkWriterParam) -> Result { Ok(SinkWriterV1Adapter::new( KafkaSinkWriter::new( diff --git a/src/connector/src/sink/kinesis.rs b/src/connector/src/sink/kinesis.rs index 114070ca1e440..f16ca3905b884 100644 --- a/src/connector/src/sink/kinesis.rs +++ b/src/connector/src/sink/kinesis.rs @@ -47,24 +47,29 @@ pub struct KinesisSink { sink_from_name: String, } -impl KinesisSink { - pub fn new(config: KinesisSinkConfig, param: SinkParam) -> Self { - Self { +impl TryFrom for KinesisSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let schema = param.schema(); + let config = KinesisSinkConfig::from_hashmap(param.properties)?; + Ok(Self { config, - schema: param.schema(), + schema, pk_indices: param.downstream_pk, is_append_only: param.sink_type.is_append_only(), db_name: param.db_name, sink_from_name: param.sink_from_name, - } + }) } } -#[async_trait::async_trait] impl Sink for KinesisSink { type Coordinator = DummySinkCommitCoordinator; type Writer = KinesisSinkWriter; + const SINK_NAME: &'static str = KINESIS_SINK; + async fn validate(&self) -> Result<()> { // For upsert Kafka sink, the primary key must be defined. if !self.is_append_only && self.pk_indices.is_empty() { diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 449f4f541f0e4..f312453481db6 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -27,7 +27,6 @@ pub mod nats; pub mod pulsar; pub mod redis; pub mod remote; -#[cfg(any(test, madsim))] pub mod test_sink; pub mod utils; @@ -37,7 +36,6 @@ use std::sync::Arc; use ::clickhouse::error::Error as ClickHouseError; use anyhow::anyhow; use async_trait::async_trait; -use enum_as_inner::EnumAsInner; use risingwave_common::array::StreamChunk; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::{ColumnDesc, Field, Schema}; @@ -50,27 +48,82 @@ use thiserror::Error; pub use tracing; use self::catalog::SinkType; -use self::clickhouse::{ClickHouseConfig, ClickHouseSink}; -use self::doris::{DorisConfig, DorisSink}; use self::encoder::SerTo; use self::formatter::SinkFormatter; -use self::iceberg::{IcebergSink, ICEBERG_SINK, REMOTE_ICEBERG_SINK}; -use self::pulsar::{PulsarConfig, PulsarSink}; -use crate::sink::boxed::BoxSink; use crate::sink::catalog::{SinkCatalog, SinkId}; -use crate::sink::clickhouse::CLICKHOUSE_SINK; -use crate::sink::doris::DORIS_SINK; -use crate::sink::iceberg::{IcebergConfig, RemoteIcebergConfig, RemoteIcebergSink}; -use crate::sink::kafka::{KafkaConfig, KafkaSink, KAFKA_SINK}; -use crate::sink::kinesis::{KinesisSink, KinesisSinkConfig, KINESIS_SINK}; -use crate::sink::nats::{NatsConfig, NatsSink, NATS_SINK}; -use crate::sink::pulsar::PULSAR_SINK; -use crate::sink::redis::{RedisConfig, RedisSink}; -use crate::sink::remote::{CoordinatedRemoteSink, RemoteConfig, RemoteSink}; -#[cfg(any(test, madsim))] -use crate::sink::test_sink::{build_test_sink, TEST_SINK_NAME}; +pub use crate::sink::clickhouse::ClickHouseSink; +pub use crate::sink::iceberg::{IcebergSink, RemoteIcebergSink}; +pub use crate::sink::kafka::KafkaSink; +pub use crate::sink::kinesis::KinesisSink; +pub use crate::sink::nats::NatsSink; +pub use crate::sink::pulsar::PulsarSink; +pub use crate::sink::redis::RedisSink; +pub use crate::sink::remote::{CassandraSink, DeltaLakeSink, ElasticSearchSink, JdbcSink}; +pub use crate::sink::test_sink::TestSink; use crate::ConnectorParams; +#[macro_export] +macro_rules! for_all_sinks { + ($macro:path $(, $arg:tt)*) => { + $macro! { + { + { Redis, $crate::sink::RedisSink }, + { Kafka, $crate::sink::KafkaSink }, + { Pulsar, $crate::sink::PulsarSink }, + { BlackHole, $crate::sink::BlackHoleSink }, + { Kinesis, $crate::sink::KinesisSink }, + { ClickHouse, $crate::sink::ClickHouseSink }, + { Iceberg, $crate::sink::IcebergSink }, + { Nats, $crate::sink::NatsSink }, + { RemoteIceberg, $crate::sink::RemoteIcebergSink }, + { Jdbc, $crate::sink::JdbcSink }, + { DeltaLake, $crate::sink::DeltaLakeSink }, + { ElasticSearch, $crate::sink::ElasticSearchSink }, + { Cassandra, $crate::sink::CassandraSink }, + { Doris, $crate::sink::doris::DorisSink }, + { Test, $crate::sink::TestSink } + } + $(,$arg)* + } + }; +} + +#[macro_export] +macro_rules! dispatch_sink { + ({$({$variant_name:ident, $sink_type:ty}),*}, $impl:tt, $sink:tt, $body:tt) => {{ + use $crate::sink::SinkImpl; + + match $impl { + $( + SinkImpl::$variant_name($sink) => $body, + )* + } + }}; + ($impl:expr, $sink:ident, $body:expr) => {{ + $crate::for_all_sinks! {$crate::dispatch_sink, {$impl}, $sink, {$body}} + }}; +} + +#[macro_export] +macro_rules! match_sink_name_str { + ({$({$variant_name:ident, $sink_type:ty}),*}, $name_str:tt, $type_name:ident, $body:tt, $on_other_closure:tt) => { + match $name_str { + $( + <$sink_type>::SINK_NAME => { + type $type_name = $sink_type; + { + $body + } + }, + )* + other => ($on_other_closure)(other), + } + }; + ($name_str:expr, $type_name:ident, $body:expr, $on_other_closure:expr) => {{ + $crate::for_all_sinks! {$crate::match_sink_name_str, {$name_str}, $type_name, {$body}, {$on_other_closure}} + }}; +} + pub const DOWNSTREAM_SINK_KEY: &str = "connector"; pub const SINK_TYPE_OPTION: &str = "type"; pub const SINK_TYPE_APPEND_ONLY: &str = "append-only"; @@ -156,13 +209,14 @@ pub struct SinkWriterParam { pub meta_client: Option, } -#[async_trait] -pub trait Sink { +pub trait Sink: TryFrom { + const SINK_NAME: &'static str; type Writer: SinkWriter; type Coordinator: SinkCommitCoordinator; async fn validate(&self) -> Result<()>; async fn new_writer(&self, writer_param: SinkWriterParam) -> Result; + #[expect(clippy::unused_async)] async fn new_coordinator( &self, _connector_client: Option, @@ -320,33 +374,25 @@ impl SinkCommitCoordinator for DummySinkCommitCoordinator { } } -#[derive(Clone, Debug, EnumAsInner)] -pub enum SinkConfig { - Redis(RedisConfig), - Kafka(Box), - Remote(RemoteConfig), - Kinesis(Box), - Iceberg(IcebergConfig), - RemoteIceberg(RemoteIcebergConfig), - Pulsar(PulsarConfig), - BlackHole, - ClickHouse(Box), - Doris(Box), - Nats(NatsConfig), - #[cfg(any(test, madsim))] - Test, -} - pub const BLACKHOLE_SINK: &str = "blackhole"; #[derive(Debug)] pub struct BlackHoleSink; -#[async_trait] +impl TryFrom for BlackHoleSink { + type Error = SinkError; + + fn try_from(_value: SinkParam) -> std::result::Result { + Ok(Self) + } +} + impl Sink for BlackHoleSink { type Coordinator = DummySinkCommitCoordinator; type Writer = Self; + const SINK_NAME: &'static str = BLACKHOLE_SINK; + async fn new_writer(&self, _writer_env: SinkWriterParam) -> Result { Ok(Self) } @@ -371,144 +417,68 @@ impl SinkWriter for BlackHoleSink { } } -impl SinkConfig { - pub fn from_hashmap(mut properties: HashMap) -> Result { +impl SinkImpl { + pub fn new(mut param: SinkParam) -> Result { const CONNECTOR_TYPE_KEY: &str = "connector"; const CONNECTION_NAME_KEY: &str = "connection.name"; const PRIVATE_LINK_TARGET_KEY: &str = "privatelink.targets"; // remove privatelink related properties if any - properties.remove(PRIVATE_LINK_TARGET_KEY); - properties.remove(CONNECTION_NAME_KEY); + param.properties.remove(PRIVATE_LINK_TARGET_KEY); + param.properties.remove(CONNECTION_NAME_KEY); - let sink_type = properties + let sink_type = param + .properties .get(CONNECTOR_TYPE_KEY) .ok_or_else(|| SinkError::Config(anyhow!("missing config: {}", CONNECTOR_TYPE_KEY)))?; - match sink_type.to_lowercase().as_str() { - KAFKA_SINK => Ok(SinkConfig::Kafka(Box::new(KafkaConfig::from_hashmap( - properties, - )?))), - KINESIS_SINK => Ok(SinkConfig::Kinesis(Box::new( - KinesisSinkConfig::from_hashmap(properties)?, - ))), - CLICKHOUSE_SINK => Ok(SinkConfig::ClickHouse(Box::new( - ClickHouseConfig::from_hashmap(properties)?, - ))), - DORIS_SINK => Ok(SinkConfig::Doris(Box::new(DorisConfig::from_hashmap( - properties, - )?))), - BLACKHOLE_SINK => Ok(SinkConfig::BlackHole), - PULSAR_SINK => Ok(SinkConfig::Pulsar(PulsarConfig::from_hashmap(properties)?)), - REMOTE_ICEBERG_SINK => Ok(SinkConfig::RemoteIceberg( - RemoteIcebergConfig::from_hashmap(properties)?, - )), - ICEBERG_SINK => Ok(SinkConfig::Iceberg(IcebergConfig::from_hashmap( - properties, - )?)), - NATS_SINK => Ok(SinkConfig::Nats(NatsConfig::from_hashmap(properties)?)), - // Only in test or deterministic test, test sink is enabled. - #[cfg(any(test, madsim))] - TEST_SINK_NAME => Ok(SinkConfig::Test), - _ => Ok(SinkConfig::Remote(RemoteConfig::from_hashmap(properties)?)), - } + match_sink_name_str!( + sink_type.to_lowercase().as_str(), + SinkType, + Ok(SinkType::try_from(param)?.into()), + |other| { + Err(SinkError::Config(anyhow!( + "unsupported sink connector {}", + other + ))) + } + ) } } pub fn build_sink(param: SinkParam) -> Result { - let config = SinkConfig::from_hashmap(param.properties.clone())?; - SinkImpl::new(config, param) + SinkImpl::new(param) } -#[derive(Debug)] -pub enum SinkImpl { - Redis(RedisSink), - Kafka(KafkaSink), - Remote(RemoteSink), - Pulsar(PulsarSink), - BlackHole(BlackHoleSink), - Kinesis(KinesisSink), - ClickHouse(ClickHouseSink), - Doris(DorisSink), - Iceberg(IcebergSink), - Nats(NatsSink), - RemoteIceberg(RemoteIcebergSink), - TestSink(BoxSink), -} - -impl SinkImpl { - pub fn get_connector(&self) -> &'static str { - match self { - SinkImpl::Kafka(_) => "kafka", - SinkImpl::Redis(_) => "redis", - SinkImpl::Remote(_) => "remote", - SinkImpl::Pulsar(_) => "pulsar", - SinkImpl::BlackHole(_) => "blackhole", - SinkImpl::Kinesis(_) => "kinesis", - SinkImpl::ClickHouse(_) => "clickhouse", - SinkImpl::Iceberg(_) => "iceberg", - SinkImpl::Nats(_) => "nats", - SinkImpl::RemoteIceberg(_) => "iceberg_java", - SinkImpl::TestSink(_) => "test", - SinkImpl::Doris(_) => "doris", +macro_rules! def_sink_impl { + () => { + $crate::for_all_sinks! { def_sink_impl } + }; + ({ $({ $variant_name:ident, $sink_type:ty }),* }) => { + #[derive(Debug)] + pub enum SinkImpl { + $( + $variant_name($sink_type), + )* } - } -} -#[macro_export] -macro_rules! dispatch_sink { - ($impl:expr, $sink:ident, $body:tt) => {{ - use $crate::sink::SinkImpl; - - match $impl { - SinkImpl::Redis($sink) => $body, - SinkImpl::Kafka($sink) => $body, - SinkImpl::Remote($sink) => $body, - SinkImpl::Pulsar($sink) => $body, - SinkImpl::BlackHole($sink) => $body, - SinkImpl::Kinesis($sink) => $body, - SinkImpl::ClickHouse($sink) => $body, - SinkImpl::Doris($sink) => $body, - SinkImpl::Iceberg($sink) => $body, - SinkImpl::Nats($sink) => $body, - SinkImpl::RemoteIceberg($sink) => $body, - SinkImpl::TestSink($sink) => $body, - } - }}; + $( + impl From<$sink_type> for SinkImpl { + fn from(sink: $sink_type) -> SinkImpl { + SinkImpl::$variant_name(sink) + } + } + )* + }; } +def_sink_impl!(); + impl SinkImpl { - pub fn new(cfg: SinkConfig, param: SinkParam) -> Result { - Ok(match cfg { - SinkConfig::Redis(cfg) => SinkImpl::Redis(RedisSink::new(cfg, param.schema())?), - SinkConfig::Kafka(cfg) => SinkImpl::Kafka(KafkaSink::new(*cfg, param)), - SinkConfig::Kinesis(cfg) => SinkImpl::Kinesis(KinesisSink::new(*cfg, param)), - SinkConfig::Remote(cfg) => SinkImpl::Remote(RemoteSink::new(cfg, param)), - SinkConfig::Pulsar(cfg) => SinkImpl::Pulsar(PulsarSink::new(cfg, param)), - SinkConfig::BlackHole => SinkImpl::BlackHole(BlackHoleSink), - SinkConfig::ClickHouse(cfg) => SinkImpl::ClickHouse(ClickHouseSink::new( - *cfg, - param.schema(), - param.downstream_pk, - param.sink_type.is_append_only(), - )?), - SinkConfig::Iceberg(cfg) => SinkImpl::Iceberg(IcebergSink::new(cfg, param)?), - SinkConfig::Nats(cfg) => SinkImpl::Nats(NatsSink::new( - cfg, - param.schema(), - param.sink_type.is_append_only(), - )), - SinkConfig::RemoteIceberg(cfg) => { - SinkImpl::RemoteIceberg(CoordinatedRemoteSink(RemoteSink::new(cfg, param))) - } - #[cfg(any(test, madsim))] - SinkConfig::Test => SinkImpl::TestSink(build_test_sink(param)?), - SinkConfig::Doris(cfg) => SinkImpl::Doris(DorisSink::new( - *cfg, - param.schema(), - param.downstream_pk, - param.sink_type.is_append_only(), - )?), - }) + pub fn get_connector(&self) -> &'static str { + fn get_name(_: &S) -> &'static str { + S::SINK_NAME + } + dispatch_sink!(self, sink, get_name(sink)) } } diff --git a/src/connector/src/sink/nats.rs b/src/connector/src/sink/nats.rs index 57b9e572c0615..1a126eb2c74d2 100644 --- a/src/connector/src/sink/nats.rs +++ b/src/connector/src/sink/nats.rs @@ -28,7 +28,7 @@ use super::utils::chunk_to_json; use super::{DummySinkCommitCoordinator, SinkWriter, SinkWriterParam}; use crate::common::NatsCommon; use crate::sink::encoder::{JsonEncoder, TimestampHandlingMode}; -use crate::sink::{Result, Sink, SinkError, SINK_TYPE_APPEND_ONLY}; +use crate::sink::{Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY}; pub const NATS_SINK: &str = "nats"; @@ -71,21 +71,26 @@ impl NatsConfig { } } -impl NatsSink { - pub fn new(config: NatsConfig, schema: Schema, is_append_only: bool) -> Self { - Self { +impl TryFrom for NatsSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let schema = param.schema(); + let config = NatsConfig::from_hashmap(param.properties)?; + Ok(Self { config, schema, - is_append_only, - } + is_append_only: param.sink_type.is_append_only(), + }) } } -#[async_trait::async_trait] impl Sink for NatsSink { type Coordinator = DummySinkCommitCoordinator; type Writer = NatsSinkWriter; + const SINK_NAME: &'static str = NATS_SINK; + async fn validate(&self) -> Result<()> { if !self.is_append_only { return Err(SinkError::Nats(anyhow!( @@ -105,7 +110,7 @@ impl Sink for NatsSink { } async fn new_writer(&self, _writer_env: SinkWriterParam) -> Result { - Ok(NatsSinkWriter::new(self.config.clone(), self.schema.clone()).await?) + NatsSinkWriter::new(self.config.clone(), self.schema.clone()).await } } diff --git a/src/connector/src/sink/pulsar.rs b/src/connector/src/sink/pulsar.rs index 8dc478f7dd82d..df13c2b4c7d01 100644 --- a/src/connector/src/sink/pulsar.rs +++ b/src/connector/src/sink/pulsar.rs @@ -145,24 +145,29 @@ pub struct PulsarSink { sink_from_name: String, } -impl PulsarSink { - pub fn new(config: PulsarConfig, param: SinkParam) -> Self { - Self { +impl TryFrom for PulsarSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let schema = param.schema(); + let config = PulsarConfig::from_hashmap(param.properties)?; + Ok(Self { config, - schema: param.schema(), + schema, downstream_pk: param.downstream_pk, is_append_only: param.sink_type.is_append_only(), db_name: param.db_name, sink_from_name: param.sink_from_name, - } + }) } } -#[async_trait] impl Sink for PulsarSink { type Coordinator = DummySinkCommitCoordinator; type Writer = PulsarSinkWriter; + const SINK_NAME: &'static str = PULSAR_SINK; + async fn new_writer(&self, _writer_param: SinkWriterParam) -> Result { PulsarSinkWriter::new( self.config.clone(), diff --git a/src/connector/src/sink/redis.rs b/src/connector/src/sink/redis.rs index f7957335aa1e3..8cd0875300d29 100644 --- a/src/connector/src/sink/redis.rs +++ b/src/connector/src/sink/redis.rs @@ -14,9 +14,10 @@ use async_trait::async_trait; use risingwave_common::array::StreamChunk; -use risingwave_common::catalog::Schema; -use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkWriter, SinkWriterParam}; +use crate::sink::{ + DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SinkWriter, SinkWriterParam, +}; #[derive(Clone, Debug)] pub struct RedisConfig; @@ -24,17 +25,20 @@ pub struct RedisConfig; #[derive(Debug)] pub struct RedisSink; -impl RedisSink { - pub fn new(_cfg: RedisConfig, _schema: Schema) -> Result { +impl TryFrom for RedisSink { + type Error = SinkError; + + fn try_from(_param: SinkParam) -> std::result::Result { todo!() } } -#[async_trait] impl Sink for RedisSink { type Coordinator = DummySinkCommitCoordinator; type Writer = RedisSinkWriter; + const SINK_NAME: &'static str = "redis"; + async fn new_writer(&self, _writer_env: SinkWriterParam) -> Result { todo!() } diff --git a/src/connector/src/sink/remote.rs b/src/connector/src/sink/remote.rs index f6fa615e44c53..51195ac366a27 100644 --- a/src/connector/src/sink/remote.rs +++ b/src/connector/src/sink/remote.rs @@ -22,7 +22,6 @@ use itertools::Itertools; use jni::objects::{JByteArray, JValue, JValueOwned}; use prost::Message; use risingwave_common::array::StreamChunk; -use risingwave_common::catalog::Schema; use risingwave_common::error::anyhow_error; use risingwave_common::types::DataType; use risingwave_jni_core::jvm_runtime::JVM; @@ -45,7 +44,6 @@ use tracing::{error, warn}; use super::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; use crate::sink::coordinate::CoordinatedSinkWriter; -use crate::sink::iceberg::REMOTE_ICEBERG_SINK; use crate::sink::SinkError::Remote; use crate::sink::{ DummySinkCommitCoordinator, Result, Sink, SinkCommitCoordinator, SinkError, SinkParam, @@ -53,68 +51,58 @@ use crate::sink::{ }; use crate::ConnectorParams; -pub const VALID_REMOTE_SINKS: [&str; 5] = [ - "jdbc", - REMOTE_ICEBERG_SINK, - "deltalake", - "elasticsearch", - "cassandra", -]; - -pub fn is_valid_remote_sink(connector_type: &str) -> bool { - VALID_REMOTE_SINKS.contains(&connector_type) -} - -#[derive(Clone, Debug)] -pub struct RemoteConfig { - pub connector_type: String, - pub properties: HashMap, +macro_rules! def_remote_sink { + () => { + def_remote_sink! { + { ElasticSearch, ElasticSearchSink, "elasticsearch" }, + { Cassandra, CassandraSink, "cassandra" }, + { Jdbc, JdbcSink, "jdbc" }, + { DeltaLake, DeltaLakeSink, "deltalake" } + } + }; + ($({ $variant_name:ident, $sink_type_name:ident, $sink_name:expr }),*) => { + $( + #[derive(Debug)] + pub struct $variant_name; + impl RemoteSinkTrait for $variant_name { + const SINK_NAME: &'static str = $sink_name; + } + pub type $sink_type_name = RemoteSink<$variant_name>; + )* + }; } -impl RemoteConfig { - pub fn from_hashmap(values: HashMap) -> Result { - let connector_type = values - .get("connector") - .expect("sink type must be specified") - .to_string(); - - if !is_valid_remote_sink(connector_type.as_str()) { - return Err(SinkError::Config(anyhow!( - "invalid connector type: {connector_type}" - ))); - } +def_remote_sink!(); - Ok(RemoteConfig { - connector_type, - properties: values, - }) - } +pub trait RemoteSinkTrait: Send + Sync + 'static { + const SINK_NAME: &'static str; } #[derive(Debug)] -pub struct RemoteSink { - config: RemoteConfig, +pub struct RemoteSink { param: SinkParam, + _phantom: PhantomData, } -impl RemoteSink { - pub fn new(config: RemoteConfig, param: SinkParam) -> Self { - Self { config, param } +impl TryFrom for RemoteSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + Ok(Self { + param, + _phantom: PhantomData, + }) } } -#[async_trait] -impl Sink for RemoteSink { +impl Sink for RemoteSink { type Coordinator = DummySinkCommitCoordinator; - type Writer = RemoteSinkWriter; + type Writer = RemoteSinkWriter; + + const SINK_NAME: &'static str = R::SINK_NAME; async fn new_writer(&self, writer_param: SinkWriterParam) -> Result { - Ok(RemoteSinkWriter::new( - self.config.clone(), - self.param.clone(), - writer_param.connector_params, - ) - .await?) + RemoteSinkWriter::new(self.param.clone(), writer_param.connector_params).await } async fn validate(&self) -> Result<()> { @@ -195,19 +183,28 @@ impl Sink for RemoteSink { } #[derive(Debug)] -pub struct CoordinatedRemoteSink(pub RemoteSink); +pub struct CoordinatedRemoteSink(pub RemoteSink); -#[async_trait] -impl Sink for CoordinatedRemoteSink { - type Coordinator = RemoteCoordinator; - type Writer = CoordinatedSinkWriter; +impl TryFrom for CoordinatedRemoteSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + RemoteSink::try_from(param).map(Self) + } +} + +impl Sink for CoordinatedRemoteSink { + type Coordinator = RemoteCoordinator; + type Writer = CoordinatedSinkWriter>; + + const SINK_NAME: &'static str = R::SINK_NAME; async fn validate(&self) -> Result<()> { self.0.validate().await } async fn new_writer(&self, writer_param: SinkWriterParam) -> Result { - Ok(CoordinatedSinkWriter::new( + CoordinatedSinkWriter::new( writer_param .meta_client .expect("should have meta client") @@ -219,50 +216,40 @@ impl Sink for CoordinatedRemoteSink { "sink needs coordination should not have singleton input" )) })?, - CoordinatedRemoteSinkWriter::new( - self.0.config.clone(), - self.0.param.clone(), - writer_param.connector_params, - ) - .await?, + CoordinatedRemoteSinkWriter::new(self.0.param.clone(), writer_param.connector_params) + .await?, ) - .await?) + .await } async fn new_coordinator( &self, connector_client: Option, ) -> Result { - Ok(RemoteCoordinator::new( + RemoteCoordinator::new( connector_client .ok_or_else(|| Remote(anyhow_error!("no connector client specified")))?, self.0.param.clone(), ) - .await?) + .await } } -pub type RemoteSinkWriter = RemoteSinkWriterInner<()>; -pub type CoordinatedRemoteSinkWriter = RemoteSinkWriterInner>; +pub type RemoteSinkWriter = RemoteSinkWriterInner<(), R>; +pub type CoordinatedRemoteSinkWriter = RemoteSinkWriterInner, R>; -pub struct RemoteSinkWriterInner { - pub connector_type: String, +pub struct RemoteSinkWriterInner { properties: HashMap, epoch: Option, batch_id: u64, - schema: Schema, payload_format: SinkPayloadFormat, stream_handle: SinkWriterStreamHandle, json_encoder: JsonEncoder, - _phantom: PhantomData, + _phantom: PhantomData<(SM, R)>, } -impl RemoteSinkWriterInner { - pub async fn new( - config: RemoteConfig, - param: SinkParam, - connector_params: ConnectorParams, - ) -> Result { +impl RemoteSinkWriterInner { + pub async fn new(param: SinkParam, connector_params: ConnectorParams) -> Result { let client = connector_params.connector_client.ok_or_else(|| { SinkError::Remote(anyhow_error!( "connector node endpoint not specified or unable to connect to connector node" @@ -274,24 +261,25 @@ impl RemoteSinkWriterInner { .inspect_err(|e| { error!( "failed to start sink stream for connector `{}`: {:?}", - &config.connector_type, e + R::SINK_NAME, + e ) })?; tracing::trace!( "{:?} sink stream started with properties: {:?}", - &config.connector_type, - &config.properties + R::SINK_NAME, + ¶m.properties ); + let schema = param.schema(); + Ok(Self { - connector_type: config.connector_type, - properties: config.properties, + properties: param.properties, epoch: None, batch_id: 0, - schema: param.schema(), stream_handle, payload_format: connector_params.sink_payload_format, - json_encoder: JsonEncoder::new(param.schema(), None, TimestampHandlingMode::String), + json_encoder: JsonEncoder::new(schema, None, TimestampHandlingMode::String), _phantom: PhantomData, }) } @@ -300,8 +288,8 @@ impl RemoteSinkWriterInner { fn for_test( response_receiver: UnboundedReceiver>, request_sender: Sender, - ) -> RemoteSinkWriter { - use risingwave_common::catalog::Field; + ) -> RemoteSinkWriter { + use risingwave_common::catalog::{Field, Schema}; let properties = HashMap::from([("output.path".to_string(), "/tmp/rw".to_string())]); let schema = Schema::new(vec![ @@ -328,12 +316,10 @@ impl RemoteSinkWriterInner { ); RemoteSinkWriter { - connector_type: "file".to_string(), properties, epoch: None, batch_id: 0, - json_encoder: JsonEncoder::new(schema.clone(), None, TimestampHandlingMode::String), - schema, + json_encoder: JsonEncoder::new(schema, None, TimestampHandlingMode::String), stream_handle, payload_format: SinkPayloadFormat::Json, _phantom: PhantomData, @@ -347,7 +333,7 @@ trait HandleBarrierResponse { fn non_checkpoint_return_value() -> Self::SinkMetadata; } -impl HandleBarrierResponse for RemoteSinkWriter { +impl HandleBarrierResponse for RemoteSinkWriter { type SinkMetadata = (); fn handle_commit_response(rsp: CommitResponse) -> Result { @@ -360,7 +346,7 @@ impl HandleBarrierResponse for RemoteSinkWriter { fn non_checkpoint_return_value() -> Self::SinkMetadata {} } -impl HandleBarrierResponse for CoordinatedRemoteSinkWriter { +impl HandleBarrierResponse for CoordinatedRemoteSinkWriter { type SinkMetadata = Option; fn handle_commit_response(rsp: CommitResponse) -> Result { @@ -379,7 +365,7 @@ impl HandleBarrierResponse for CoordinatedRemoteSinkWriter { } #[async_trait] -impl SinkWriter for RemoteSinkWriterInner +impl SinkWriter for RemoteSinkWriterInner where Self: HandleBarrierResponse, { @@ -449,21 +435,25 @@ where } } -pub struct RemoteCoordinator { +pub struct RemoteCoordinator { stream_handle: SinkCoordinatorStreamHandle, + _phantom: PhantomData, } -impl RemoteCoordinator { +impl RemoteCoordinator { pub async fn new(client: ConnectorClient, param: SinkParam) -> Result { let stream_handle = client .start_sink_coordinator_stream(param.to_proto()) .await?; - Ok(RemoteCoordinator { stream_handle }) + Ok(RemoteCoordinator { + stream_handle, + _phantom: PhantomData, + }) } } #[async_trait] -impl SinkCommitCoordinator for RemoteCoordinator { +impl SinkCommitCoordinator for RemoteCoordinator { async fn init(&mut self) -> Result<()> { Ok(()) } @@ -486,15 +476,20 @@ mod test { use risingwave_pb::data; use tokio::sync::mpsc; - use crate::sink::remote::RemoteSinkWriter; + use crate::sink::remote::{RemoteSinkTrait, RemoteSinkWriter}; use crate::sink::SinkWriter; + struct TestRemote; + impl RemoteSinkTrait for TestRemote { + const SINK_NAME: &'static str = "test-remote"; + } + #[tokio::test] async fn test_epoch_check() { let (request_sender, mut request_recv) = mpsc::channel(16); let (_, resp_recv) = mpsc::unbounded_channel(); - let mut sink = RemoteSinkWriter::for_test(resp_recv, request_sender); + let mut sink = >::for_test(resp_recv, request_sender); let chunk = StreamChunk::from_pretty( " i T + 1 Ripper @@ -531,7 +526,7 @@ mod test { async fn test_remote_sink() { let (request_sender, mut request_receiver) = mpsc::channel(16); let (response_sender, response_receiver) = mpsc::unbounded_channel(); - let mut sink = RemoteSinkWriter::for_test(response_receiver, request_sender); + let mut sink = >::for_test(response_receiver, request_sender); let chunk_a = StreamChunk::from_pretty( " i T diff --git a/src/connector/src/sink/test_sink.rs b/src/connector/src/sink/test_sink.rs index d288253eb7a23..96ed91d71bb54 100644 --- a/src/connector/src/sink/test_sink.rs +++ b/src/connector/src/sink/test_sink.rs @@ -14,23 +14,57 @@ use std::sync::{Arc, OnceLock}; +use anyhow::anyhow; use parking_lot::Mutex; -use crate::sink::boxed::BoxSink; -use crate::sink::{SinkError, SinkParam}; +use crate::sink::boxed::{BoxCoordinator, BoxWriter}; +use crate::sink::{Sink, SinkError, SinkParam, SinkWriterParam}; -pub type BuildBoxSink = - Box Result + Send + Sync + 'static>; +pub trait BuildBoxWriterTrait = FnMut(SinkParam, SinkWriterParam) -> BoxWriter<()> + Send + 'static; + +pub type BuildBoxWriter = Box; pub const TEST_SINK_NAME: &str = "test"; +#[derive(Debug)] +pub struct TestSink { + param: SinkParam, +} + +impl TryFrom for TestSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> Result { + if cfg!(any(madsim, test)) { + Ok(TestSink { param }) + } else { + Err(SinkError::Config(anyhow!("test sink only support in test"))) + } + } +} + +impl Sink for TestSink { + type Coordinator = BoxCoordinator; + type Writer = BoxWriter<()>; + + const SINK_NAME: &'static str = "test"; + + async fn validate(&self) -> crate::sink::Result<()> { + Ok(()) + } + + async fn new_writer(&self, writer_param: SinkWriterParam) -> crate::sink::Result { + Ok(build_box_writer(self.param.clone(), writer_param)) + } +} + struct TestSinkRegistry { - build_box_sink: Arc>>, + build_box_writer: Arc>>, } impl TestSinkRegistry { fn new() -> Self { TestSinkRegistry { - build_box_sink: Arc::new(Mutex::new(None)), + build_box_writer: Arc::new(Mutex::new(None)), } } } @@ -44,25 +78,23 @@ pub struct TestSinkRegistryGuard; impl Drop for TestSinkRegistryGuard { fn drop(&mut self) { - assert!(get_registry().build_box_sink.lock().take().is_some()); + assert!(get_registry().build_box_writer.lock().take().is_some()); } } -pub fn registry_build_sink( - build_sink: impl Fn(SinkParam) -> Result + Send + Sync + 'static, -) -> TestSinkRegistryGuard { +pub fn registry_build_sink(build_box_writer: impl BuildBoxWriterTrait) -> TestSinkRegistryGuard { assert!(get_registry() - .build_box_sink + .build_box_writer .lock() - .replace(Box::new(build_sink)) + .replace(Box::new(build_box_writer)) .is_none()); TestSinkRegistryGuard } -pub fn build_test_sink(param: SinkParam) -> Result { +pub fn build_box_writer(param: SinkParam, writer_param: SinkWriterParam) -> BoxWriter<()> { (get_registry() - .build_box_sink + .build_box_writer .lock() - .as_ref() - .expect("should not be empty"))(param) + .as_mut() + .expect("should not be empty"))(param, writer_param) } diff --git a/src/meta/src/stream/sink.rs b/src/meta/src/stream/sink.rs index 4717a1ffdfe95..8544011071ec2 100644 --- a/src/meta/src/stream/sink.rs +++ b/src/meta/src/stream/sink.rs @@ -25,5 +25,5 @@ pub async fn validate_sink(prost_sink_catalog: &PbSink) -> MetaResult<()> { let sink = build_sink(param)?; - dispatch_sink!(sink, sink, { Ok(sink.validate().await?) }) + dispatch_sink!(sink, sink, Ok(sink.validate().await?)) } diff --git a/src/tests/simulation/tests/integration_tests/sink/basic.rs b/src/tests/simulation/tests/integration_tests/sink/basic.rs index a5715a8471c44..f2c6d9f3c8b98 100644 --- a/src/tests/simulation/tests/integration_tests/sink/basic.rs +++ b/src/tests/simulation/tests/integration_tests/sink/basic.rs @@ -74,32 +74,6 @@ impl Drop for TestWriter { } } -struct TestSink { - row_counter: Arc, - parallelism_counter: Arc, -} - -#[async_trait] -impl Sink for TestSink { - type Coordinator = BoxCoordinator; - type Writer = BoxWriter<()>; - - async fn validate(&self) -> risingwave_connector::sink::Result<()> { - Ok(()) - } - - async fn new_writer( - &self, - _writer_param: SinkWriterParam, - ) -> risingwave_connector::sink::Result { - self.parallelism_counter.fetch_add(1, Relaxed); - Ok(Box::new(TestWriter { - parallelism_counter: self.parallelism_counter.clone(), - row_counter: self.row_counter.clone(), - })) - } -} - fn build_stream_chunk(row_iter: impl Iterator) -> StreamChunk { let mut builder = DataChunkBuilder::new(vec![DataType::Int32, DataType::Varchar], 100000); for (id, name) in row_iter { @@ -142,11 +116,12 @@ async fn test_sink_basic() -> Result<()> { let _sink_guard = registry_build_sink({ let row_counter = row_counter.clone(); let parallelism_counter = parallelism_counter.clone(); - move |_param| { - Ok(Box::new(TestSink { + move |_, _| { + parallelism_counter.fetch_add(1, Relaxed); + Box::new(TestWriter { row_counter: row_counter.clone(), parallelism_counter: parallelism_counter.clone(), - })) + }) } }); @@ -239,11 +214,12 @@ async fn test_sink_decouple_basic() -> Result<()> { let _sink_guard = registry_build_sink({ let row_counter = row_counter.clone(); let parallelism_counter = parallelism_counter.clone(); - move |_param| { - Ok(Box::new(TestSink { + move |_, _| { + parallelism_counter.fetch_add(1, Relaxed); + Box::new(TestWriter { row_counter: row_counter.clone(), parallelism_counter: parallelism_counter.clone(), - })) + }) } }); From 1ff58bc53f445d3f01a1ddaab5f4a92f5b337ea9 Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Tue, 26 Sep 2023 16:52:36 +0800 Subject: [PATCH 06/31] refactor(backup): refine error message (#12388) --- proto/backup_service.proto | 6 +-- src/ctl/src/cmd_impl/meta/backup_meta.rs | 22 +++++++-- src/meta/src/backup_restore/backup_manager.rs | 48 +++++++++---------- src/meta/src/rpc/service/backup_service.rs | 5 +- src/rpc_client/src/meta_client.rs | 4 +- 5 files changed, 47 insertions(+), 38 deletions(-) diff --git a/proto/backup_service.proto b/proto/backup_service.proto index 425d3abb24e2f..feca5f17b7dc3 100644 --- a/proto/backup_service.proto +++ b/proto/backup_service.proto @@ -13,11 +13,8 @@ enum BackupJobStatus { UNSPECIFIED = 0; RUNNING = 1; SUCCEEDED = 2; - // NOT_FOUND indicates one of these cases: - // - Invalid job id. - // - Job has failed. - // - Job has succeeded, but its resulted backup has been deleted later. NOT_FOUND = 3; + FAILED = 4; } message BackupMetaRequest {} message BackupMetaResponse { @@ -29,6 +26,7 @@ message GetBackupJobStatusRequest { message GetBackupJobStatusResponse { uint64 job_id = 1; BackupJobStatus job_status = 2; + string message = 3; } message DeleteMetaSnapshotRequest { repeated uint64 snapshot_ids = 1; diff --git a/src/ctl/src/cmd_impl/meta/backup_meta.rs b/src/ctl/src/cmd_impl/meta/backup_meta.rs index 77c7f0edb7ca2..3238e22b35050 100644 --- a/src/ctl/src/cmd_impl/meta/backup_meta.rs +++ b/src/ctl/src/cmd_impl/meta/backup_meta.rs @@ -22,21 +22,33 @@ pub async fn backup_meta(context: &CtlContext) -> anyhow::Result<()> { let meta_client = context.meta_client().await?; let job_id = meta_client.backup_meta().await?; loop { - let job_status = meta_client.get_backup_job_status(job_id).await?; + let (job_status, message) = meta_client.get_backup_job_status(job_id).await?; match job_status { BackupJobStatus::Running => { - tracing::info!("backup job is still running: job {}", job_id); + tracing::info!("backup job is still running: job {}, {}", job_id, message); tokio::time::sleep(Duration::from_secs(1)).await; } BackupJobStatus::Succeeded => { + tracing::info!("backup job succeeded: job {}, {}", job_id, message); break; } - _ => { - return Err(anyhow::anyhow!("backup job failed: job {}", job_id)); + BackupJobStatus::NotFound => { + return Err(anyhow::anyhow!( + "backup job status not found: job {}, {}", + job_id, + message + )); } + BackupJobStatus::Failed => { + return Err(anyhow::anyhow!( + "backup job failed: job {}, {}", + job_id, + message + )); + } + _ => unreachable!("unknown backup job status"), } } - tracing::info!("backup job succeeded: job {}", job_id); Ok(()) } diff --git a/src/meta/src/backup_restore/backup_manager.rs b/src/meta/src/backup_restore/backup_manager.rs index c280572c796d4..819ea02e36346 100644 --- a/src/meta/src/backup_restore/backup_manager.rs +++ b/src/meta/src/backup_restore/backup_manager.rs @@ -68,9 +68,11 @@ pub struct BackupManager { hummock_manager: HummockManagerRef, backup_store: ArcSwap<(BoxedMetaSnapshotStorage, StoreConfig)>, /// Tracks the running backup job. Concurrent jobs is not supported. - running_backup_job: tokio::sync::Mutex>, + running_job_handle: tokio::sync::Mutex>, metrics: BackupManagerMetrics, meta_metrics: Arc, + /// (job id, status, message) + latest_job_info: ArcSwap<(MetaBackupJobId, BackupJobStatus, String)>, } impl BackupManager { @@ -147,9 +149,10 @@ impl BackupManager { env, hummock_manager, backup_store: ArcSwap::from_pointee(backup_store), - running_backup_job: tokio::sync::Mutex::new(None), + running_job_handle: tokio::sync::Mutex::new(None), metrics: BackupManagerMetrics::default(), meta_metrics, + latest_job_info: ArcSwap::from_pointee((0, BackupJobStatus::NotFound, "".into())), } } @@ -181,7 +184,7 @@ impl BackupManager { /// Starts a backup job in background. It's non-blocking. /// Returns job id. pub async fn start_backup_job(self: &Arc) -> MetaResult { - let mut guard = self.running_backup_job.lock().await; + let mut guard = self.running_job_handle.lock().await; if let Some(job) = (*guard).as_ref() { bail!(format!( "concurrent backup job is not supported: existent job {}", @@ -213,6 +216,8 @@ impl BackupManager { .id_gen_manager() .generate::<{ IdCategory::Backup }>() .await?; + self.latest_job_info + .store(Arc::new((job_id, BackupJobStatus::Running, "".into()))); let hummock_version_safe_point = self.hummock_manager.register_safe_point().await; // Ideally `BackupWorker` and its r/w IO can be made external to meta node. // The justification of keeping `BackupWorker` in meta node are: @@ -227,27 +232,12 @@ impl BackupManager { Ok(job_id) } - pub async fn get_backup_job_status( - &self, - job_id: MetaBackupJobId, - ) -> MetaResult { - if let Some(running_job) = self.running_backup_job.lock().await.as_ref() { - if running_job.job_id == job_id { - return Ok(BackupJobStatus::Running); - } - } - if self - .backup_store - .load() - .0 - .manifest() - .snapshot_metadata - .iter() - .any(|m| m.id == job_id) - { - return Ok(BackupJobStatus::Succeeded); + pub fn get_backup_job_status(&self, job_id: MetaBackupJobId) -> (BackupJobStatus, String) { + let last = self.latest_job_info.load(); + if last.0 == job_id { + return (last.1, last.2.clone()); } - Ok(BackupJobStatus::NotFound) + (BackupJobStatus::NotFound, "".into()) } async fn finish_backup_job(&self, job_id: MetaBackupJobId, job_result: BackupJobResult) { @@ -269,16 +259,24 @@ impl BackupManager { id: self.backup_store.load().0.manifest().manifest_id, }), ); + self.latest_job_info.store(Arc::new(( + job_id, + BackupJobStatus::Succeeded, + "".into(), + ))); } BackupJobResult::Failed(e) => { self.metrics.job_latency_failure.observe(job_latency); - tracing::warn!("failed backup job {}: {}", job_id, e); + let message = format!("failed backup job {}: {}", job_id, e); + tracing::warn!(message); + self.latest_job_info + .store(Arc::new((job_id, BackupJobStatus::Failed, message))); } } } async fn take_job_handle_by_job_id(&self, job_id: u64) -> Option { - let mut guard = self.running_backup_job.lock().await; + let mut guard = self.running_job_handle.lock().await; match (*guard).as_ref() { None => { return None; diff --git a/src/meta/src/rpc/service/backup_service.rs b/src/meta/src/rpc/service/backup_service.rs index 22897d8bb770e..d83b4d0d1e8e4 100644 --- a/src/meta/src/rpc/service/backup_service.rs +++ b/src/meta/src/rpc/service/backup_service.rs @@ -49,10 +49,11 @@ impl BackupService for BackupServiceImpl { request: Request, ) -> Result, Status> { let job_id = request.into_inner().job_id; - let job_status = self.backup_manager.get_backup_job_status(job_id).await? as _; + let (job_status, message) = self.backup_manager.get_backup_job_status(job_id); Ok(Response::new(GetBackupJobStatusResponse { job_id, - job_status, + job_status: job_status as _, + message, })) } diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 357cd4cf37f1d..35f69aaa74fbd 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -936,10 +936,10 @@ impl MetaClient { Ok(resp.job_id) } - pub async fn get_backup_job_status(&self, job_id: u64) -> Result { + pub async fn get_backup_job_status(&self, job_id: u64) -> Result<(BackupJobStatus, String)> { let req = GetBackupJobStatusRequest { job_id }; let resp = self.inner.get_backup_job_status(req).await?; - Ok(resp.job_status()) + Ok((resp.job_status(), resp.message)) } pub async fn delete_meta_snapshot(&self, snapshot_ids: &[u64]) -> Result<()> { From 471bfa2c91fb4c32e94171afb49771de294cf741 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 26 Sep 2023 17:21:47 +0800 Subject: [PATCH 07/31] feat(sink): turn sink writer into higher level log sinker (#12152) --- Cargo.lock | 1 + src/connector/Cargo.toml | 1 + src/connector/src/sink/blackhole.rs | 68 +++++ src/connector/src/sink/clickhouse.rs | 13 +- src/connector/src/sink/coordinate.rs | 3 +- src/connector/src/sink/doris.rs | 10 +- src/connector/src/sink/iceberg.rs | 13 +- src/connector/src/sink/kafka.rs | 16 +- src/connector/src/sink/kinesis.rs | 16 +- .../src/sink/log_store.rs} | 77 +++-- src/connector/src/sink/mod.rs | 252 +++------------- src/connector/src/sink/nats.rs | 11 +- src/connector/src/sink/pulsar.rs | 12 +- src/connector/src/sink/redis.rs | 5 +- src/connector/src/sink/remote.rs | 25 +- src/connector/src/sink/test_sink.rs | 11 +- src/connector/src/sink/writer.rs | 270 ++++++++++++++++++ .../{log_store => log_store_impl}/in_mem.rs | 31 +- .../kv_log_store/buffer.rs | 4 +- .../kv_log_store/mod.rs | 20 +- .../kv_log_store/reader.rs | 20 +- .../kv_log_store/serde.rs | 59 ++-- .../kv_log_store/test_utils.rs | 0 .../kv_log_store/writer.rs | 8 +- src/stream/src/common/log_store_impl/mod.rs | 16 ++ src/stream/src/common/mod.rs | 2 +- src/stream/src/error.rs | 10 + src/stream/src/executor/error.rs | 11 - src/stream/src/executor/sink.rs | 137 ++------- src/stream/src/from_proto/sink.rs | 44 ++- .../tests/integration_tests/sink/basic.rs | 3 +- 31 files changed, 677 insertions(+), 492 deletions(-) create mode 100644 src/connector/src/sink/blackhole.rs rename src/{stream/src/common/log_store/mod.rs => connector/src/sink/log_store.rs} (80%) create mode 100644 src/connector/src/sink/writer.rs rename src/stream/src/common/{log_store => log_store_impl}/in_mem.rs (96%) rename src/stream/src/common/{log_store => log_store_impl}/kv_log_store/buffer.rs (98%) rename src/stream/src/common/{log_store => log_store_impl}/kv_log_store/mod.rs (96%) rename src/stream/src/common/{log_store => log_store_impl}/kv_log_store/reader.rs (96%) rename src/stream/src/common/{log_store => log_store_impl}/kv_log_store/serde.rs (96%) rename src/stream/src/common/{log_store => log_store_impl}/kv_log_store/test_utils.rs (100%) rename src/stream/src/common/{log_store => log_store_impl}/kv_log_store/writer.rs (94%) create mode 100644 src/stream/src/common/log_store_impl/mod.rs diff --git a/Cargo.lock b/Cargo.lock index 919c5a234dc00..d46a420f8b71c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6889,6 +6889,7 @@ dependencies = [ "criterion", "csv", "duration-str", + "easy-ext", "enum-as-inner", "futures", "futures-async-stream", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 8ae92464caaaa..c919bd5ed2397 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -46,6 +46,7 @@ clickhouse = { git = "https://github.com/risingwavelabs/clickhouse.rs", rev = "6 ] } csv = "1.2" duration-str = "0.5.1" +easy-ext = "1" enum-as-inner = "0.6" futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } diff --git a/src/connector/src/sink/blackhole.rs b/src/connector/src/sink/blackhole.rs new file mode 100644 index 0000000000000..1f1ace3b0d104 --- /dev/null +++ b/src/connector/src/sink/blackhole.rs @@ -0,0 +1,68 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use crate::sink::log_store::{LogReader, LogStoreReadItem, TruncateOffset}; +use crate::sink::{ + DummySinkCommitCoordinator, LogSinker, Result, Sink, SinkError, SinkParam, SinkWriterParam, +}; + +pub const BLACKHOLE_SINK: &str = "blackhole"; + +#[derive(Debug)] +pub struct BlackHoleSink; + +impl TryFrom for BlackHoleSink { + type Error = SinkError; + + fn try_from(_value: SinkParam) -> std::result::Result { + Ok(Self) + } +} + +impl Sink for BlackHoleSink { + type Coordinator = DummySinkCommitCoordinator; + type LogSinker = Self; + + const SINK_NAME: &'static str = BLACKHOLE_SINK; + + async fn new_log_sinker(&self, _writer_env: SinkWriterParam) -> Result { + Ok(Self) + } + + async fn validate(&self) -> Result<()> { + Ok(()) + } +} + +impl LogSinker for BlackHoleSink { + async fn consume_log_and_sink(self, mut log_reader: impl LogReader) -> Result<()> { + log_reader.init().await?; + loop { + let (epoch, item) = log_reader.next_item().await?; + match item { + LogStoreReadItem::StreamChunk { chunk_id, .. } => { + log_reader + .truncate(TruncateOffset::Chunk { epoch, chunk_id }) + .await?; + } + LogStoreReadItem::Barrier { .. } => { + log_reader + .truncate(TruncateOffset::Barrier { epoch }) + .await?; + } + LogStoreReadItem::UpdateVnodeBitmap(_) => {} + } + } + } +} diff --git a/src/connector/src/sink/clickhouse.rs b/src/connector/src/sink/clickhouse.rs index 08287b789eaf9..b9733863feccb 100644 --- a/src/connector/src/sink/clickhouse.rs +++ b/src/connector/src/sink/clickhouse.rs @@ -29,9 +29,9 @@ use serde_with::serde_as; use super::{DummySinkCommitCoordinator, SinkWriterParam}; use crate::common::ClickHouseCommon; +use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; use crate::sink::{ - Result, Sink, SinkError, SinkParam, SinkWriter, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, - SINK_TYPE_UPSERT, + Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; pub const CLICKHOUSE_SINK: &str = "clickhouse"; @@ -209,7 +209,7 @@ impl ClickHouseSink { } impl Sink for ClickHouseSink { type Coordinator = DummySinkCommitCoordinator; - type Writer = ClickHouseSinkWriter; + type LogSinker = LogSinkerOf; const SINK_NAME: &'static str = CLICKHOUSE_SINK; @@ -243,14 +243,15 @@ impl Sink for ClickHouseSink { Ok(()) } - async fn new_writer(&self, _writer_env: SinkWriterParam) -> Result { - ClickHouseSinkWriter::new( + async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + Ok(ClickHouseSinkWriter::new( self.config.clone(), self.schema.clone(), self.pk_indices.clone(), self.is_append_only, ) - .await + .await? + .into_log_sinker(writer_param.sink_metrics)) } } pub struct ClickHouseSinkWriter { diff --git a/src/connector/src/sink/coordinate.rs b/src/connector/src/sink/coordinate.rs index bbcc7b636b17c..0b233d3c4155a 100644 --- a/src/connector/src/sink/coordinate.rs +++ b/src/connector/src/sink/coordinate.rs @@ -21,7 +21,8 @@ use risingwave_pb::connector_service::SinkMetadata; use risingwave_rpc_client::{CoordinatorStreamHandle, SinkCoordinationRpcClient}; use tracing::warn; -use crate::sink::{Result, SinkError, SinkParam, SinkWriter}; +use crate::sink::writer::SinkWriter; +use crate::sink::{Result, SinkError, SinkParam}; pub struct CoordinatedSinkWriter>> { epoch: u64, diff --git a/src/connector/src/sink/doris.rs b/src/connector/src/sink/doris.rs index 3e226bcb37d46..0a32e4406108b 100644 --- a/src/connector/src/sink/doris.rs +++ b/src/connector/src/sink/doris.rs @@ -29,6 +29,7 @@ use super::doris_connector::{DorisField, DorisInsert, DorisInsertClient, DORIS_D use super::{SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; use crate::common::DorisCommon; use crate::sink::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; +use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{ DummySinkCommitCoordinator, Result, Sink, SinkParam, SinkWriter, SinkWriterParam, }; @@ -155,18 +156,19 @@ impl DorisSink { impl Sink for DorisSink { type Coordinator = DummySinkCommitCoordinator; - type Writer = DorisSinkWriter; + type LogSinker = LogSinkerOf; const SINK_NAME: &'static str = DORIS_SINK; - async fn new_writer(&self, _writer_env: SinkWriterParam) -> Result { - DorisSinkWriter::new( + async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + Ok(DorisSinkWriter::new( self.config.clone(), self.schema.clone(), self.pk_indices.clone(), self.is_append_only, ) - .await + .await? + .into_log_sinker(writer_param.sink_metrics)) } async fn validate(&self) -> Result<()> { diff --git a/src/connector/src/sink/iceberg.rs b/src/connector/src/sink/iceberg.rs index 25c5aa7765114..24249531ffbf5 100644 --- a/src/connector/src/sink/iceberg.rs +++ b/src/connector/src/sink/iceberg.rs @@ -36,12 +36,12 @@ use serde_json::Value; use url::Url; use super::{ - Sink, SinkError, SinkWriter, SinkWriterParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, - SINK_TYPE_UPSERT, + Sink, SinkError, SinkWriterParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; use crate::deserialize_bool_from_string; use crate::sink::coordinate::CoordinatedSinkWriter; use crate::sink::remote::{CoordinatedRemoteSink, RemoteSinkTrait}; +use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; use crate::sink::{Result, SinkCommitCoordinator, SinkParam}; /// This iceberg sink is WIP. When it ready, we will change this name to "iceberg". @@ -257,7 +257,7 @@ impl IcebergSink { impl Sink for IcebergSink { type Coordinator = IcebergSinkCommitter; - type Writer = CoordinatedSinkWriter; + type LogSinker = LogSinkerOf>; const SINK_NAME: &'static str = ICEBERG_SINK; @@ -266,7 +266,7 @@ impl Sink for IcebergSink { Ok(()) } - async fn new_writer(&self, writer_param: SinkWriterParam) -> Result { + async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { let table = self.create_table().await?; let inner = IcebergWriter { @@ -277,7 +277,7 @@ impl Sink for IcebergSink { .map_err(|err| SinkError::Iceberg(anyhow!(err)))?, table, }; - CoordinatedSinkWriter::new( + Ok(CoordinatedSinkWriter::new( writer_param .meta_client .expect("should have meta client") @@ -291,7 +291,8 @@ impl Sink for IcebergSink { })?, inner, ) - .await + .await? + .into_log_sinker(writer_param.sink_metrics)) } async fn new_coordinator( diff --git a/src/connector/src/sink/kafka.rs b/src/connector/src/sink/kafka.rs index 0cad48bbb57f6..a5a524048bfd1 100644 --- a/src/connector/src/sink/kafka.rs +++ b/src/connector/src/sink/kafka.rs @@ -32,14 +32,15 @@ use serde_with::{serde_as, DisplayFromStr}; use strum_macros::{Display, EnumString}; use super::{ - FormattedSink, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_DEBEZIUM, - SINK_TYPE_OPTION, SINK_TYPE_UPSERT, + Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_DEBEZIUM, SINK_TYPE_OPTION, + SINK_TYPE_UPSERT, }; use crate::common::KafkaCommon; use crate::sink::formatter::SinkFormatterImpl; -use crate::sink::{ - DummySinkCommitCoordinator, Result, SinkWriterParam, SinkWriterV1, SinkWriterV1Adapter, +use crate::sink::writer::{ + FormattedSink, LogSinkerOf, SinkWriterExt, SinkWriterV1, SinkWriterV1Adapter, }; +use crate::sink::{DummySinkCommitCoordinator, Result, SinkWriterParam}; use crate::source::kafka::{KafkaProperties, KafkaSplitEnumerator, PrivateLinkProducerContext}; use crate::source::{SourceEnumeratorContext, SplitEnumerator}; use crate::{ @@ -300,11 +301,11 @@ impl TryFrom for KafkaSink { impl Sink for KafkaSink { type Coordinator = DummySinkCommitCoordinator; - type Writer = SinkWriterV1Adapter; + type LogSinker = LogSinkerOf>; const SINK_NAME: &'static str = KAFKA_SINK; - async fn new_writer(&self, _writer_param: SinkWriterParam) -> Result { + async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { Ok(SinkWriterV1Adapter::new( KafkaSinkWriter::new( self.config.clone(), @@ -318,7 +319,8 @@ impl Sink for KafkaSink { )?, ) .await?, - )) + ) + .into_log_sinker(writer_param.sink_metrics)) } async fn validate(&self) -> Result<()> { diff --git a/src/connector/src/sink/kinesis.rs b/src/connector/src/sink/kinesis.rs index f16ca3905b884..bf024ba53b252 100644 --- a/src/connector/src/sink/kinesis.rs +++ b/src/connector/src/sink/kinesis.rs @@ -26,13 +26,14 @@ use serde_with::serde_as; use tokio_retry::strategy::{jitter, ExponentialBackoff}; use tokio_retry::Retry; -use super::{FormattedSink, SinkParam}; +use super::SinkParam; use crate::common::KinesisCommon; use crate::dispatch_sink_formatter_impl; use crate::sink::formatter::SinkFormatterImpl; +use crate::sink::writer::{FormattedSink, LogSinkerOf, SinkWriter, SinkWriterExt}; use crate::sink::{ - DummySinkCommitCoordinator, Result, Sink, SinkError, SinkWriter, SinkWriterParam, - SINK_TYPE_APPEND_ONLY, SINK_TYPE_DEBEZIUM, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, + DummySinkCommitCoordinator, Result, Sink, SinkError, SinkWriterParam, SINK_TYPE_APPEND_ONLY, + SINK_TYPE_DEBEZIUM, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; pub const KINESIS_SINK: &str = "kinesis"; @@ -66,7 +67,7 @@ impl TryFrom for KinesisSink { impl Sink for KinesisSink { type Coordinator = DummySinkCommitCoordinator; - type Writer = KinesisSinkWriter; + type LogSinker = LogSinkerOf; const SINK_NAME: &'static str = KINESIS_SINK; @@ -93,8 +94,8 @@ impl Sink for KinesisSink { Ok(()) } - async fn new_writer(&self, _writer_env: SinkWriterParam) -> Result { - KinesisSinkWriter::new( + async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + Ok(KinesisSinkWriter::new( self.config.clone(), self.schema.clone(), self.pk_indices.clone(), @@ -102,7 +103,8 @@ impl Sink for KinesisSink { self.db_name.clone(), self.sink_from_name.clone(), ) - .await + .await? + .into_log_sinker(writer_param.sink_metrics)) } } diff --git a/src/stream/src/common/log_store/mod.rs b/src/connector/src/sink/log_store.rs similarity index 80% rename from src/stream/src/common/log_store/mod.rs rename to src/connector/src/sink/log_store.rs index 35f1a4145ec55..d8dae1db0ce19 100644 --- a/src/stream/src/common/log_store/mod.rs +++ b/src/connector/src/sink/log_store.rs @@ -12,9 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub mod in_mem; -pub mod kv_log_store; - use std::cmp::Ordering; use std::fmt::Debug; use std::future::Future; @@ -24,25 +21,8 @@ use anyhow::anyhow; use risingwave_common::array::StreamChunk; use risingwave_common::buffer::Bitmap; use risingwave_common::util::epoch::EpochPair; -use risingwave_common::util::value_encoding::error::ValueEncodingError; -use risingwave_storage::error::StorageError; - -#[derive(thiserror::Error, Debug)] -pub enum LogStoreError { - #[error("EndOfLogStream")] - EndOfLogStream, - - #[error("Storage error: {0}")] - StorageError(#[from] StorageError), - - #[error(transparent)] - Internal(#[from] anyhow::Error), - - #[error("Value encoding error: {0}")] - ValueEncoding(#[from] ValueEncodingError), -} -pub type LogStoreResult = Result; +pub type LogStoreResult = Result; pub type ChunkId = usize; #[derive(Debug, PartialEq, Copy, Clone)] @@ -88,8 +68,7 @@ impl TruncateOffset { "new item epoch {} not match current chunk offset epoch {}", epoch, offset_epoch - ) - .into()); + )); } } TruncateOffset::Barrier { @@ -100,8 +79,7 @@ impl TruncateOffset { "new item epoch {} not exceed barrier offset epoch {}", epoch, offset_epoch - ) - .into()); + )); } } } @@ -145,7 +123,7 @@ pub trait LogWriter { ) -> impl Future> + Send + '_; } -pub trait LogReader { +pub trait LogReader: Send + Sized + 'static { /// Initialize the log reader. Usually function as waiting for log writer to be initialized. fn init(&mut self) -> impl Future> + Send + '_; @@ -169,9 +147,54 @@ pub trait LogStoreFactory: 'static { fn build(self) -> impl Future + Send; } +pub struct TransformChunkLogReader StreamChunk, R: LogReader> { + f: F, + inner: R, +} + +impl StreamChunk + Send + 'static, R: LogReader> LogReader + for TransformChunkLogReader +{ + fn init(&mut self) -> impl Future> + Send + '_ { + self.inner.init() + } + + async fn next_item(&mut self) -> LogStoreResult<(u64, LogStoreReadItem)> { + let (epoch, item) = self.inner.next_item().await?; + let item = match item { + LogStoreReadItem::StreamChunk { chunk, chunk_id } => LogStoreReadItem::StreamChunk { + chunk: (self.f)(chunk), + chunk_id, + }, + other => other, + }; + Ok((epoch, item)) + } + + fn truncate( + &mut self, + offset: TruncateOffset, + ) -> impl Future> + Send + '_ { + self.inner.truncate(offset) + } +} + +#[easy_ext::ext(LogStoreTransformChunkLogReader)] +impl T +where + T: LogReader, +{ + pub fn transform_chunk StreamChunk + Sized>( + self, + f: F, + ) -> TransformChunkLogReader { + TransformChunkLogReader { f, inner: self } + } +} + #[cfg(test)] mod tests { - use crate::common::log_store::TruncateOffset; + use crate::sink::log_store::TruncateOffset; #[test] fn test_truncate_offset_cmp() { diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index f312453481db6..c67fdb8c1983e 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod blackhole; pub mod boxed; pub mod catalog; pub mod clickhouse; @@ -23,20 +24,22 @@ pub mod formatter; pub mod iceberg; pub mod kafka; pub mod kinesis; +pub mod log_store; pub mod nats; pub mod pulsar; pub mod redis; pub mod remote; pub mod test_sink; pub mod utils; +pub mod writer; use std::collections::HashMap; -use std::sync::Arc; +use std::future::Future; use ::clickhouse::error::Error as ClickHouseError; use anyhow::anyhow; use async_trait::async_trait; -use risingwave_common::array::StreamChunk; +use prometheus::{Histogram, HistogramOpts}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::{ColumnDesc, Field, Schema}; use risingwave_common::error::{anyhow_error, ErrorCode, RwError}; @@ -48,18 +51,9 @@ use thiserror::Error; pub use tracing; use self::catalog::SinkType; -use self::encoder::SerTo; -use self::formatter::SinkFormatter; use crate::sink::catalog::{SinkCatalog, SinkId}; -pub use crate::sink::clickhouse::ClickHouseSink; -pub use crate::sink::iceberg::{IcebergSink, RemoteIcebergSink}; -pub use crate::sink::kafka::KafkaSink; -pub use crate::sink::kinesis::KinesisSink; -pub use crate::sink::nats::NatsSink; -pub use crate::sink::pulsar::PulsarSink; -pub use crate::sink::redis::RedisSink; -pub use crate::sink::remote::{CassandraSink, DeltaLakeSink, ElasticSearchSink, JdbcSink}; -pub use crate::sink::test_sink::TestSink; +use crate::sink::log_store::LogReader; +use crate::sink::writer::SinkWriter; use crate::ConnectorParams; #[macro_export] @@ -67,21 +61,21 @@ macro_rules! for_all_sinks { ($macro:path $(, $arg:tt)*) => { $macro! { { - { Redis, $crate::sink::RedisSink }, - { Kafka, $crate::sink::KafkaSink }, - { Pulsar, $crate::sink::PulsarSink }, - { BlackHole, $crate::sink::BlackHoleSink }, - { Kinesis, $crate::sink::KinesisSink }, - { ClickHouse, $crate::sink::ClickHouseSink }, - { Iceberg, $crate::sink::IcebergSink }, - { Nats, $crate::sink::NatsSink }, - { RemoteIceberg, $crate::sink::RemoteIcebergSink }, - { Jdbc, $crate::sink::JdbcSink }, - { DeltaLake, $crate::sink::DeltaLakeSink }, - { ElasticSearch, $crate::sink::ElasticSearchSink }, - { Cassandra, $crate::sink::CassandraSink }, + { Redis, $crate::sink::redis::RedisSink }, + { Kafka, $crate::sink::kafka::KafkaSink }, + { Pulsar, $crate::sink::pulsar::PulsarSink }, + { BlackHole, $crate::sink::blackhole::BlackHoleSink }, + { Kinesis, $crate::sink::kinesis::KinesisSink }, + { ClickHouse, $crate::sink::clickhouse::ClickHouseSink }, + { Iceberg, $crate::sink::iceberg::IcebergSink }, + { Nats, $crate::sink::nats::NatsSink }, + { RemoteIceberg, $crate::sink::iceberg::RemoteIcebergSink }, + { Jdbc, $crate::sink::remote::JdbcSink }, + { DeltaLake, $crate::sink::remote::DeltaLakeSink }, + { ElasticSearch, $crate::sink::remote::ElasticSearchSink }, + { Cassandra, $crate::sink::remote::CassandraSink }, { Doris, $crate::sink::doris::DorisSink }, - { Test, $crate::sink::TestSink } + { Test, $crate::sink::test_sink::TestSink } } $(,$arg)* } @@ -124,7 +118,7 @@ macro_rules! match_sink_name_str { }}; } -pub const DOWNSTREAM_SINK_KEY: &str = "connector"; +pub const CONNECTOR_TYPE_KEY: &str = "connector"; pub const SINK_TYPE_OPTION: &str = "type"; pub const SINK_TYPE_APPEND_ONLY: &str = "append-only"; pub const SINK_TYPE_DEBEZIUM: &str = "debezium"; @@ -201,21 +195,38 @@ impl From for SinkParam { } } +#[derive(Clone)] +pub struct SinkMetrics { + pub sink_commit_duration_metrics: Histogram, +} + +impl Default for SinkMetrics { + fn default() -> Self { + SinkMetrics { + sink_commit_duration_metrics: Histogram::with_opts(HistogramOpts::new( + "unused", "unused", + )) + .unwrap(), + } + } +} + #[derive(Clone, Default)] pub struct SinkWriterParam { pub connector_params: ConnectorParams, pub executor_id: u64, pub vnode_bitmap: Option, pub meta_client: Option, + pub sink_metrics: SinkMetrics, } pub trait Sink: TryFrom { const SINK_NAME: &'static str; - type Writer: SinkWriter; + type LogSinker: LogSinker; type Coordinator: SinkCommitCoordinator; async fn validate(&self) -> Result<()>; - async fn new_writer(&self, writer_param: SinkWriterParam) -> Result; + async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result; #[expect(clippy::unused_async)] async fn new_coordinator( &self, @@ -225,129 +236,11 @@ pub trait Sink: TryFrom { } } -#[async_trait] -pub trait SinkWriter: Send + 'static { - type CommitMetadata: Send = (); - /// Begin a new epoch - async fn begin_epoch(&mut self, epoch: u64) -> Result<()>; - - /// Write a stream chunk to sink - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()>; - - /// Receive a barrier and mark the end of current epoch. When `is_checkpoint` is true, the sink - /// writer should commit the current epoch. - async fn barrier(&mut self, is_checkpoint: bool) -> Result; - - /// Clean up - async fn abort(&mut self) -> Result<()> { - Ok(()) - } - - /// Update the vnode bitmap of current sink writer - async fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc) -> Result<()> { - Ok(()) - } -} - -#[async_trait] -// An old version of SinkWriter for backward compatibility -pub trait SinkWriterV1: Send + 'static { - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()>; - - // the following interface is for transactions, if not supported, return Ok(()) - // start a transaction with epoch number. Note that epoch number should be increasing. - async fn begin_epoch(&mut self, epoch: u64) -> Result<()>; - - // commits the current transaction and marks all messages in the transaction success. - async fn commit(&mut self) -> Result<()>; - - // aborts the current transaction because some error happens. we should rollback to the last - // commit point. - async fn abort(&mut self) -> Result<()>; -} - -/// A free-form sink that may output in multiple formats and encodings. Examples include kafka, -/// kinesis, nats and redis. -/// -/// The implementor specifies required key & value type (likely string or bytes), as well as how to -/// write a single pair. The provided `write_chunk` method would handle the interaction with a -/// `SinkFormatter`. -/// -/// Currently kafka takes `&mut self` while kinesis takes `&self`. So we use `&mut self` in trait -/// but implement it for `&Kinesis`. This allows us to hold `&mut &Kinesis` and `&Kinesis` -/// simultaneously, preventing the schema clone issue propagating from kafka to kinesis. -pub trait FormattedSink { - type K; - type V; - async fn write_one(&mut self, k: Option, v: Option) -> Result<()>; - - async fn write_chunk( - &mut self, - chunk: StreamChunk, - formatter: &F, - ) -> Result<()> - where - F::K: SerTo, - F::V: SerTo, - { - for r in formatter.format_chunk(&chunk) { - let (event_key_object, event_object) = r?; - - self.write_one( - event_key_object.map(SerTo::ser_to).transpose()?, - event_object.map(SerTo::ser_to).transpose()?, - ) - .await?; - } - - Ok(()) - } -} - -pub struct SinkWriterV1Adapter { - is_empty: bool, - epoch: u64, - inner: W, -} - -impl SinkWriterV1Adapter { - pub(crate) fn new(inner: W) -> Self { - Self { - inner, - is_empty: true, - epoch: u64::MIN, - } - } -} - -#[async_trait] -impl SinkWriter for SinkWriterV1Adapter { - async fn begin_epoch(&mut self, epoch: u64) -> Result<()> { - self.epoch = epoch; - Ok(()) - } - - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { - if self.is_empty { - self.is_empty = false; - self.inner.begin_epoch(self.epoch).await?; - } - self.inner.write_batch(chunk).await - } - - async fn barrier(&mut self, is_checkpoint: bool) -> Result<()> { - if is_checkpoint { - if !self.is_empty { - self.inner.commit().await? - } - self.is_empty = true; - } - Ok(()) - } - - async fn abort(&mut self) -> Result<()> { - self.inner.abort().await - } +pub trait LogSinker: Send + 'static { + fn consume_log_and_sink( + self, + log_reader: impl LogReader, + ) -> impl Future> + Send + 'static; } #[async_trait] @@ -374,52 +267,8 @@ impl SinkCommitCoordinator for DummySinkCommitCoordinator { } } -pub const BLACKHOLE_SINK: &str = "blackhole"; - -#[derive(Debug)] -pub struct BlackHoleSink; - -impl TryFrom for BlackHoleSink { - type Error = SinkError; - - fn try_from(_value: SinkParam) -> std::result::Result { - Ok(Self) - } -} - -impl Sink for BlackHoleSink { - type Coordinator = DummySinkCommitCoordinator; - type Writer = Self; - - const SINK_NAME: &'static str = BLACKHOLE_SINK; - - async fn new_writer(&self, _writer_env: SinkWriterParam) -> Result { - Ok(Self) - } - - async fn validate(&self) -> Result<()> { - Ok(()) - } -} - -#[async_trait] -impl SinkWriter for BlackHoleSink { - async fn write_batch(&mut self, _chunk: StreamChunk) -> Result<()> { - Ok(()) - } - - async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { - Ok(()) - } - - async fn barrier(&mut self, _is_checkpoint: bool) -> Result<()> { - Ok(()) - } -} - impl SinkImpl { pub fn new(mut param: SinkParam) -> Result { - const CONNECTOR_TYPE_KEY: &str = "connector"; const CONNECTION_NAME_KEY: &str = "connection.name"; const PRIVATE_LINK_TARGET_KEY: &str = "privatelink.targets"; @@ -473,15 +322,6 @@ macro_rules! def_sink_impl { def_sink_impl!(); -impl SinkImpl { - pub fn get_connector(&self) -> &'static str { - fn get_name(_: &S) -> &'static str { - S::SINK_NAME - } - dispatch_sink!(self, sink, get_name(sink)) - } -} - pub type Result = std::result::Result; #[derive(Error, Debug)] diff --git a/src/connector/src/sink/nats.rs b/src/connector/src/sink/nats.rs index 1a126eb2c74d2..8e3f3e2c18022 100644 --- a/src/connector/src/sink/nats.rs +++ b/src/connector/src/sink/nats.rs @@ -28,6 +28,7 @@ use super::utils::chunk_to_json; use super::{DummySinkCommitCoordinator, SinkWriter, SinkWriterParam}; use crate::common::NatsCommon; use crate::sink::encoder::{JsonEncoder, TimestampHandlingMode}; +use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY}; pub const NATS_SINK: &str = "nats"; @@ -87,7 +88,7 @@ impl TryFrom for NatsSink { impl Sink for NatsSink { type Coordinator = DummySinkCommitCoordinator; - type Writer = NatsSinkWriter; + type LogSinker = LogSinkerOf; const SINK_NAME: &'static str = NATS_SINK; @@ -109,8 +110,12 @@ impl Sink for NatsSink { Ok(()) } - async fn new_writer(&self, _writer_env: SinkWriterParam) -> Result { - NatsSinkWriter::new(self.config.clone(), self.schema.clone()).await + async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + Ok( + NatsSinkWriter::new(self.config.clone(), self.schema.clone()) + .await? + .into_log_sinker(writer_param.sink_metrics), + ) } } diff --git a/src/connector/src/sink/pulsar.rs b/src/connector/src/sink/pulsar.rs index df13c2b4c7d01..148c4e4c9e41e 100644 --- a/src/connector/src/sink/pulsar.rs +++ b/src/connector/src/sink/pulsar.rs @@ -28,11 +28,12 @@ use serde::Deserialize; use serde_with::{serde_as, DisplayFromStr}; use super::{ - FormattedSink, Sink, SinkError, SinkParam, SinkWriter, SinkWriterParam, SINK_TYPE_APPEND_ONLY, + Sink, SinkError, SinkParam, SinkWriter, SinkWriterParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; use crate::common::PulsarCommon; use crate::sink::formatter::SinkFormatterImpl; +use crate::sink::writer::{FormattedSink, LogSinkerOf, SinkWriterExt}; use crate::sink::{DummySinkCommitCoordinator, Result}; use crate::{deserialize_duration_from_string, dispatch_sink_formatter_impl}; @@ -164,12 +165,12 @@ impl TryFrom for PulsarSink { impl Sink for PulsarSink { type Coordinator = DummySinkCommitCoordinator; - type Writer = PulsarSinkWriter; + type LogSinker = LogSinkerOf; const SINK_NAME: &'static str = PULSAR_SINK; - async fn new_writer(&self, _writer_param: SinkWriterParam) -> Result { - PulsarSinkWriter::new( + async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + Ok(PulsarSinkWriter::new( self.config.clone(), self.schema.clone(), self.downstream_pk.clone(), @@ -177,7 +178,8 @@ impl Sink for PulsarSink { self.db_name.clone(), self.sink_from_name.clone(), ) - .await + .await? + .into_log_sinker(writer_param.sink_metrics)) } async fn validate(&self) -> Result<()> { diff --git a/src/connector/src/sink/redis.rs b/src/connector/src/sink/redis.rs index 8cd0875300d29..e85e984821b15 100644 --- a/src/connector/src/sink/redis.rs +++ b/src/connector/src/sink/redis.rs @@ -15,6 +15,7 @@ use async_trait::async_trait; use risingwave_common::array::StreamChunk; +use crate::sink::writer::LogSinkerOf; use crate::sink::{ DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SinkWriter, SinkWriterParam, }; @@ -35,11 +36,11 @@ impl TryFrom for RedisSink { impl Sink for RedisSink { type Coordinator = DummySinkCommitCoordinator; - type Writer = RedisSinkWriter; + type LogSinker = LogSinkerOf; const SINK_NAME: &'static str = "redis"; - async fn new_writer(&self, _writer_env: SinkWriterParam) -> Result { + async fn new_log_sinker(&self, _writer_env: SinkWriterParam) -> Result { todo!() } diff --git a/src/connector/src/sink/remote.rs b/src/connector/src/sink/remote.rs index 51195ac366a27..e7d20d69fbbd4 100644 --- a/src/connector/src/sink/remote.rs +++ b/src/connector/src/sink/remote.rs @@ -42,12 +42,14 @@ use tokio::sync::mpsc::{Sender, UnboundedReceiver}; use tonic::Status; use tracing::{error, warn}; -use super::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; +use super::encoder::{JsonEncoder, RowEncoder}; use crate::sink::coordinate::CoordinatedSinkWriter; +use crate::sink::encoder::TimestampHandlingMode; +use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; use crate::sink::SinkError::Remote; use crate::sink::{ DummySinkCommitCoordinator, Result, Sink, SinkCommitCoordinator, SinkError, SinkParam, - SinkWriter, SinkWriterParam, + SinkWriterParam, }; use crate::ConnectorParams; @@ -97,12 +99,16 @@ impl TryFrom for RemoteSink { impl Sink for RemoteSink { type Coordinator = DummySinkCommitCoordinator; - type Writer = RemoteSinkWriter; + type LogSinker = LogSinkerOf>; const SINK_NAME: &'static str = R::SINK_NAME; - async fn new_writer(&self, writer_param: SinkWriterParam) -> Result { - RemoteSinkWriter::new(self.param.clone(), writer_param.connector_params).await + async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + Ok( + RemoteSinkWriter::new(self.param.clone(), writer_param.connector_params) + .await? + .into_log_sinker(writer_param.sink_metrics), + ) } async fn validate(&self) -> Result<()> { @@ -195,7 +201,7 @@ impl TryFrom for CoordinatedRemoteSink { impl Sink for CoordinatedRemoteSink { type Coordinator = RemoteCoordinator; - type Writer = CoordinatedSinkWriter>; + type LogSinker = LogSinkerOf>>; const SINK_NAME: &'static str = R::SINK_NAME; @@ -203,8 +209,8 @@ impl Sink for CoordinatedRemoteSink { self.0.validate().await } - async fn new_writer(&self, writer_param: SinkWriterParam) -> Result { - CoordinatedSinkWriter::new( + async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + Ok(CoordinatedSinkWriter::new( writer_param .meta_client .expect("should have meta client") @@ -219,7 +225,8 @@ impl Sink for CoordinatedRemoteSink { CoordinatedRemoteSinkWriter::new(self.0.param.clone(), writer_param.connector_params) .await?, ) - .await + .await? + .into_log_sinker(writer_param.sink_metrics)) } async fn new_coordinator( diff --git a/src/connector/src/sink/test_sink.rs b/src/connector/src/sink/test_sink.rs index 96ed91d71bb54..6f327ceaf9cbc 100644 --- a/src/connector/src/sink/test_sink.rs +++ b/src/connector/src/sink/test_sink.rs @@ -18,6 +18,7 @@ use anyhow::anyhow; use parking_lot::Mutex; use crate::sink::boxed::{BoxCoordinator, BoxWriter}; +use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{Sink, SinkError, SinkParam, SinkWriterParam}; pub trait BuildBoxWriterTrait = FnMut(SinkParam, SinkWriterParam) -> BoxWriter<()> + Send + 'static; @@ -44,7 +45,7 @@ impl TryFrom for TestSink { impl Sink for TestSink { type Coordinator = BoxCoordinator; - type Writer = BoxWriter<()>; + type LogSinker = LogSinkerOf>; const SINK_NAME: &'static str = "test"; @@ -52,8 +53,12 @@ impl Sink for TestSink { Ok(()) } - async fn new_writer(&self, writer_param: SinkWriterParam) -> crate::sink::Result { - Ok(build_box_writer(self.param.clone(), writer_param)) + async fn new_log_sinker( + &self, + writer_param: SinkWriterParam, + ) -> crate::sink::Result { + let metrics = writer_param.sink_metrics.clone(); + Ok(build_box_writer(self.param.clone(), writer_param).into_log_sinker(metrics)) } } diff --git a/src/connector/src/sink/writer.rs b/src/connector/src/sink/writer.rs new file mode 100644 index 0000000000000..94f1312853593 --- /dev/null +++ b/src/connector/src/sink/writer.rs @@ -0,0 +1,270 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; +use std::time::Instant; + +use async_trait::async_trait; +use risingwave_common::array::StreamChunk; +use risingwave_common::buffer::Bitmap; + +use crate::sink::encoder::SerTo; +use crate::sink::formatter::SinkFormatter; +use crate::sink::log_store::{LogReader, LogStoreReadItem, TruncateOffset}; +use crate::sink::{LogSinker, Result, SinkMetrics}; + +#[async_trait] +pub trait SinkWriter: Send + 'static { + type CommitMetadata: Send = (); + /// Begin a new epoch + async fn begin_epoch(&mut self, epoch: u64) -> Result<()>; + + /// Write a stream chunk to sink + async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()>; + + /// Receive a barrier and mark the end of current epoch. When `is_checkpoint` is true, the sink + /// writer should commit the current epoch. + async fn barrier(&mut self, is_checkpoint: bool) -> Result; + + /// Clean up + async fn abort(&mut self) -> Result<()> { + Ok(()) + } + + /// Update the vnode bitmap of current sink writer + async fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc) -> Result<()> { + Ok(()) + } +} + +// TODO: remove this trait after KafkaSinkWriter implements SinkWriter +#[async_trait] +// An old version of SinkWriter for backward compatibility +pub trait SinkWriterV1: Send + 'static { + async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()>; + + // the following interface is for transactions, if not supported, return Ok(()) + // start a transaction with epoch number. Note that epoch number should be increasing. + async fn begin_epoch(&mut self, epoch: u64) -> Result<()>; + + // commits the current transaction and marks all messages in the transaction success. + async fn commit(&mut self) -> Result<()>; + + // aborts the current transaction because some error happens. we should rollback to the last + // commit point. + async fn abort(&mut self) -> Result<()>; +} + +/// A free-form sink that may output in multiple formats and encodings. Examples include kafka, +/// kinesis, nats and redis. +/// +/// The implementor specifies required key & value type (likely string or bytes), as well as how to +/// write a single pair. The provided `write_chunk` method would handle the interaction with a +/// `SinkFormatter`. +/// +/// Currently kafka takes `&mut self` while kinesis takes `&self`. So we use `&mut self` in trait +/// but implement it for `&Kinesis`. This allows us to hold `&mut &Kinesis` and `&Kinesis` +/// simultaneously, preventing the schema clone issue propagating from kafka to kinesis. +pub trait FormattedSink { + type K; + type V; + async fn write_one(&mut self, k: Option, v: Option) -> Result<()>; + + async fn write_chunk( + &mut self, + chunk: StreamChunk, + formatter: &F, + ) -> Result<()> + where + F::K: SerTo, + F::V: SerTo, + { + for r in formatter.format_chunk(&chunk) { + let (event_key_object, event_object) = r?; + + self.write_one( + event_key_object.map(SerTo::ser_to).transpose()?, + event_object.map(SerTo::ser_to).transpose()?, + ) + .await?; + } + + Ok(()) + } +} + +pub struct SinkWriterV1Adapter { + is_empty: bool, + epoch: u64, + inner: W, +} + +impl SinkWriterV1Adapter { + pub(crate) fn new(inner: W) -> Self { + Self { + inner, + is_empty: true, + epoch: u64::MIN, + } + } +} + +#[async_trait] +impl SinkWriter for SinkWriterV1Adapter { + async fn begin_epoch(&mut self, epoch: u64) -> Result<()> { + self.epoch = epoch; + Ok(()) + } + + async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { + if self.is_empty { + self.is_empty = false; + self.inner.begin_epoch(self.epoch).await?; + } + self.inner.write_batch(chunk).await + } + + async fn barrier(&mut self, is_checkpoint: bool) -> Result<()> { + if is_checkpoint { + if !self.is_empty { + self.inner.commit().await? + } + self.is_empty = true; + } + Ok(()) + } + + async fn abort(&mut self) -> Result<()> { + self.inner.abort().await + } +} + +pub struct LogSinkerOf> { + writer: W, + sink_metrics: SinkMetrics, +} + +impl> LogSinkerOf { + pub fn new(writer: W, sink_metrics: SinkMetrics) -> Self { + LogSinkerOf { + writer, + sink_metrics, + } + } +} + +impl> LogSinker for LogSinkerOf { + async fn consume_log_and_sink(self, mut log_reader: impl LogReader) -> Result<()> { + let mut sink_writer = self.writer; + let sink_metrics = self.sink_metrics; + #[derive(Debug)] + enum LogConsumerState { + /// Mark that the log consumer is not initialized yet + Uninitialized, + + /// Mark that a new epoch has begun. + EpochBegun { curr_epoch: u64 }, + + /// Mark that the consumer has just received a barrier + BarrierReceived { prev_epoch: u64 }, + } + + let mut state = LogConsumerState::Uninitialized; + + log_reader.init().await?; + + loop { + let (epoch, item): (u64, LogStoreReadItem) = log_reader.next_item().await?; + if let LogStoreReadItem::UpdateVnodeBitmap(_) = &item { + match &state { + LogConsumerState::BarrierReceived { .. } => {} + _ => unreachable!( + "update vnode bitmap can be accepted only right after \ + barrier, but current state is {:?}", + state + ), + } + } + // begin_epoch when not previously began + state = match state { + LogConsumerState::Uninitialized => { + sink_writer.begin_epoch(epoch).await?; + LogConsumerState::EpochBegun { curr_epoch: epoch } + } + LogConsumerState::EpochBegun { curr_epoch } => { + assert!( + epoch >= curr_epoch, + "new epoch {} should not be below the current epoch {}", + epoch, + curr_epoch + ); + LogConsumerState::EpochBegun { curr_epoch: epoch } + } + LogConsumerState::BarrierReceived { prev_epoch } => { + assert!( + epoch > prev_epoch, + "new epoch {} should be greater than prev epoch {}", + epoch, + prev_epoch + ); + sink_writer.begin_epoch(epoch).await?; + LogConsumerState::EpochBegun { curr_epoch: epoch } + } + }; + match item { + LogStoreReadItem::StreamChunk { chunk, .. } => { + if let Err(e) = sink_writer.write_batch(chunk).await { + sink_writer.abort().await?; + return Err(e); + } + } + LogStoreReadItem::Barrier { is_checkpoint } => { + let prev_epoch = match state { + LogConsumerState::EpochBegun { curr_epoch } => curr_epoch, + _ => unreachable!("epoch must have begun before handling barrier"), + }; + if is_checkpoint { + let start_time = Instant::now(); + sink_writer.barrier(true).await?; + sink_metrics + .sink_commit_duration_metrics + .observe(start_time.elapsed().as_millis() as f64); + log_reader + .truncate(TruncateOffset::Barrier { epoch }) + .await?; + } else { + sink_writer.barrier(false).await?; + } + state = LogConsumerState::BarrierReceived { prev_epoch } + } + LogStoreReadItem::UpdateVnodeBitmap(vnode_bitmap) => { + sink_writer.update_vnode_bitmap(vnode_bitmap).await?; + } + } + } + } +} + +#[easy_ext::ext(SinkWriterExt)] +impl T +where + T: SinkWriter + Sized, +{ + pub fn into_log_sinker(self, sink_metrics: SinkMetrics) -> LogSinkerOf { + LogSinkerOf { + writer: self, + sink_metrics, + } + } +} diff --git a/src/stream/src/common/log_store/in_mem.rs b/src/stream/src/common/log_store_impl/in_mem.rs similarity index 96% rename from src/stream/src/common/log_store/in_mem.rs rename to src/stream/src/common/log_store_impl/in_mem.rs index 01192f951b843..35040be82c93b 100644 --- a/src/stream/src/common/log_store/in_mem.rs +++ b/src/stream/src/common/log_store_impl/in_mem.rs @@ -18,16 +18,15 @@ use anyhow::anyhow; use risingwave_common::array::StreamChunk; use risingwave_common::buffer::Bitmap; use risingwave_common::util::epoch::{EpochPair, INVALID_EPOCH}; +use risingwave_connector::sink::log_store::{ + LogReader, LogStoreFactory, LogStoreReadItem, LogStoreResult, LogWriter, TruncateOffset, +}; use tokio::sync::mpsc::{ channel, unbounded_channel, Receiver, Sender, UnboundedReceiver, UnboundedSender, }; use tokio::sync::oneshot; -use crate::common::log_store::in_mem::LogReaderEpochProgress::{AwaitingTruncate, Consuming}; -use crate::common::log_store::{ - LogReader, LogStoreError, LogStoreFactory, LogStoreReadItem, LogStoreResult, LogWriter, - TruncateOffset, -}; +use crate::common::log_store_impl::in_mem::LogReaderEpochProgress::{AwaitingTruncate, Consuming}; enum InMemLogStoreItem { StreamChunk(StreamChunk), @@ -193,10 +192,9 @@ impl LogReader for BoundedInMemLogStoreReader { }, AwaitingTruncate { .. } => Err(anyhow!( "should not call next_item on checkpoint barrier for in-mem log store" - ) - .into()), + )), }, - None => Err(LogStoreError::EndOfLogStream), + None => Err(anyhow!("end of log stream")), } } @@ -207,8 +205,7 @@ impl LogReader for BoundedInMemLogStoreReader { "truncate offset {:?} but prev truncate offset is {:?}", offset, self.truncate_offset - ) - .into()); + )); } // check the truncate offset does not exceed the latest possible offset @@ -217,8 +214,7 @@ impl LogReader for BoundedInMemLogStoreReader { "truncate at {:?} but latest offset is {:?}", offset, self.latest_offset - ) - .into()); + )); } if let AwaitingTruncate { @@ -288,11 +284,10 @@ impl LogWriter for BoundedInMemLogStoreWriter { } async fn update_vnode_bitmap(&mut self, new_vnodes: Arc) -> LogStoreResult<()> { - Ok(self - .item_tx + self.item_tx .send(InMemLogStoreItem::UpdateVnodeBitmap(new_vnodes)) .await - .map_err(|_| anyhow!("unable to send vnode bitmap"))?) + .map_err(|_| anyhow!("unable to send vnode bitmap")) } } @@ -305,11 +300,11 @@ mod tests { use risingwave_common::array::Op; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::epoch::EpochPair; - - use crate::common::log_store::in_mem::BoundedInMemLogStoreFactory; - use crate::common::log_store::{ + use risingwave_connector::sink::log_store::{ LogReader, LogStoreFactory, LogStoreReadItem, LogWriter, TruncateOffset, }; + + use crate::common::log_store_impl::in_mem::BoundedInMemLogStoreFactory; use crate::common::StreamChunkBuilder; #[tokio::test] diff --git a/src/stream/src/common/log_store/kv_log_store/buffer.rs b/src/stream/src/common/log_store_impl/kv_log_store/buffer.rs similarity index 98% rename from src/stream/src/common/log_store/kv_log_store/buffer.rs rename to src/stream/src/common/log_store_impl/kv_log_store/buffer.rs index b478123e6d9cb..fc3a4b569f7da 100644 --- a/src/stream/src/common/log_store/kv_log_store/buffer.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/buffer.rs @@ -19,10 +19,10 @@ use std::sync::Arc; use parking_lot::{Mutex, MutexGuard}; use risingwave_common::array::StreamChunk; use risingwave_common::buffer::Bitmap; +use risingwave_connector::sink::log_store::{ChunkId, LogStoreResult, TruncateOffset}; use tokio::sync::{oneshot, Notify}; -use crate::common::log_store::kv_log_store::{ReaderTruncationOffsetType, SeqIdType}; -use crate::common::log_store::{ChunkId, LogStoreResult, TruncateOffset}; +use crate::common::log_store_impl::kv_log_store::{ReaderTruncationOffsetType, SeqIdType}; #[derive(Clone)] pub(crate) enum LogStoreBufferItem { diff --git a/src/stream/src/common/log_store/kv_log_store/mod.rs b/src/stream/src/common/log_store_impl/kv_log_store/mod.rs similarity index 96% rename from src/stream/src/common/log_store/kv_log_store/mod.rs rename to src/stream/src/common/log_store_impl/kv_log_store/mod.rs index dc27d1f63b3e8..a84850b04f069 100644 --- a/src/stream/src/common/log_store/kv_log_store/mod.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/mod.rs @@ -16,15 +16,15 @@ use std::sync::Arc; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::{TableId, TableOption}; +use risingwave_connector::sink::log_store::LogStoreFactory; use risingwave_pb::catalog::Table; use risingwave_storage::store::NewLocalOptions; use risingwave_storage::StateStore; -use crate::common::log_store::kv_log_store::buffer::new_log_store_buffer; -use crate::common::log_store::kv_log_store::reader::KvLogStoreReader; -use crate::common::log_store::kv_log_store::serde::LogStoreRowSerde; -use crate::common::log_store::kv_log_store::writer::KvLogStoreWriter; -use crate::common::log_store::LogStoreFactory; +use crate::common::log_store_impl::kv_log_store::buffer::new_log_store_buffer; +use crate::common::log_store_impl::kv_log_store::reader::KvLogStoreReader; +use crate::common::log_store_impl::kv_log_store::serde::LogStoreRowSerde; +use crate::common::log_store_impl::kv_log_store::writer::KvLogStoreWriter; mod buffer; mod reader; @@ -102,18 +102,18 @@ impl LogStoreFactory for KvLogStoreFactory { #[cfg(test)] mod tests { use risingwave_common::util::epoch::EpochPair; + use risingwave_connector::sink::log_store::{ + LogReader, LogStoreFactory, LogStoreReadItem, LogWriter, TruncateOffset, + }; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_hummock_test::test_utils::prepare_hummock_test_env; use risingwave_storage::store::SyncResult; use risingwave_storage::StateStore; - use crate::common::log_store::kv_log_store::test_utils::{ + use crate::common::log_store_impl::kv_log_store::test_utils::{ gen_stream_chunk, gen_test_log_store_table, }; - use crate::common::log_store::kv_log_store::KvLogStoreFactory; - use crate::common::log_store::{ - LogReader, LogStoreFactory, LogStoreReadItem, LogWriter, TruncateOffset, - }; + use crate::common::log_store_impl::kv_log_store::KvLogStoreFactory; #[tokio::test] async fn test_basic() { diff --git a/src/stream/src/common/log_store/kv_log_store/reader.rs b/src/stream/src/common/log_store_impl/kv_log_store/reader.rs similarity index 96% rename from src/stream/src/common/log_store/kv_log_store/reader.rs rename to src/stream/src/common/log_store_impl/kv_log_store/reader.rs index 2d01ab3bc0b9d..4336c3d961626 100644 --- a/src/stream/src/common/log_store/kv_log_store/reader.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/reader.rs @@ -22,18 +22,20 @@ use futures::stream::select_all; use risingwave_common::cache::CachePriority; use risingwave_common::catalog::TableId; use risingwave_common::hash::VnodeBitmapExt; +use risingwave_connector::sink::log_store::{ + LogReader, LogStoreReadItem, LogStoreResult, TruncateOffset, +}; use risingwave_hummock_sdk::key::TableKey; use risingwave_storage::hummock::CachePolicy; use risingwave_storage::store::{PrefetchOptions, ReadOptions}; use risingwave_storage::StateStore; use tokio_stream::StreamExt; -use crate::common::log_store::kv_log_store::buffer::{LogStoreBufferItem, LogStoreBufferReceiver}; -use crate::common::log_store::kv_log_store::serde::{ - new_log_store_item_stream, KvLogStoreItem, LogStoreItemStream, LogStoreRowSerde, +use crate::common::log_store_impl::kv_log_store::buffer::{ + LogStoreBufferItem, LogStoreBufferReceiver, }; -use crate::common::log_store::{ - LogReader, LogStoreError, LogStoreReadItem, LogStoreResult, TruncateOffset, +use crate::common::log_store_impl::kv_log_store::serde::{ + new_log_store_item_stream, KvLogStoreItem, LogStoreItemStream, LogStoreRowSerde, }; pub struct KvLogStoreReader { @@ -174,7 +176,7 @@ impl LogReader for KvLogStoreReader { // Use u64::MAX here because the epoch to consume may be below the safe // epoch async move { - Ok::<_, LogStoreError>(Box::pin( + Ok::<_, anyhow::Error>(Box::pin( state_store .iter( (Included(range_start), Included(range_end)), @@ -233,16 +235,14 @@ impl LogReader for KvLogStoreReader { "truncate at a later offset {:?} than the current latest offset {:?}", offset, self.latest_offset - ) - .into()); + )); } if offset <= self.truncate_offset { return Err(anyhow!( "truncate offset {:?} earlier than prev truncate offset {:?}", offset, self.truncate_offset - ) - .into()); + )); } if offset.epoch() >= self.first_write_epoch.expect("should have init") { self.rx.truncate(offset); diff --git a/src/stream/src/common/log_store/kv_log_store/serde.rs b/src/stream/src/common/log_store_impl/kv_log_store/serde.rs similarity index 96% rename from src/stream/src/common/log_store/kv_log_store/serde.rs rename to src/stream/src/common/log_store_impl/kv_log_store/serde.rs index 627aee6c22f2f..2aa01c42af196 100644 --- a/src/stream/src/common/log_store/kv_log_store/serde.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/serde.rs @@ -38,6 +38,7 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_common::util::value_encoding::{ BasicSerde, ValueRowDeserializer, ValueRowSerializer, }; +use risingwave_connector::sink::log_store::LogStoreResult; use risingwave_hummock_sdk::key::{next_key, TableKey}; use risingwave_pb::catalog::Table; use risingwave_storage::row_serde::row_serde_util::serialize_pk_with_vnode; @@ -45,10 +46,9 @@ use risingwave_storage::row_serde::value_serde::ValueRowSerdeNew; use risingwave_storage::store::StateStoreReadIterStream; use risingwave_storage::table::{compute_vnode, Distribution}; -use crate::common::log_store::kv_log_store::{ +use crate::common::log_store_impl::kv_log_store::{ ReaderTruncationOffsetType, RowOpCodeType, SeqIdType, }; -use crate::common::log_store::{LogStoreError, LogStoreResult}; const INSERT_OP_CODE: RowOpCodeType = 1; const DELETE_OP_CODE: RowOpCodeType = 2; @@ -313,34 +313,32 @@ impl LogStoreRowSerde { match self.deserialize(value)? { (epoch, LogStoreRowOp::Row { op, row }) => { if epoch != expected_epoch { - return Err(LogStoreError::Internal(anyhow!( + return Err(anyhow!( "decoded epoch {} not match expected epoch {}", epoch, expected_epoch - ))); + )); } ops.push(op); if ops.len() > size_bound { - return Err(LogStoreError::Internal(anyhow!( + return Err(anyhow!( "row count {} exceed size bound {}", ops.len(), size_bound - ))); + )); } assert!(data_chunk_builder.append_one_row(row).is_none()); } (_, LogStoreRowOp::Barrier { .. }) => { - return Err(LogStoreError::Internal(anyhow!( - "should not get barrier when decoding stream chunk" - ))); + return Err(anyhow!("should not get barrier when decoding stream chunk")); } } } if ops.is_empty() { - return Err(LogStoreError::Internal(anyhow!( + return Err(anyhow!( "should not get empty row when decoding stream chunk. start seq id: {}, end seq id {}", start_seq_id, - end_seq_id)) + end_seq_id) ); } Ok(StreamChunk::from_parts( @@ -405,11 +403,11 @@ impl LogStoreRowOpStream { StreamState::AllConsumingRow { curr_epoch } | StreamState::BarrierAligning { curr_epoch, .. } => { if *curr_epoch != epoch { - Err(LogStoreError::Internal(anyhow!( + Err(anyhow!( "epoch {} does not match with current epoch {}", epoch, curr_epoch - ))) + )) } else { Ok(()) } @@ -417,11 +415,11 @@ impl LogStoreRowOpStream { StreamState::BarrierEmitted { prev_epoch } => { if *prev_epoch >= epoch { - Err(LogStoreError::Internal(anyhow!( + Err(anyhow!( "epoch {} should be greater than prev epoch {}", epoch, prev_epoch - ))) + )) } else { Ok(()) } @@ -438,18 +436,18 @@ impl LogStoreRowOpStream { if is_checkpoint == *curr_is_checkpoint { Ok(()) } else { - Err(LogStoreError::Internal(anyhow!( + Err(anyhow!( "current aligning barrier is_checkpoint: {}, current barrier is_checkpoint {}", curr_is_checkpoint, is_checkpoint - ))) + )) } } else { Ok(()) } } - #[try_stream(ok = (u64, KvLogStoreItem), error = LogStoreError)] + #[try_stream(ok = (u64, KvLogStoreItem), error = anyhow::Error)] async fn into_log_store_item_stream(self, chunk_size: usize) { let mut ops = Vec::with_capacity(chunk_size); let mut data_chunk_builder = @@ -547,13 +545,14 @@ impl LogStoreRowOpStream { } // End of stream match &self.stream_state { - StreamState::BarrierEmitted { .. } | StreamState::Uninitialized => {}, - s => return Err(LogStoreError::Internal( - anyhow!( - "when any of the stream reaches the end, it should be right after emitting an barrier. Current state: {:?}", - s) - ) - ), + StreamState::BarrierEmitted { .. } | StreamState::Uninitialized => {} + s => { + return Err(anyhow!( + "when any of the stream reaches the end, it should be right \ + after emitting an barrier. Current state: {:?}", + s + )); + } } assert!( self.barrier_streams.is_empty(), @@ -562,8 +561,8 @@ impl LogStoreRowOpStream { if cfg!(debug_assertion) { while let Some((opt, _stream)) = self.row_streams.next().await { if let Some(result) = opt { - return Err(LogStoreError::Internal( - anyhow!("when any of the stream reaches the end, other stream should also reaches the end, but poll result: {:?}", result)) + return Err( + anyhow!("when any of the stream reaches the end, other stream should also reaches the end, but poll result: {:?}", result) ); } } @@ -592,14 +591,14 @@ mod tests { use tokio::sync::oneshot; use tokio::sync::oneshot::Sender; - use crate::common::log_store::kv_log_store::serde::{ + use crate::common::log_store_impl::kv_log_store::serde::{ new_log_store_item_stream, KvLogStoreItem, LogStoreRowOp, LogStoreRowOpStream, LogStoreRowSerde, }; - use crate::common::log_store::kv_log_store::test_utils::{ + use crate::common::log_store_impl::kv_log_store::test_utils::{ gen_test_data, gen_test_log_store_table, TEST_TABLE_ID, }; - use crate::common::log_store::kv_log_store::SeqIdType; + use crate::common::log_store_impl::kv_log_store::SeqIdType; const EPOCH1: u64 = 233; const EPOCH2: u64 = EPOCH1 + 1; diff --git a/src/stream/src/common/log_store/kv_log_store/test_utils.rs b/src/stream/src/common/log_store_impl/kv_log_store/test_utils.rs similarity index 100% rename from src/stream/src/common/log_store/kv_log_store/test_utils.rs rename to src/stream/src/common/log_store_impl/kv_log_store/test_utils.rs diff --git a/src/stream/src/common/log_store/kv_log_store/writer.rs b/src/stream/src/common/log_store_impl/kv_log_store/writer.rs similarity index 94% rename from src/stream/src/common/log_store/kv_log_store/writer.rs rename to src/stream/src/common/log_store_impl/kv_log_store/writer.rs index 54d7db38b8570..a07757a5510a9 100644 --- a/src/stream/src/common/log_store/kv_log_store/writer.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/writer.rs @@ -21,12 +21,12 @@ use risingwave_common::buffer::{Bitmap, BitmapBuilder}; use risingwave_common::catalog::TableId; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::util::epoch::EpochPair; +use risingwave_connector::sink::log_store::{LogStoreResult, LogWriter}; use risingwave_storage::store::{InitOptions, LocalStateStore}; -use crate::common::log_store::kv_log_store::buffer::LogStoreBufferSender; -use crate::common::log_store::kv_log_store::serde::LogStoreRowSerde; -use crate::common::log_store::kv_log_store::{SeqIdType, FIRST_SEQ_ID}; -use crate::common::log_store::{LogStoreResult, LogWriter}; +use crate::common::log_store_impl::kv_log_store::buffer::LogStoreBufferSender; +use crate::common::log_store_impl::kv_log_store::serde::LogStoreRowSerde; +use crate::common::log_store_impl::kv_log_store::{SeqIdType, FIRST_SEQ_ID}; pub struct KvLogStoreWriter { _table_id: TableId, diff --git a/src/stream/src/common/log_store_impl/mod.rs b/src/stream/src/common/log_store_impl/mod.rs new file mode 100644 index 0000000000000..633fa07f2617d --- /dev/null +++ b/src/stream/src/common/log_store_impl/mod.rs @@ -0,0 +1,16 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +pub mod in_mem; +pub mod kv_log_store; diff --git a/src/stream/src/common/mod.rs b/src/stream/src/common/mod.rs index e865214cb0990..7f5111c29e03e 100644 --- a/src/stream/src/common/mod.rs +++ b/src/stream/src/common/mod.rs @@ -18,6 +18,6 @@ pub use column_mapping::*; mod builder; pub mod cache; mod column_mapping; -pub mod log_store; +pub mod log_store_impl; pub mod metrics; pub mod table; diff --git a/src/stream/src/error.rs b/src/stream/src/error.rs index b737de4d2560b..2930cda31747e 100644 --- a/src/stream/src/error.rs +++ b/src/stream/src/error.rs @@ -16,6 +16,7 @@ use std::backtrace::Backtrace; use risingwave_common::array::ArrayError; use risingwave_connector::error::ConnectorError; +use risingwave_connector::sink::SinkError; use risingwave_expr::ExprError; use risingwave_pb::PbFieldNotFound; use risingwave_storage::error::StorageError; @@ -58,6 +59,9 @@ enum ErrorKind { #[error("Executor error: {0:?}")] Executor(#[source] StreamExecutorError), + #[error("Sink error: {0:?}")] + Sink(#[source] SinkError), + #[error(transparent)] Internal(anyhow::Error), } @@ -115,6 +119,12 @@ impl From for StreamError { } } +impl From for StreamError { + fn from(value: SinkError) -> Self { + ErrorKind::Sink(value).into() + } +} + impl From for StreamError { fn from(err: PbFieldNotFound) -> Self { Self::from(anyhow::anyhow!( diff --git a/src/stream/src/executor/error.rs b/src/stream/src/executor/error.rs index 32d7ee8479110..b1e436c841577 100644 --- a/src/stream/src/executor/error.rs +++ b/src/stream/src/executor/error.rs @@ -25,7 +25,6 @@ use risingwave_rpc_client::error::RpcError; use risingwave_storage::error::StorageError; use super::Barrier; -use crate::common::log_store::LogStoreError; /// A specialized Result type for streaming executors. pub type StreamExecutorResult = std::result::Result; @@ -54,9 +53,6 @@ enum ErrorKind { StorageError, ), - #[error("Log store error: {0}")] - LogStoreError(#[source] LogStoreError), - #[error("Chunk operation error: {0}")] ArrayError(#[source] ArrayError), @@ -154,13 +150,6 @@ impl From for StreamExecutorError { } } -/// Log store error -impl From for StreamExecutorError { - fn from(e: LogStoreError) -> Self { - ErrorKind::LogStoreError(e).into() - } -} - /// Chunk operation error. impl From for StreamExecutorError { fn from(e: ArrayError) -> Self { diff --git a/src/stream/src/executor/sink.rs b/src/stream/src/executor/sink.rs index b36a5613ba7bb..14394073e8df7 100644 --- a/src/stream/src/executor/sink.rs +++ b/src/stream/src/executor/sink.rs @@ -13,34 +13,33 @@ // limitations under the License. use std::sync::Arc; -use std::time::Instant; +use anyhow::anyhow; use futures::stream::select; use futures::{FutureExt, StreamExt}; use futures_async_stream::try_stream; use itertools::Itertools; -use prometheus::Histogram; use risingwave_common::array::stream_chunk::StreamChunkMut; use risingwave_common::array::{merge_chunk_row, Op, StreamChunk}; use risingwave_common::catalog::{ColumnCatalog, Field, Schema}; use risingwave_common::util::epoch::EpochPair; use risingwave_connector::dispatch_sink; use risingwave_connector::sink::catalog::SinkType; +use risingwave_connector::sink::log_store::{ + LogReader, LogStoreFactory, LogStoreTransformChunkLogReader, LogWriter, +}; use risingwave_connector::sink::{ - build_sink, Sink, SinkImpl, SinkParam, SinkWriter, SinkWriterParam, + build_sink, LogSinker, Sink, SinkImpl, SinkParam, SinkWriterParam, }; use super::error::{StreamExecutorError, StreamExecutorResult}; use super::{BoxedExecutor, Executor, Message, PkIndices}; -use crate::common::log_store::{ - LogReader, LogStoreFactory, LogStoreReadItem, LogWriter, TruncateOffset, -}; use crate::executor::monitor::StreamingMetrics; use crate::executor::{expect_first_barrier, ActorContextRef, BoxedMessageStream}; pub struct SinkExecutor { input: BoxedExecutor, - metrics: Arc, + _metrics: Arc, sink: SinkImpl, identity: String, pk_indices: PkIndices, @@ -53,10 +52,6 @@ pub struct SinkExecutor { sink_writer_param: SinkWriterParam, } -struct SinkMetrics { - sink_commit_duration_metrics: Histogram, -} - // Drop all the DELETE messages in this chunk and convert UPDATE INSERT into INSERT. fn force_append_only(c: StreamChunk) -> StreamChunk { let mut c: StreamChunkMut = c.into(); @@ -91,7 +86,7 @@ impl SinkExecutor { .collect(); Ok(Self { input, - metrics, + _metrics: metrics, sink, identity: format!("SinkExecutor {:X?}", sink_writer_param.executor_id), pk_indices, @@ -106,15 +101,6 @@ impl SinkExecutor { } fn execute_inner(self) -> BoxedMessageStream { - let sink_commit_duration_metrics = self - .metrics - .sink_commit_duration - .with_label_values(&[self.identity.as_str(), self.sink.get_connector()]); - - let sink_metrics = SinkMetrics { - sink_commit_duration_metrics, - }; - let write_log_stream = Self::execute_write_log( self.input, self.pk_indices, @@ -128,7 +114,6 @@ impl SinkExecutor { sink, self.log_reader, self.input_columns, - sink_metrics, self.sink_writer_param, ); select(consume_log_stream.into_stream(), write_log_stream).boxed() @@ -193,13 +178,11 @@ impl SinkExecutor { async fn execute_consume_log( sink: S, - mut log_reader: R, + log_reader: R, columns: Vec, - sink_metrics: SinkMetrics, sink_writer_param: SinkWriterParam, ) -> StreamExecutorResult { - log_reader.init().await?; - let mut sink_writer = sink.new_writer(sink_writer_param).await?; + let log_sinker = sink.new_log_sinker(sink_writer_param).await?; let visible_columns = columns .iter() @@ -207,96 +190,18 @@ impl SinkExecutor { .filter_map(|(idx, column)| (!column.is_hidden).then_some(idx)) .collect_vec(); - #[derive(Debug)] - enum LogConsumerState { - /// Mark that the log consumer is not initialized yet - Uninitialized, - - /// Mark that a new epoch has begun. - EpochBegun { curr_epoch: u64 }, - - /// Mark that the consumer has just received a barrier - BarrierReceived { prev_epoch: u64 }, - } - - let mut state = LogConsumerState::Uninitialized; - - loop { - let (epoch, item): (u64, LogStoreReadItem) = log_reader.next_item().await?; - if let LogStoreReadItem::UpdateVnodeBitmap(_) = &item { - match &state { - LogConsumerState::BarrierReceived { .. } => {} - _ => unreachable!( - "update vnode bitmap can be accepted only right after \ - barrier, but current state is {:?}", - state - ), - } - } - // begin_epoch when not previously began - state = match state { - LogConsumerState::Uninitialized => { - sink_writer.begin_epoch(epoch).await?; - LogConsumerState::EpochBegun { curr_epoch: epoch } - } - LogConsumerState::EpochBegun { curr_epoch } => { - assert!( - epoch >= curr_epoch, - "new epoch {} should not be below the current epoch {}", - epoch, - curr_epoch - ); - LogConsumerState::EpochBegun { curr_epoch: epoch } - } - LogConsumerState::BarrierReceived { prev_epoch } => { - assert!( - epoch > prev_epoch, - "new epoch {} should be greater than prev epoch {}", - epoch, - prev_epoch - ); - sink_writer.begin_epoch(epoch).await?; - LogConsumerState::EpochBegun { curr_epoch: epoch } - } - }; - match item { - LogStoreReadItem::StreamChunk { chunk, .. } => { - let chunk = if visible_columns.len() != columns.len() { - // Do projection here because we may have columns that aren't visible to - // the downstream. - chunk.project(&visible_columns) - } else { - chunk - }; - if let Err(e) = sink_writer.write_batch(chunk).await { - sink_writer.abort().await?; - return Err(e.into()); - } - } - LogStoreReadItem::Barrier { is_checkpoint } => { - let prev_epoch = match state { - LogConsumerState::EpochBegun { curr_epoch } => curr_epoch, - _ => unreachable!("epoch must have begun before handling barrier"), - }; - if is_checkpoint { - let start_time = Instant::now(); - sink_writer.barrier(true).await?; - sink_metrics - .sink_commit_duration_metrics - .observe(start_time.elapsed().as_millis() as f64); - log_reader - .truncate(TruncateOffset::Barrier { epoch: prev_epoch }) - .await?; - } else { - sink_writer.barrier(false).await?; - } - state = LogConsumerState::BarrierReceived { prev_epoch } - } - LogStoreReadItem::UpdateVnodeBitmap(vnode_bitmap) => { - sink_writer.update_vnode_bitmap(vnode_bitmap).await?; - } + let log_reader = log_reader.transform_chunk(move |chunk| { + if visible_columns.len() != columns.len() { + // Do projection here because we may have columns that aren't visible to + // the downstream. + chunk.project(&visible_columns) + } else { + chunk } - } + }); + + log_sinker.consume_log_and_sink(log_reader).await?; + Err(anyhow!("end of stream").into()) } } @@ -323,7 +228,7 @@ mod test { use risingwave_common::catalog::{ColumnDesc, ColumnId}; use super::*; - use crate::common::log_store::in_mem::BoundedInMemLogStoreFactory; + use crate::common::log_store_impl::in_mem::BoundedInMemLogStoreFactory; use crate::executor::test_utils::*; use crate::executor::ActorContext; diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs index a95b7fce22738..295a36376b376 100644 --- a/src/stream/src/from_proto/sink.rs +++ b/src/stream/src/from_proto/sink.rs @@ -14,15 +14,19 @@ use std::sync::Arc; +use anyhow::anyhow; use risingwave_common::catalog::ColumnCatalog; +use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::SinkType; -use risingwave_connector::sink::{SinkParam, SinkWriterParam}; +use risingwave_connector::sink::{ + SinkError, SinkMetrics, SinkParam, SinkWriterParam, CONNECTOR_TYPE_KEY, +}; use risingwave_pb::stream_plan::{SinkLogStoreType, SinkNode}; use risingwave_storage::dispatch_state_store; use super::*; -use crate::common::log_store::in_mem::BoundedInMemLogStoreFactory; -use crate::common::log_store::kv_log_store::KvLogStoreFactory; +use crate::common::log_store_impl::in_mem::BoundedInMemLogStoreFactory; +use crate::common::log_store_impl::kv_log_store::KvLogStoreFactory; use crate::executor::SinkExecutor; pub struct SinkExecutorBuilder; @@ -56,6 +60,27 @@ impl ExecutorBuilder for SinkExecutorBuilder { .into_iter() .map(ColumnCatalog::from) .collect_vec(); + + let connector = { + let sink_type = properties.get(CONNECTOR_TYPE_KEY).ok_or_else(|| { + SinkError::Config(anyhow!("missing config: {}", CONNECTOR_TYPE_KEY)) + })?; + + use risingwave_connector::sink::Sink; + + match_sink_name_str!( + sink_type.to_lowercase().as_str(), + SinkType, + Ok(SinkType::SINK_NAME), + |other| { + Err(SinkError::Config(anyhow!( + "unsupported sink connector {}", + other + ))) + } + ) + }?; + let sink_param = SinkParam { sink_id, properties, @@ -70,6 +95,17 @@ impl ExecutorBuilder for SinkExecutorBuilder { sink_from_name, }; + let identity = format!("SinkExecutor {:X?}", params.executor_id); + + let sink_commit_duration_metrics = stream + .streaming_metrics + .sink_commit_duration + .with_label_values(&[identity.as_str(), connector]); + + let sink_metrics = SinkMetrics { + sink_commit_duration_metrics, + }; + match node.log_store_type() { // Default value is the normal in memory log store to be backward compatible with the // previously unset value @@ -84,6 +120,7 @@ impl ExecutorBuilder for SinkExecutorBuilder { executor_id: params.executor_id, vnode_bitmap: params.vnode_bitmap, meta_client: params.env.meta_client(), + sink_metrics, }, sink_param, columns, @@ -112,6 +149,7 @@ impl ExecutorBuilder for SinkExecutorBuilder { executor_id: params.executor_id, vnode_bitmap: params.vnode_bitmap, meta_client: params.env.meta_client(), + sink_metrics, }, sink_param, columns, diff --git a/src/tests/simulation/tests/integration_tests/sink/basic.rs b/src/tests/simulation/tests/integration_tests/sink/basic.rs index f2c6d9f3c8b98..ad8eb8a4d9cf2 100644 --- a/src/tests/simulation/tests/integration_tests/sink/basic.rs +++ b/src/tests/simulation/tests/integration_tests/sink/basic.rs @@ -31,7 +31,8 @@ use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_connector::sink::boxed::{BoxCoordinator, BoxWriter}; use risingwave_connector::sink::test_sink::registry_build_sink; -use risingwave_connector::sink::{Sink, SinkWriter, SinkWriterParam}; +use risingwave_connector::sink::writer::SinkWriter; +use risingwave_connector::sink::{Sink, SinkWriterParam}; use risingwave_connector::source::test_source::{registry_test_source, BoxSource, TestSourceSplit}; use risingwave_connector::source::StreamChunkWithState; use risingwave_simulation::cluster::{Cluster, ConfigPath, Configuration}; From 09a1dcb22355677b3a7dcc589def5727b475bd88 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Tue, 26 Sep 2023 17:31:19 +0800 Subject: [PATCH 08/31] fix(over window): don't expect stream keys in UpdateDelete and UpdateInsert the same (#12536) Signed-off-by: Richard Chien --- .../src/executor/over_window/general.rs | 41 ++++--------------- 1 file changed, 9 insertions(+), 32 deletions(-) diff --git a/src/stream/src/executor/over_window/general.rs b/src/stream/src/executor/over_window/general.rs index 091e199d7b52a..5a01fabd25149 100644 --- a/src/stream/src/executor/over_window/general.rs +++ b/src/stream/src/executor/over_window/general.rs @@ -21,7 +21,7 @@ use futures::StreamExt; use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::stream_record::Record; -use risingwave_common::array::{RowRef, StreamChunk}; +use risingwave_common::array::{Op, RowRef, StreamChunk}; use risingwave_common::catalog::Field; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::session_config::OverWindowCachePolicy as CachePolicy; @@ -225,26 +225,25 @@ impl OverWindowExecutor { chunk: &'a StreamChunk, ) -> impl Iterator>> { let mut changes_merged = BTreeMap::new(); - for record in chunk.records() { - match record { - Record::Insert { new_row } => { - let pk = DefaultOrdered(this.get_input_pk(new_row)); + for (op, row) in chunk.rows() { + let pk = DefaultOrdered(this.get_input_pk(row)); + match op { + Op::Insert | Op::UpdateInsert => { if let Some(prev_change) = changes_merged.get_mut(&pk) { match prev_change { Record::Delete { old_row } => { *prev_change = Record::Update { old_row: *old_row, - new_row, + new_row: row, }; } _ => panic!("inconsistent changes in input chunk"), } } else { - changes_merged.insert(pk, record); + changes_merged.insert(pk, Record::Insert { new_row: row }); } } - Record::Delete { old_row } => { - let pk = DefaultOrdered(this.get_input_pk(old_row)); + Op::Delete | Op::UpdateDelete => { if let Some(prev_change) = changes_merged.get_mut(&pk) { match prev_change { Record::Insert { .. } => { @@ -261,29 +260,7 @@ impl OverWindowExecutor { _ => panic!("inconsistent changes in input chunk"), } } else { - changes_merged.insert(pk, record); - } - } - Record::Update { old_row, new_row } => { - let pk = DefaultOrdered(this.get_input_pk(old_row)); - if let Some(prev_change) = changes_merged.get_mut(&pk) { - match prev_change { - Record::Insert { .. } => { - *prev_change = Record::Insert { new_row }; - } - Record::Update { - old_row: real_old_row, - .. - } => { - *prev_change = Record::Update { - old_row: *real_old_row, - new_row, - }; - } - _ => panic!("inconsistent changes in input chunk"), - } - } else { - changes_merged.insert(pk, record); + changes_merged.insert(pk, Record::Delete { old_row: row }); } } } From d583594284a8a364b430114b0d8cf63d1fae585a Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Wed, 27 Sep 2023 01:11:50 +0800 Subject: [PATCH 09/31] refactor(expr): separate function implementations into a new crate (#12485) Signed-off-by: Runji Wang --- Cargo.lock | 50 +- Cargo.toml | 6 +- src/batch/Cargo.toml | 1 + src/batch/benches/hash_agg.rs | 2 +- .../src/executor/aggregation/distinct.rs | 4 +- src/batch/src/executor/aggregation/filter.rs | 4 +- src/batch/src/executor/aggregation/mod.rs | 4 +- src/batch/src/executor/aggregation/orderby.rs | 4 +- .../src/executor/aggregation/projection.rs | 2 +- src/batch/src/executor/hash_agg.rs | 2 +- src/batch/src/executor/hop_window.rs | 14 +- .../src/executor/join/local_lookup_join.rs | 8 +- src/batch/src/executor/sort_agg.rs | 2 +- src/batch/src/lib.rs | 3 + src/cmd/Cargo.toml | 1 + src/cmd/src/lib.rs | 2 + src/cmd_all/Cargo.toml | 1 + src/cmd_all/src/lib.rs | 2 + src/common/src/cast/mod.rs | 112 +---- src/common/src/types/datetime.rs | 200 +++++++- src/common/src/types/mod.rs | 36 +- .../src/parser/canal/simd_json_parser.rs | 7 +- src/connector/src/parser/csv_parser.rs | 9 +- src/connector/src/parser/json_parser.rs | 16 +- .../src/parser/maxwell/simd_json_parser.rs | 7 +- src/connector/src/parser/unified/json.rs | 27 +- src/expr/{ => core}/Cargo.toml | 35 +- .../{src/agg => core/src/aggregate}/def.rs | 20 +- .../{src/agg => core/src/aggregate}/mod.rs | 12 - src/expr/core/src/codegen.rs | 19 + src/expr/{ => core}/src/error.rs | 9 - .../src/expr/and_or.rs} | 91 ++-- src/expr/{ => core}/src/expr/build.rs | 3 +- .../src/expr/expr_array_transform.rs | 0 src/expr/{ => core}/src/expr/expr_case.rs | 135 ++---- src/expr/{ => core}/src/expr/expr_coalesce.rs | 0 src/expr/{ => core}/src/expr/expr_field.rs | 0 src/expr/{ => core}/src/expr/expr_in.rs | 0 .../{ => core}/src/expr/expr_input_ref.rs | 0 src/expr/{ => core}/src/expr/expr_literal.rs | 0 src/expr/{ => core}/src/expr/expr_some_all.rs | 0 src/expr/{ => core}/src/expr/expr_udf.rs | 0 src/expr/{ => core}/src/expr/expr_vnode.rs | 0 src/expr/{ => core}/src/expr/mod.rs | 6 +- src/expr/{ => core}/src/expr/test_utils.rs | 0 src/expr/{ => core}/src/expr/value.rs | 0 .../{ => core}/src/expr/wrapper/checked.rs | 0 src/expr/{ => core}/src/expr/wrapper/mod.rs | 0 src/expr/{ => core}/src/lib.rs | 11 +- .../vector_op => core/src/scalar}/like.rs | 2 +- src/expr/core/src/scalar/mod.rs | 15 + src/expr/{ => core}/src/sig/agg.rs | 2 +- src/expr/{ => core}/src/sig/cast.rs | 0 src/expr/core/src/sig/func.rs | 117 +++++ src/expr/{ => core}/src/sig/mod.rs | 0 src/expr/{ => core}/src/sig/table_function.rs | 0 .../{ => core}/src/table_function/empty.rs | 0 src/expr/{ => core}/src/table_function/mod.rs | 6 - .../{ => core}/src/table_function/repeat.rs | 0 .../src/table_function/user_defined.rs | 0 .../{ => core}/src/window_function/call.rs | 2 +- .../{ => core}/src/window_function/kind.rs | 2 +- .../{ => core}/src/window_function/mod.rs | 0 .../src/window_function/state/aggregate.rs | 4 +- .../src/window_function/state/buffer.rs | 0 .../src/window_function/state/mod.rs | 0 .../src/window_function/state/row_number.rs | 2 +- .../{ => core}/src/window_function/states.rs | 0 src/expr/impl/Cargo.toml | 64 +++ src/expr/{ => impl}/README.md | 0 src/expr/{ => impl}/benches/expr.rs | 4 +- .../approx_count_distinct/append_only.rs | 2 +- .../aggregate}/approx_count_distinct/mod.rs | 16 +- .../approx_count_distinct/updatable.rs | 0 .../agg => impl/src/aggregate}/array_agg.rs | 13 +- .../agg => impl/src/aggregate}/bool_and.rs | 4 +- .../agg => impl/src/aggregate}/bool_or.rs | 4 +- .../agg => impl/src/aggregate}/general.rs | 47 +- .../agg => impl/src/aggregate}/jsonb_agg.rs | 4 +- src/expr/impl/src/aggregate/mod.rs | 24 + .../{src/agg => impl/src/aggregate}/mode.rs | 9 +- .../src/aggregate}/percentile_cont.rs | 17 +- .../src/aggregate}/percentile_disc.rs | 9 +- .../agg => impl/src/aggregate}/string_agg.rs | 11 +- src/expr/impl/src/lib.rs | 45 ++ .../src/scalar}/arithmetic_op.rs | 46 +- .../vector_op => impl/src/scalar}/array.rs | 2 +- .../src/scalar}/array_access.rs | 8 +- .../src/scalar}/array_concat.rs | 5 +- .../src/scalar}/array_distinct.rs | 2 +- .../src/scalar}/array_length.rs | 12 +- .../src/scalar}/array_min_max.rs | 4 +- .../src/scalar}/array_positions.rs | 9 +- .../src/scalar}/array_range_access.rs | 6 +- .../src/scalar}/array_remove.rs | 2 +- .../src/scalar}/array_replace.rs | 12 +- .../src/scalar}/array_sort.rs | 2 +- .../src/scalar}/array_sum.rs | 16 +- .../src/scalar}/array_to_string.rs | 5 +- .../vector_op => impl/src/scalar}/ascii.rs | 4 +- .../src/scalar}/bitwise_op.rs | 28 +- .../src/scalar}/cardinality.rs | 12 +- .../vector_op => impl/src/scalar}/cast.rs | 441 ++++++++++-------- .../{src/vector_op => impl/src/scalar}/cmp.rs | 265 ++++++++++- .../src/scalar}/concat_op.rs | 2 +- .../src/scalar}/concat_ws.rs | 5 +- .../src/scalar/conjunction.rs} | 25 +- .../src/scalar}/date_trunc.rs | 8 +- .../vector_op => impl/src/scalar}/delay.rs | 4 +- .../vector_op => impl/src/scalar}/encdec.rs | 5 +- .../{src/vector_op => impl/src/scalar}/exp.rs | 12 +- .../vector_op => impl/src/scalar}/extract.rs | 17 +- .../vector_op => impl/src/scalar}/format.rs | 6 +- src/expr/impl/src/scalar/format_type.rs | 50 ++ .../vector_op => impl/src/scalar}/int256.rs | 10 +- .../src/scalar}/jsonb_access.rs | 6 +- .../src/scalar}/jsonb_info.rs | 6 +- .../vector_op => impl/src/scalar}/length.rs | 16 +- .../vector_op => impl/src/scalar}/lower.rs | 2 +- .../{src/vector_op => impl/src/scalar}/md5.rs | 2 +- src/expr/impl/src/scalar/mod.rs | 70 +++ .../vector_op => impl/src/scalar}/overlay.rs | 8 +- .../vector_op => impl/src/scalar}/position.rs | 6 +- .../vector_op => impl/src/scalar}/proctime.rs | 4 +- .../vector_op => impl/src/scalar}/regexp.rs | 16 +- .../vector_op => impl/src/scalar}/repeat.rs | 4 +- .../vector_op => impl/src/scalar}/replace.rs | 2 +- .../vector_op => impl/src/scalar}/round.rs | 16 +- .../{src/vector_op => impl/src/scalar}/sha.rs | 2 +- .../src/scalar}/split_part.rs | 6 +- .../vector_op => impl/src/scalar}/string.rs | 20 +- .../src/scalar}/string_to_array.rs | 2 +- .../vector_op => impl/src/scalar}/substr.rs | 8 +- .../src/scalar}/timestamptz.rs | 19 +- .../vector_op => impl/src/scalar}/to_char.rs | 5 +- .../src/scalar}/to_timestamp.rs | 5 +- .../src/scalar}/translate.rs | 2 +- .../src/scalar}/trigonometric.rs | 48 +- .../vector_op => impl/src/scalar}/trim.rs | 2 +- .../src/scalar}/trim_array.rs | 7 +- .../vector_op => impl/src/scalar}/tumble.rs | 6 +- .../vector_op => impl/src/scalar}/upper.rs | 2 +- .../src/table_function/generate_series.rs | 39 +- .../src/table_function/generate_subscripts.rs | 16 +- .../{ => impl}/src/table_function/jsonb.rs | 4 +- src/expr/impl/src/table_function/mod.rs | 20 + .../src/table_function/pg_expandarray.rs | 8 +- .../src/table_function/regexp_matches.rs | 8 +- .../{ => impl}/src/table_function/unnest.rs | 4 +- src/expr/impl/tests/sig.rs | 107 +++++ src/expr/macro/src/gen.rs | 27 +- src/expr/macro/src/lib.rs | 10 +- src/expr/macro/src/types.rs | 61 ++- src/expr/src/expr/expr_binary_nonnull.rs | 232 --------- src/expr/src/expr/expr_unary.rs | 206 -------- src/expr/src/sig/func.rs | 216 --------- src/expr/src/vector_op/conjunction.rs | 77 --- src/expr/src/vector_op/mod.rs | 71 --- src/frontend/Cargo.toml | 1 + src/frontend/planner_test/Cargo.toml | 1 + src/frontend/planner_test/src/lib.rs | 2 + src/frontend/src/binder/expr/function.rs | 2 +- src/frontend/src/binder/select.rs | 2 +- src/frontend/src/expr/agg_call.rs | 2 +- src/frontend/src/expr/function_call.rs | 8 +- .../src/expr/function_impl/col_description.rs | 2 +- src/frontend/src/expr/mod.rs | 2 +- src/frontend/src/handler/show.rs | 2 +- src/frontend/src/handler/util.rs | 18 +- src/frontend/src/lib.rs | 3 + .../src/optimizer/plan_node/generic/agg.rs | 2 +- .../src/optimizer/plan_node/logical_agg.rs | 2 +- .../plan_node/logical_over_window.rs | 2 +- .../rule/agg_group_by_simplify_rule.rs | 2 +- .../rule/apply_agg_transpose_rule.rs | 2 +- .../src/optimizer/rule/distinct_agg_rule.rs | 2 +- .../rule/grouping_sets_to_expand_rule.rs | 2 +- .../optimizer/rule/min_max_on_index_rule.rs | 2 +- src/jni_core/src/lib.rs | 7 +- src/stream/Cargo.toml | 8 +- src/stream/benches/stream_hash_agg.rs | 2 +- src/stream/src/executor/agg_common.rs | 2 +- .../src/executor/aggregation/agg_group.rs | 2 +- .../src/executor/aggregation/agg_state.rs | 2 +- src/stream/src/executor/aggregation/minput.rs | 4 +- src/stream/src/executor/aggregation/mod.rs | 2 +- src/stream/src/executor/hash_agg.rs | 2 +- src/stream/src/executor/hop_window.rs | 6 +- src/stream/src/executor/integration_tests.rs | 2 +- .../executor/over_window/over_partition.rs | 2 +- src/stream/src/executor/simple_agg.rs | 4 +- .../src/executor/stateless_simple_agg.rs | 4 +- src/stream/src/executor/test_utils.rs | 2 +- src/stream/src/from_proto/hash_agg.rs | 2 +- src/stream/src/from_proto/simple_agg.rs | 2 +- .../src/from_proto/stateless_simple_agg.rs | 2 +- src/stream/src/lib.rs | 3 + .../integration_tests/eowc_over_window.rs | 2 +- .../tests/integration_tests/hash_agg.rs | 2 +- .../tests/integration_tests/hop_window.rs | 13 +- src/stream/tests/integration_tests/main.rs | 2 + .../tests/integration_tests/over_window.rs | 2 +- src/tests/simulation/Cargo.toml | 1 + src/tests/simulation/src/lib.rs | 2 + src/tests/sqlsmith/Cargo.toml | 1 + src/tests/sqlsmith/src/bin/main.rs | 2 + src/tests/sqlsmith/src/sql_gen/agg.rs | 2 +- src/tests/sqlsmith/src/sql_gen/types.rs | 2 +- 208 files changed, 1960 insertions(+), 1907 deletions(-) rename src/expr/{ => core}/Cargo.toml (64%) rename src/expr/{src/agg => core/src/aggregate}/def.rs (94%) rename src/expr/{src/agg => core/src/aggregate}/mod.rs (96%) create mode 100644 src/expr/core/src/codegen.rs rename src/expr/{ => core}/src/error.rs (93%) rename src/expr/{src/expr/expr_binary_nullable.rs => core/src/expr/and_or.rs} (79%) rename src/expr/{ => core}/src/expr/build.rs (99%) rename src/expr/{ => core}/src/expr/expr_array_transform.rs (100%) rename src/expr/{ => core}/src/expr/expr_case.rs (62%) rename src/expr/{ => core}/src/expr/expr_coalesce.rs (100%) rename src/expr/{ => core}/src/expr/expr_field.rs (100%) rename src/expr/{ => core}/src/expr/expr_in.rs (100%) rename src/expr/{ => core}/src/expr/expr_input_ref.rs (100%) rename src/expr/{ => core}/src/expr/expr_literal.rs (100%) rename src/expr/{ => core}/src/expr/expr_some_all.rs (100%) rename src/expr/{ => core}/src/expr/expr_udf.rs (100%) rename src/expr/{ => core}/src/expr/expr_vnode.rs (100%) rename src/expr/{ => core}/src/expr/mod.rs (98%) rename src/expr/{ => core}/src/expr/test_utils.rs (100%) rename src/expr/{ => core}/src/expr/value.rs (100%) rename src/expr/{ => core}/src/expr/wrapper/checked.rs (100%) rename src/expr/{ => core}/src/expr/wrapper/mod.rs (100%) rename src/expr/{ => core}/src/lib.rs (86%) rename src/expr/{src/vector_op => core/src/scalar}/like.rs (99%) create mode 100644 src/expr/core/src/scalar/mod.rs rename src/expr/{ => core}/src/sig/agg.rs (98%) rename src/expr/{ => core}/src/sig/cast.rs (100%) create mode 100644 src/expr/core/src/sig/func.rs rename src/expr/{ => core}/src/sig/mod.rs (100%) rename src/expr/{ => core}/src/sig/table_function.rs (100%) rename src/expr/{ => core}/src/table_function/empty.rs (100%) rename src/expr/{ => core}/src/table_function/mod.rs (98%) rename src/expr/{ => core}/src/table_function/repeat.rs (100%) rename src/expr/{ => core}/src/table_function/user_defined.rs (100%) rename src/expr/{ => core}/src/window_function/call.rs (99%) rename src/expr/{ => core}/src/window_function/kind.rs (98%) rename src/expr/{ => core}/src/window_function/mod.rs (100%) rename src/expr/{ => core}/src/window_function/state/aggregate.rs (98%) rename src/expr/{ => core}/src/window_function/state/buffer.rs (100%) rename src/expr/{ => core}/src/window_function/state/mod.rs (100%) rename src/expr/{ => core}/src/window_function/state/row_number.rs (99%) rename src/expr/{ => core}/src/window_function/states.rs (100%) create mode 100644 src/expr/impl/Cargo.toml rename src/expr/{ => impl}/README.md (100%) rename src/expr/{ => impl}/benches/expr.rs (99%) rename src/expr/{src/agg => impl/src/aggregate}/approx_count_distinct/append_only.rs (97%) rename src/expr/{src/agg => impl/src/aggregate}/approx_count_distinct/mod.rs (96%) rename src/expr/{src/agg => impl/src/aggregate}/approx_count_distinct/updatable.rs (100%) rename src/expr/{src/agg => impl/src/aggregate}/array_agg.rs (89%) rename src/expr/{src/agg => impl/src/aggregate}/bool_and.rs (96%) rename src/expr/{src/agg => impl/src/aggregate}/bool_or.rs (96%) rename src/expr/{src/agg => impl/src/aggregate}/general.rs (93%) rename src/expr/{src/agg => impl/src/aggregate}/jsonb_agg.rs (96%) create mode 100644 src/expr/impl/src/aggregate/mod.rs rename src/expr/{src/agg => impl/src/aggregate}/mode.rs (95%) rename src/expr/{src/agg => impl/src/aggregate}/percentile_cont.rs (89%) rename src/expr/{src/agg => impl/src/aggregate}/percentile_disc.rs (95%) rename src/expr/{src/agg => impl/src/aggregate}/string_agg.rs (89%) create mode 100644 src/expr/impl/src/lib.rs rename src/expr/{src/vector_op => impl/src/scalar}/arithmetic_op.rs (94%) rename src/expr/{src/vector_op => impl/src/scalar}/array.rs (96%) rename src/expr/{src/vector_op => impl/src/scalar}/array_access.rs (96%) rename src/expr/{src/vector_op => impl/src/scalar}/array_concat.rs (98%) rename src/expr/{src/vector_op => impl/src/scalar}/array_distinct.rs (98%) rename src/expr/{src/vector_op => impl/src/scalar}/array_length.rs (94%) rename src/expr/{src/vector_op => impl/src/scalar}/array_min_max.rs (97%) rename src/expr/{src/vector_op => impl/src/scalar}/array_positions.rs (96%) rename src/expr/{src/vector_op => impl/src/scalar}/array_range_access.rs (92%) rename src/expr/{src/vector_op => impl/src/scalar}/array_remove.rs (98%) rename src/expr/{src/vector_op => impl/src/scalar}/array_replace.rs (90%) rename src/expr/{src/vector_op => impl/src/scalar}/array_sort.rs (96%) rename src/expr/{src/vector_op => impl/src/scalar}/array_sum.rs (90%) rename src/expr/{src/vector_op => impl/src/scalar}/array_to_string.rs (98%) rename src/expr/{src/vector_op => impl/src/scalar}/ascii.rs (92%) rename src/expr/{src/vector_op => impl/src/scalar}/bitwise_op.rs (83%) rename src/expr/{src/vector_op => impl/src/scalar}/cardinality.rs (87%) rename src/expr/{src/vector_op => impl/src/scalar}/cast.rs (61%) rename src/expr/{src/vector_op => impl/src/scalar}/cmp.rs (60%) rename src/expr/{src/vector_op => impl/src/scalar}/concat_op.rs (96%) rename src/expr/{src/vector_op => impl/src/scalar}/concat_ws.rs (96%) rename src/expr/{src/vector_op/format_type.rs => impl/src/scalar/conjunction.rs} (58%) rename src/expr/{src/vector_op => impl/src/scalar}/date_trunc.rs (95%) rename src/expr/{src/vector_op => impl/src/scalar}/delay.rs (94%) rename src/expr/{src/vector_op => impl/src/scalar}/encdec.rs (99%) rename src/expr/{src/vector_op => impl/src/scalar}/exp.rs (94%) rename src/expr/{src/vector_op => impl/src/scalar}/extract.rs (96%) rename src/expr/{src/vector_op => impl/src/scalar}/format.rs (98%) create mode 100644 src/expr/impl/src/scalar/format_type.rs rename src/expr/{src/vector_op => impl/src/scalar}/int256.rs (94%) rename src/expr/{src/vector_op => impl/src/scalar}/jsonb_access.rs (92%) rename src/expr/{src/vector_op => impl/src/scalar}/jsonb_info.rs (93%) rename src/expr/{src/vector_op => impl/src/scalar}/length.rs (82%) rename src/expr/{src/vector_op => impl/src/scalar}/lower.rs (97%) rename src/expr/{src/vector_op => impl/src/scalar}/md5.rs (98%) create mode 100644 src/expr/impl/src/scalar/mod.rs rename src/expr/{src/vector_op => impl/src/scalar}/overlay.rs (95%) rename src/expr/{src/vector_op => impl/src/scalar}/position.rs (91%) rename src/expr/{src/vector_op => impl/src/scalar}/proctime.rs (95%) rename src/expr/{src/vector_op => impl/src/scalar}/regexp.rs (96%) rename src/expr/{src/vector_op => impl/src/scalar}/repeat.rs (93%) rename src/expr/{src/vector_op => impl/src/scalar}/replace.rs (98%) rename src/expr/{src/vector_op => impl/src/scalar}/round.rs (93%) rename src/expr/{src/vector_op => impl/src/scalar}/sha.rs (98%) rename src/expr/{src/vector_op => impl/src/scalar}/split_part.rs (96%) rename src/expr/{src/vector_op => impl/src/scalar}/string.rs (96%) rename src/expr/{src/vector_op => impl/src/scalar}/string_to_array.rs (98%) rename src/expr/{src/vector_op => impl/src/scalar}/substr.rs (94%) rename src/expr/{src/vector_op => impl/src/scalar}/timestamptz.rs (95%) rename src/expr/{src/vector_op => impl/src/scalar}/to_char.rs (97%) rename src/expr/{src/vector_op => impl/src/scalar}/to_timestamp.rs (97%) rename src/expr/{src/vector_op => impl/src/scalar}/translate.rs (98%) rename src/expr/{src/vector_op => impl/src/scalar}/trigonometric.rs (94%) rename src/expr/{src/vector_op => impl/src/scalar}/trim.rs (99%) rename src/expr/{src/vector_op => impl/src/scalar}/trim_array.rs (94%) rename src/expr/{src/vector_op => impl/src/scalar}/tumble.rs (98%) rename src/expr/{src/vector_op => impl/src/scalar}/upper.rs (97%) rename src/expr/{ => impl}/src/table_function/generate_series.rs (88%) rename src/expr/{ => impl}/src/table_function/generate_subscripts.rs (87%) rename src/expr/{ => impl}/src/table_function/jsonb.rs (97%) create mode 100644 src/expr/impl/src/table_function/mod.rs rename src/expr/{ => impl}/src/table_function/pg_expandarray.rs (87%) rename src/expr/{ => impl}/src/table_function/regexp_matches.rs (91%) rename src/expr/{ => impl}/src/table_function/unnest.rs (94%) create mode 100644 src/expr/impl/tests/sig.rs delete mode 100644 src/expr/src/expr/expr_binary_nonnull.rs delete mode 100644 src/expr/src/expr/expr_unary.rs delete mode 100644 src/expr/src/sig/func.rs delete mode 100644 src/expr/src/vector_op/conjunction.rs delete mode 100644 src/expr/src/vector_op/mod.rs diff --git a/Cargo.lock b/Cargo.lock index d46a420f8b71c..876f9f520e1b3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6546,6 +6546,7 @@ dependencies = [ "risingwave_common", "risingwave_connector", "risingwave_expr", + "risingwave_expr_impl", "risingwave_hummock_sdk", "risingwave_pb", "risingwave_rpc_client", @@ -6608,6 +6609,7 @@ dependencies = [ "risingwave_compactor", "risingwave_compute", "risingwave_ctl", + "risingwave_expr_impl", "risingwave_frontend", "risingwave_meta", "risingwave_rt", @@ -6633,6 +6635,7 @@ dependencies = [ "risingwave_compactor", "risingwave_compute", "risingwave_ctl", + "risingwave_expr_impl", "risingwave_frontend", "risingwave_meta", "risingwave_rt", @@ -7002,46 +7005,65 @@ dependencies = [ name = "risingwave_expr" version = "1.3.0-alpha" dependencies = [ - "aho-corasick", "anyhow", "arrow-array", "arrow-schema", "async-trait", - "auto_enums", "await-tree", "cfg-or-panic", "chrono", - "chrono-tz", - "criterion", "ctor", "downcast-rs", "easy-ext", "either", "expect-test", - "fancy-regex", - "futures", "futures-async-stream", "futures-util", - "hex", "itertools 0.11.0", "madsim-tokio", - "md5", "num-traits", "parse-display", "paste", - "regex", "risingwave_common", "risingwave_expr_macro", "risingwave_pb", "risingwave_udf", + "smallvec", + "static_assertions", + "thiserror", + "tracing", + "workspace-hack", +] + +[[package]] +name = "risingwave_expr_impl" +version = "1.3.0-alpha" +dependencies = [ + "aho-corasick", + "anyhow", + "async-trait", + "auto_enums", + "chrono", + "criterion", + "expect-test", + "fancy-regex", + "futures-async-stream", + "futures-util", + "hex", + "itertools 0.11.0", + "madsim-tokio", + "md5", + "num-traits", + "regex", + "risingwave_common", + "risingwave_expr", + "risingwave_pb", "rust_decimal", "self_cell", "serde", "serde_json", "sha1", "sha2", - "smallvec", - "static_assertions", "thiserror", "tracing", "workspace-hack", @@ -7103,6 +7125,7 @@ dependencies = [ "risingwave_common_service", "risingwave_connector", "risingwave_expr", + "risingwave_expr_impl", "risingwave_pb", "risingwave_rpc_client", "risingwave_source", @@ -7351,6 +7374,7 @@ dependencies = [ "libtest-mimic", "madsim-tokio", "paste", + "risingwave_expr_impl", "risingwave_frontend", "risingwave_sqlparser", "serde", @@ -7461,6 +7485,7 @@ dependencies = [ "risingwave_connector", "risingwave_ctl", "risingwave_e2e_extended_mode_test", + "risingwave_expr_impl", "risingwave_frontend", "risingwave_meta", "risingwave_pb", @@ -7545,6 +7570,7 @@ dependencies = [ "regex", "risingwave_common", "risingwave_expr", + "risingwave_expr_impl", "risingwave_frontend", "risingwave_pb", "risingwave_sqlparser", @@ -7680,7 +7706,7 @@ dependencies = [ "risingwave_common", "risingwave_connector", "risingwave_expr", - "risingwave_frontend", + "risingwave_expr_impl", "risingwave_hummock_sdk", "risingwave_hummock_test", "risingwave_pb", diff --git a/Cargo.toml b/Cargo.toml index fe6a516e2dada..c36d9821e389f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -10,7 +10,8 @@ members = [ "src/compute", "src/connector", "src/ctl", - "src/expr", + "src/expr/core", + "src/expr/impl", "src/expr/macro", "src/frontend", "src/frontend/planner_test", @@ -125,7 +126,8 @@ risingwave_compactor = { path = "./src/storage/compactor" } risingwave_compute = { path = "./src/compute" } risingwave_ctl = { path = "./src/ctl" } risingwave_connector = { path = "./src/connector" } -risingwave_expr = { path = "./src/expr" } +risingwave_expr = { path = "./src/expr/core" } +risingwave_expr_impl = { path = "./src/expr/impl" } risingwave_frontend = { path = "./src/frontend" } risingwave_hummock_sdk = { path = "./src/storage/hummock_sdk" } risingwave_hummock_test = { path = "./src/storage/hummock_test" } diff --git a/src/batch/Cargo.toml b/src/batch/Cargo.toml index 35ede881bc47f..fd3f1b85bae7e 100644 --- a/src/batch/Cargo.toml +++ b/src/batch/Cargo.toml @@ -64,6 +64,7 @@ workspace-hack = { path = "../workspace-hack" } [dev-dependencies] criterion = { workspace = true, features = ["async_tokio", "async"] } rand = "0.8" +risingwave_expr_impl = { workspace = true } tempfile = "3" [target.'cfg(unix)'.dev-dependencies] diff --git a/src/batch/benches/hash_agg.rs b/src/batch/benches/hash_agg.rs index e5a561e03a535..16bcd91de9353 100644 --- a/src/batch/benches/hash_agg.rs +++ b/src/batch/benches/hash_agg.rs @@ -22,7 +22,7 @@ use risingwave_common::catalog::{Field, Schema}; use risingwave_common::memory::MemoryContext; use risingwave_common::types::DataType; use risingwave_common::{enable_jemalloc_on_unix, hash}; -use risingwave_expr::agg::{AggCall, AggKind}; +use risingwave_expr::aggregate::{AggCall, AggKind}; use risingwave_pb::expr::{PbAggCall, PbInputRef}; use tokio::runtime::Runtime; use utils::{create_input, execute_executor}; diff --git a/src/batch/src/executor/aggregation/distinct.rs b/src/batch/src/executor/aggregation/distinct.rs index 9ce4b2dc66112..c2844d558acf4 100644 --- a/src/batch/src/executor/aggregation/distinct.rs +++ b/src/batch/src/executor/aggregation/distinct.rs @@ -20,7 +20,7 @@ use risingwave_common::buffer::BitmapBuilder; use risingwave_common::estimate_size::EstimateSize; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{DataType, Datum}; -use risingwave_expr::agg::{ +use risingwave_expr::aggregate::{ AggStateDyn, AggregateFunction, AggregateState, BoxedAggregateFunction, }; use risingwave_expr::Result; @@ -112,7 +112,7 @@ mod tests { use risingwave_common::array::StreamChunk; use risingwave_common::test_prelude::StreamChunkTestExt; use risingwave_common::types::{Datum, Decimal}; - use risingwave_expr::agg::AggCall; + use risingwave_expr::aggregate::AggCall; use super::super::build; diff --git a/src/batch/src/executor/aggregation/filter.rs b/src/batch/src/executor/aggregation/filter.rs index 06e8d894d5278..2db2320ed3534 100644 --- a/src/batch/src/executor/aggregation/filter.rs +++ b/src/batch/src/executor/aggregation/filter.rs @@ -17,7 +17,7 @@ use std::sync::Arc; use risingwave_common::array::StreamChunk; use risingwave_common::types::{DataType, Datum}; -use risingwave_expr::agg::{AggregateFunction, AggregateState, BoxedAggregateFunction}; +use risingwave_expr::aggregate::{AggregateFunction, AggregateState, BoxedAggregateFunction}; use risingwave_expr::expr::Expression; use risingwave_expr::Result; @@ -74,7 +74,7 @@ impl AggregateFunction for Filter { #[cfg(test)] mod tests { use risingwave_common::test_prelude::StreamChunkTestExt; - use risingwave_expr::agg::{build_append_only, AggCall}; + use risingwave_expr::aggregate::{build_append_only, AggCall}; use risingwave_expr::expr::{build_from_pretty, Expression, LiteralExpression}; use super::*; diff --git a/src/batch/src/executor/aggregation/mod.rs b/src/batch/src/executor/aggregation/mod.rs index a794823e75636..e756b126f013b 100644 --- a/src/batch/src/executor/aggregation/mod.rs +++ b/src/batch/src/executor/aggregation/mod.rs @@ -20,7 +20,7 @@ mod filter; mod orderby; mod projection; -use risingwave_expr::agg::{self, AggCall, BoxedAggregateFunction}; +use risingwave_expr::aggregate::{build_append_only, AggCall, BoxedAggregateFunction}; use risingwave_expr::Result; use self::distinct::Distinct; @@ -30,7 +30,7 @@ use self::projection::Projection; /// Build an `BoxedAggregateFunction` from `AggCall`. pub fn build(agg: &AggCall) -> Result { - let mut aggregator = agg::build_append_only(agg)?; + let mut aggregator = build_append_only(agg)?; if agg.distinct { aggregator = Box::new(Distinct::new(aggregator)); diff --git a/src/batch/src/executor/aggregation/orderby.rs b/src/batch/src/executor/aggregation/orderby.rs index 47ef7189bb6c4..427fa15688ca3 100644 --- a/src/batch/src/executor/aggregation/orderby.rs +++ b/src/batch/src/executor/aggregation/orderby.rs @@ -22,7 +22,7 @@ use risingwave_common::types::{DataType, Datum}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::memcmp_encoding; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; -use risingwave_expr::agg::{ +use risingwave_expr::aggregate::{ AggStateDyn, AggregateFunction, AggregateState, BoxedAggregateFunction, }; use risingwave_expr::{ExprError, Result}; @@ -151,7 +151,7 @@ impl AggregateFunction for ProjectionOrderBy { mod tests { use risingwave_common::array::{ListValue, StreamChunk}; use risingwave_common::test_prelude::StreamChunkTestExt; - use risingwave_expr::agg::AggCall; + use risingwave_expr::aggregate::AggCall; use super::super::build; diff --git a/src/batch/src/executor/aggregation/projection.rs b/src/batch/src/executor/aggregation/projection.rs index 890d8da58f1a8..00343891e9ae6 100644 --- a/src/batch/src/executor/aggregation/projection.rs +++ b/src/batch/src/executor/aggregation/projection.rs @@ -16,7 +16,7 @@ use std::ops::Range; use risingwave_common::array::StreamChunk; use risingwave_common::types::{DataType, Datum}; -use risingwave_expr::agg::{AggregateFunction, AggregateState, BoxedAggregateFunction}; +use risingwave_expr::aggregate::{AggregateFunction, AggregateState, BoxedAggregateFunction}; use risingwave_expr::Result; pub struct Projection { diff --git a/src/batch/src/executor/hash_agg.rs b/src/batch/src/executor/hash_agg.rs index d4480d9bd4064..03ce86d475620 100644 --- a/src/batch/src/executor/hash_agg.rs +++ b/src/batch/src/executor/hash_agg.rs @@ -24,7 +24,7 @@ use risingwave_common::hash::{HashKey, HashKeyDispatcher, PrecomputedBuildHasher use risingwave_common::memory::MemoryContext; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_expr::agg::{AggCall, AggregateState, BoxedAggregateFunction}; +use risingwave_expr::aggregate::{AggCall, AggregateState, BoxedAggregateFunction}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashAggNode; diff --git a/src/batch/src/executor/hop_window.rs b/src/batch/src/executor/hop_window.rs index 85e485db00376..01ba275e730ee 100644 --- a/src/batch/src/executor/hop_window.rs +++ b/src/batch/src/executor/hop_window.rs @@ -239,7 +239,7 @@ mod tests { 6 2 ^10:42:00 7 1 ^10:51:00 8 3 ^11:02:00" - .replace('^', "2022-2-2T"), + .replace('^', "2022-02-02T"), ); let mut mock_executor = MockExecutor::new(schema.clone()); mock_executor.add(chunk); @@ -326,7 +326,7 @@ mod tests { 6 2 ^10:42:00 ^10:14:00 ^10:44:00 7 1 ^10:51:00 ^10:29:00 ^10:59:00 8 3 ^11:02:00 ^10:44:00 ^11:14:00" - .replace('^', "2022-2-2T"), + .replace('^', "2022-02-02T"), ) ); assert_eq!( @@ -341,7 +341,7 @@ mod tests { 6 2 ^10:42:00 ^10:14:00 ^10:44:00 7 1 ^10:51:00 ^10:29:00 ^10:59:00 8 3 ^11:02:00 ^10:44:00 ^11:14:00" - .replace('^', "2022-2-2T"), + .replace('^', "2022-02-02T"), ) ); } @@ -371,7 +371,7 @@ mod tests { 6 2 ^10:42:00 ^10:15:00 ^10:45:00 7 1 ^10:51:00 ^10:30:00 ^11:00:00 8 3 ^11:02:00 ^10:45:00 ^11:15:00" - .replace('^', "2022-2-2T"), + .replace('^', "2022-02-02T"), ) ); @@ -388,7 +388,7 @@ mod tests { 6 2 ^10:42:00 ^10:30:00 ^11:00:00 7 1 ^10:51:00 ^10:45:00 ^11:15:00 8 3 ^11:02:00 ^11:00:00 ^11:30:00" - .replace('^', "2022-2-2T"), + .replace('^', "2022-02-02T"), ) ); } @@ -415,7 +415,7 @@ mod tests { 2 ^10:15:00 ^10:45:00 ^10:42:00 1 ^10:30:00 ^11:00:00 ^10:51:00 3 ^10:45:00 ^11:15:00 ^11:02:00" - .replace('^', "2022-2-2T"), + .replace('^', "2022-02-02T"), ) ); @@ -432,7 +432,7 @@ mod tests { 2 ^10:30:00 ^11:00:00 ^10:42:00 1 ^10:45:00 ^11:15:00 ^10:51:00 3 ^11:00:00 ^11:30:00 ^11:02:00" - .replace('^', "2022-2-2T"), + .replace('^', "2022-02-02T"), ) ); } diff --git a/src/batch/src/executor/join/local_lookup_join.rs b/src/batch/src/executor/join/local_lookup_join.rs index af2bd80f24198..31d101c1ecfa3 100644 --- a/src/batch/src/executor/join/local_lookup_join.rs +++ b/src/batch/src/executor/join/local_lookup_join.rs @@ -692,7 +692,7 @@ mod tests { 2 5.5 2 5.5 2 8.4 2 5.5", ); - let condition = build_from_pretty("(less_than:boolean (cast:float4 5:int4) $3:float4)"); + let condition = build_from_pretty("(less_than:boolean 5:float4 $3:float4)"); do_test(JoinType::Inner, Some(condition), false, expected).await; } @@ -709,7 +709,7 @@ mod tests { 5 9.1 . . . . . .", ); - let condition = build_from_pretty("(less_than:boolean (cast:float4 5:int4) $3:float4)"); + let condition = build_from_pretty("(less_than:boolean 5:float4 $3:float4)"); do_test(JoinType::LeftOuter, Some(condition), false, expected).await; } @@ -722,7 +722,7 @@ mod tests { 2 5.5 2 8.4", ); - let condition = build_from_pretty("(less_than:boolean (cast:float4 5:int4) $3:float4)"); + let condition = build_from_pretty("(less_than:boolean 5:float4 $3:float4)"); do_test(JoinType::LeftSemi, Some(condition), false, expected).await; } @@ -736,7 +736,7 @@ mod tests { 5 9.1 . .", ); - let condition = build_from_pretty("(less_than:boolean (cast:float4 5:int4) $3:float4)"); + let condition = build_from_pretty("(less_than:boolean 5:float4 $3:float4)"); do_test(JoinType::LeftAnti, Some(condition), false, expected).await; } diff --git a/src/batch/src/executor/sort_agg.rs b/src/batch/src/executor/sort_agg.rs index 8a1427ffc6d6a..e8f07bc5fa5d9 100644 --- a/src/batch/src/executor/sort_agg.rs +++ b/src/batch/src/executor/sort_agg.rs @@ -20,7 +20,7 @@ use risingwave_common::array::{Array, ArrayBuilderImpl, ArrayImpl, DataChunk, St use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::{Result, RwError}; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_expr::agg::{AggCall, AggregateState, BoxedAggregateFunction}; +use risingwave_expr::aggregate::{AggCall, AggregateState, BoxedAggregateFunction}; use risingwave_expr::expr::{build_from_prost, BoxedExpression}; use risingwave_pb::batch_plan::plan_node::NodeBody; diff --git a/src/batch/src/lib.rs b/src/batch/src/lib.rs index ac062a16c1c10..17fc6cfab2bc1 100644 --- a/src/batch/src/lib.rs +++ b/src/batch/src/lib.rs @@ -48,3 +48,6 @@ pub mod task; extern crate tracing; #[macro_use] extern crate risingwave_common; + +#[cfg(test)] +risingwave_expr_impl::enable!(); diff --git a/src/cmd/Cargo.toml b/src/cmd/Cargo.toml index 894d19c9f969a..2837cb8ce2340 100644 --- a/src/cmd/Cargo.toml +++ b/src/cmd/Cargo.toml @@ -25,6 +25,7 @@ risingwave_common = { workspace = true } risingwave_compactor = { workspace = true } risingwave_compute = { workspace = true } risingwave_ctl = { workspace = true } +risingwave_expr_impl = { workspace = true } risingwave_frontend = { workspace = true } risingwave_meta = { workspace = true } risingwave_rt = { workspace = true } diff --git a/src/cmd/src/lib.rs b/src/cmd/src/lib.rs index 48cbaaa0e63a2..6b748bf067c75 100644 --- a/src/cmd/src/lib.rs +++ b/src/cmd/src/lib.rs @@ -37,6 +37,8 @@ macro_rules! main { }; } +risingwave_expr_impl::enable!(); + // Entry point functions. pub fn compute(opts: ComputeNodeOpts) { diff --git a/src/cmd_all/Cargo.toml b/src/cmd_all/Cargo.toml index f22f0b59ea2db..ab185abeb46c7 100644 --- a/src/cmd_all/Cargo.toml +++ b/src/cmd_all/Cargo.toml @@ -29,6 +29,7 @@ risingwave_common = { workspace = true } risingwave_compactor = { workspace = true } risingwave_compute = { workspace = true } risingwave_ctl = { workspace = true } +risingwave_expr_impl = { workspace = true } risingwave_frontend = { workspace = true } risingwave_meta = { workspace = true } risingwave_rt = { workspace = true } diff --git a/src/cmd_all/src/lib.rs b/src/cmd_all/src/lib.rs index be6a4e1b70a29..c3d5e60f01a05 100644 --- a/src/cmd_all/src/lib.rs +++ b/src/cmd_all/src/lib.rs @@ -20,3 +20,5 @@ mod standalone; pub use playground::*; pub use standalone::*; + +risingwave_expr_impl::enable!(); diff --git a/src/common/src/cast/mod.rs b/src/common/src/cast/mod.rs index 82c69984ec0ea..21a217967830a 100644 --- a/src/common/src/cast/mod.rs +++ b/src/common/src/cast/mod.rs @@ -12,81 +12,39 @@ // See the License for the specific language governing permissions and // limitations under the License. -use chrono::{NaiveDate, NaiveDateTime, NaiveTime}; use itertools::Itertools; -use speedate::{Date as SpeedDate, DateTime as SpeedDateTime, Time as SpeedTime}; -use crate::types::{Date, Time, Timestamp, Timestamptz}; +use crate::types::{Timestamp, Timestamptz}; type Result = std::result::Result; -pub const PARSE_ERROR_STR_TO_TIMESTAMP: &str = "Can't cast string to timestamp (expected format is YYYY-MM-DD HH:MM:SS[.D+{up to 6 digits}] or YYYY-MM-DD HH:MM or YYYY-MM-DD or ISO 8601 format)"; -pub const PARSE_ERROR_STR_TO_TIME: &str = - "Can't cast string to time (expected format is HH:MM:SS[.D+{up to 6 digits}][Z] or HH:MM)"; -pub const PARSE_ERROR_STR_TO_DATE: &str = - "Can't cast string to date (expected format is YYYY-MM-DD)"; pub const PARSE_ERROR_STR_TO_BYTEA: &str = "Invalid Bytea syntax"; const ERROR_INT_TO_TIMESTAMP: &str = "Can't cast negative integer to timestamp"; -pub fn str_to_date(elem: &str) -> Result { - Ok(Date::new(parse_naive_date(elem)?)) -} - -pub fn str_to_time(elem: &str) -> Result