From 5bd5f44a5933d0a095f719223db24cbf2e592860 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 4 Sep 2023 17:53:47 +0800 Subject: [PATCH 1/9] refactor source with trait --- Cargo.lock | 1 + src/connector/Cargo.toml | 1 + src/connector/src/macros.rs | 138 ++---------- src/connector/src/source/base.rs | 180 ++++++--------- .../src/source/cdc/enumerator/mod.rs | 132 ++++++----- src/connector/src/source/cdc/mod.rs | 95 ++++++-- src/connector/src/source/cdc/source/reader.rs | 44 ++-- src/connector/src/source/cdc/split.rs | 12 +- src/connector/src/source/common.rs | 76 +++++++ src/connector/src/source/datagen/mod.rs | 10 + .../src/source/datagen/source/reader.rs | 52 ++--- src/connector/src/source/dummy_connector.rs | 45 ---- src/connector/src/source/filesystem/s3/mod.rs | 10 + .../src/source/filesystem/s3/source/reader.rs | 11 +- src/connector/src/source/google_pubsub/mod.rs | 10 + .../src/source/google_pubsub/source/reader.rs | 28 +-- src/connector/src/source/kafka/mod.rs | 10 + .../src/source/kafka/source/reader.rs | 27 +-- src/connector/src/source/kinesis/mod.rs | 12 + .../src/source/kinesis/source/reader.rs | 44 ++-- src/connector/src/source/mod.rs | 4 +- src/connector/src/source/nexmark/mod.rs | 11 + .../src/source/nexmark/source/reader.rs | 18 +- src/connector/src/source/pulsar/mod.rs | 10 + .../src/source/pulsar/source/reader.rs | 25 +- src/frontend/src/scheduler/plan_fragmenter.rs | 15 +- src/meta/src/rpc/service/cloud_service.rs | 52 +++-- src/meta/src/stream/source_manager.rs | 213 ++++++++++-------- src/source/src/connector_source.rs | 93 ++++---- src/source/src/fs_connector_source.rs | 24 +- .../src/executor/source/fs_source_executor.rs | 5 +- 31 files changed, 747 insertions(+), 661 deletions(-) create mode 100644 src/connector/src/source/common.rs delete mode 100644 src/connector/src/source/dummy_connector.rs diff --git a/Cargo.lock b/Cargo.lock index dbf61ce31ee26..68242a7f36bb5 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6566,6 +6566,7 @@ dependencies = [ "num-bigint", "opendal", "parking_lot 0.12.1", + "paste", "prometheus", "prost", "prost-reflect", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 5ab001cfe7a11..315c32b677a8a 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -57,6 +57,7 @@ nexmark = { version = "0.2", features = ["serde"] } num-bigint = "0.4" opendal = "0.39" parking_lot = "0.12" +paste = "1" prometheus = { version = "0.13", features = ["process"] } prost = { version = "0.11.9", features = ["no-recursion-limit"] } prost-reflect = "0.11.4" diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index 70fb6130b8717..feac1f63cdec2 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -12,39 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#[macro_export] -macro_rules! impl_split_enumerator { - ($({ $variant_name:ident, $split_enumerator_name:ident} ),*) => { - impl SplitEnumeratorImpl { - - pub async fn create(properties: ConnectorProperties, context: SourceEnumeratorContextRef) -> Result { - match properties { - $( ConnectorProperties::$variant_name(props) => $split_enumerator_name::new(*props, context).await.map(Self::$variant_name), )* - other => Err(anyhow!( - "split enumerator type for config {:?} is not supported", - other - )), - } - } - - pub async fn list_splits(&mut self) -> Result> { - match self { - $( Self::$variant_name(inner) => inner - .list_splits() - .await - .map(|ss| { - ss.into_iter() - .map(SplitImpl::$variant_name) - .collect_vec() - }) - .map_err(|e| ErrorCode::ConnectorError(e.into()).into()), - )* - } - } - } - } -} - #[macro_export] macro_rules! impl_split { ($({ $variant_name:ident, $connector_name:ident, $split:ty} ),*) => { @@ -55,6 +22,25 @@ macro_rules! impl_split { } } } + $( + impl TryFrom for $split { + type Error = anyhow::Error; + + fn try_from(split: SplitImpl) -> std::result::Result { + match split { + SplitImpl::$variant_name(inner) => Ok(inner), + other => Err(anyhow::anyhow!("expect {} but get {:?}", stringify!($split), other)) + } + } + } + + impl From<$split> for SplitImpl { + fn from(split: $split) -> SplitImpl { + SplitImpl::$variant_name(split) + } + } + + )* impl TryFrom<&ConnectorSplit> for SplitImpl { type Error = anyhow::Error; @@ -124,36 +110,6 @@ macro_rules! impl_split { } } -#[macro_export] -macro_rules! impl_split_reader { - ($({ $variant_name:ident, $split_reader_name:ident} ),*) => { - impl SplitReaderImpl { - pub fn into_stream(self) -> BoxSourceWithStateStream { - match self { - $( Self::$variant_name(inner) => inner.into_stream(), )* } - } - - pub async fn create( - config: ConnectorProperties, - state: ConnectorState, - parser_config: ParserConfig, - source_ctx: SourceContextRef, - columns: Option>, - ) -> Result { - if state.is_none() { - return Ok(Self::Dummy(Box::new(DummySplitReader {}))); - } - let splits = state.unwrap(); - let connector = match config { - $( ConnectorProperties::$variant_name(props) => Self::$variant_name(Box::new($split_reader_name::new(*props, splits, parser_config, source_ctx, columns).await?)), )* - }; - - Ok(connector) - } - } - } -} - #[macro_export] macro_rules! impl_connector_properties { ($({ $variant_name:ident, $connector_name:ident } ),*) => { @@ -161,7 +117,7 @@ macro_rules! impl_connector_properties { pub fn extract(mut props: HashMap) -> Result { const UPSTREAM_SOURCE_KEY: &str = "connector"; let connector = props.remove(UPSTREAM_SOURCE_KEY).ok_or_else(|| anyhow!("Must specify 'connector' in WITH clause"))?; - if connector.ends_with("cdc") { + if connector.ends_with("-cdc") { ConnectorProperties::new_cdc_properties(&connector, props) } else { let json_value = serde_json::to_value(props).map_err(|e| anyhow!(e))?; @@ -180,57 +136,3 @@ macro_rules! impl_connector_properties { } } } - -#[macro_export] -macro_rules! impl_common_split_reader_logic { - ($reader:ty, $props:ty) => { - impl $reader { - #[try_stream(boxed, ok = $crate::source::StreamChunkWithState, error = risingwave_common::error::RwError)] - pub(crate) async fn into_chunk_stream(self) { - let parser_config = self.parser_config.clone(); - let actor_id = self.source_ctx.source_info.actor_id.to_string(); - let source_id = self.source_ctx.source_info.source_id.to_string(); - let metrics = self.source_ctx.metrics.clone(); - let source_ctx = self.source_ctx.clone(); - - let data_stream = self.into_data_stream(); - - let data_stream = data_stream - .inspect_ok(move |data_batch| { - let mut by_split_id = std::collections::HashMap::new(); - - for msg in data_batch { - by_split_id - .entry(msg.split_id.as_ref()) - .or_insert_with(Vec::new) - .push(msg); - } - - for (split_id, msgs) in by_split_id { - metrics - .partition_input_count - .with_label_values(&[&actor_id, &source_id, split_id]) - .inc_by(msgs.len() as u64); - - let sum_bytes = msgs - .iter() - .flat_map(|msg| msg.payload.as_ref().map(|p| p.len() as u64)) - .sum(); - - metrics - .partition_input_bytes - .with_label_values(&[&actor_id, &source_id, &split_id]) - .inc_by(sum_bytes); - } - }).boxed(); - - let parser = - $crate::parser::ByteStreamSourceParserImpl::create(parser_config, source_ctx).await?; - #[for_await] - for msg_batch in parser.into_stream(data_stream) { - yield msg_batch?; - } - } - } - }; -} diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 8b5b3287fb445..24d5ab38199f6 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -25,58 +25,60 @@ use itertools::Itertools; use parking_lot::Mutex; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::TableId; -use risingwave_common::error::{ErrorCode, ErrorSuppressor, Result as RwResult, RwError}; +use risingwave_common::error::{ErrorSuppressor, Result as RwResult, RwError}; use risingwave_common::types::{JsonbVal, Scalar}; use risingwave_pb::connector_service::PbTableSchema; use risingwave_pb::source::ConnectorSplit; use risingwave_rpc_client::ConnectorClient; -use serde::{Deserialize, Serialize}; use super::datagen::DatagenMeta; -use super::filesystem::{FsSplit, S3FileReader, S3Properties, S3SplitEnumerator, S3_CONNECTOR}; +use super::filesystem::{FsSplit, S3Properties, S3_CONNECTOR}; use super::google_pubsub::GooglePubsubMeta; use super::kafka::KafkaMeta; use super::monitor::SourceMetrics; use super::nexmark::source::message::NexmarkMeta; use crate::parser::ParserConfig; use crate::source::cdc::{ - CdcProperties, CdcSplitReader, DebeziumCdcSplit, DebeziumSplitEnumerator, CITUS_CDC_CONNECTOR, + CdcProperties, Citus, DebeziumCdcSplit, Mysql, Postgres, CITUS_CDC_CONNECTOR, MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, }; -use crate::source::datagen::{ - DatagenProperties, DatagenSplit, DatagenSplitEnumerator, DatagenSplitReader, DATAGEN_CONNECTOR, -}; -use crate::source::dummy_connector::DummySplitReader; -use crate::source::google_pubsub::{ - PubsubProperties, PubsubSplit, PubsubSplitEnumerator, PubsubSplitReader, - GOOGLE_PUBSUB_CONNECTOR, -}; -use crate::source::kafka::enumerator::KafkaSplitEnumerator; -use crate::source::kafka::source::KafkaSplitReader; +pub(crate) use crate::source::common::CommonSplitReader; +use crate::source::datagen::{DatagenProperties, DatagenSplit, DATAGEN_CONNECTOR}; +use crate::source::google_pubsub::{PubsubProperties, PubsubSplit, GOOGLE_PUBSUB_CONNECTOR}; use crate::source::kafka::{KafkaProperties, KafkaSplit, KAFKA_CONNECTOR}; -use crate::source::kinesis::enumerator::client::KinesisSplitEnumerator; -use crate::source::kinesis::source::reader::KinesisSplitReader; use crate::source::kinesis::split::KinesisSplit; use crate::source::kinesis::{KinesisProperties, KINESIS_CONNECTOR}; use crate::source::monitor::EnumeratorMetrics; -use crate::source::nexmark::source::reader::NexmarkSplitReader; -use crate::source::nexmark::{ - NexmarkProperties, NexmarkSplit, NexmarkSplitEnumerator, NEXMARK_CONNECTOR, -}; -use crate::source::pulsar::source::reader::PulsarSplitReader; -use crate::source::pulsar::{ - PulsarProperties, PulsarSplit, PulsarSplitEnumerator, PULSAR_CONNECTOR, -}; -use crate::{impl_connector_properties, impl_split, impl_split_enumerator, impl_split_reader}; +use crate::source::nexmark::{NexmarkProperties, NexmarkSplit, NEXMARK_CONNECTOR}; +use crate::source::pulsar::{PulsarProperties, PulsarSplit, PULSAR_CONNECTOR}; +use crate::{impl_connector_properties, impl_split}; const SPLIT_TYPE_FIELD: &str = "split_type"; const SPLIT_INFO_FIELD: &str = "split_info"; +pub trait SourceProperties: Clone { + const SOURCE_NAME: &'static str; + type Split: SplitMetaData + TryFrom + Into; + type SplitEnumerator: SplitEnumerator; + type SplitReader: SplitReader; +} + +pub async fn create_split_reader( + prop: P, + splits: Vec, + parser_config: ParserConfig, + source_ctx: SourceContextRef, + columns: Option>, +) -> Result { + let splits = splits.into_iter().map(P::Split::try_from).try_collect()?; + P::SplitReader::new(prop, splits, parser_config, source_ctx, columns).await +} + /// [`SplitEnumerator`] fetches the split metadata from the external source service. /// NOTE: It runs in the meta server, so probably it should be moved to the `meta` crate. #[async_trait] pub trait SplitEnumerator: Sized { - type Split: SplitMetaData + Send + Sync; + type Split: SplitMetaData + Send; type Properties; async fn new(properties: Self::Properties, context: SourceEnumeratorContextRef) @@ -273,12 +275,13 @@ impl From for StreamChunkWithState { /// responsible for parsing, it is used to read messages from the outside and transform them into a /// stream of parsed [`StreamChunk`] #[async_trait] -pub trait SplitReader: Sized { +pub trait SplitReader: Sized + Send { type Properties; + type Split: SplitMetaData; async fn new( properties: Self::Properties, - state: Vec, + state: Vec, parser_config: ParserConfig, source_ctx: SourceContextRef, columns: Option>, @@ -287,7 +290,7 @@ pub trait SplitReader: Sized { fn into_stream(self) -> BoxSourceWithStateStream; } -#[derive(Clone, Debug, Deserialize)] +#[derive(Clone, Debug)] pub enum ConnectorProperties { Kafka(Box), Pulsar(Box), @@ -295,11 +298,30 @@ pub enum ConnectorProperties { Nexmark(Box), Datagen(Box), S3(Box), - MySqlCdc(Box), - PostgresCdc(Box), - CitusCdc(Box), + MySqlCdc(Box>), + PostgresCdc(Box>), + CitusCdc(Box>), GooglePubsub(Box), - Dummy(Box<()>), +} + +#[macro_export] +macro_rules! dispatch_source_prop { + ($impl:expr, $source_prop:ident, $body:tt) => {{ + use $crate::source::base::ConnectorProperties; + + match $impl { + ConnectorProperties::Kafka($source_prop) => $body, + ConnectorProperties::Pulsar($source_prop) => $body, + ConnectorProperties::Kinesis($source_prop) => $body, + ConnectorProperties::Nexmark($source_prop) => $body, + ConnectorProperties::Datagen($source_prop) => $body, + ConnectorProperties::S3($source_prop) => $body, + ConnectorProperties::MySqlCdc($source_prop) => $body, + ConnectorProperties::PostgresCdc($source_prop) => $body, + ConnectorProperties::CitusCdc($source_prop) => $body, + ConnectorProperties::GooglePubsub($source_prop) => $body, + } + }}; } impl ConnectorProperties { @@ -308,19 +330,16 @@ impl ConnectorProperties { properties: HashMap, ) -> Result { match connector_name { - MYSQL_CDC_CONNECTOR => Ok(Self::MySqlCdc(Box::new(CdcProperties { + MYSQL_CDC_CONNECTOR => Ok(Self::MySqlCdc(Box::new(CdcProperties:: { props: properties, - source_type: "mysql".to_string(), ..Default::default() }))), - POSTGRES_CDC_CONNECTOR => Ok(Self::PostgresCdc(Box::new(CdcProperties { + POSTGRES_CDC_CONNECTOR => Ok(Self::PostgresCdc(Box::new(CdcProperties:: { props: properties, - source_type: "postgres".to_string(), ..Default::default() }))), - CITUS_CDC_CONNECTOR => Ok(Self::CitusCdc(Box::new(CdcProperties { + CITUS_CDC_CONNECTOR => Ok(Self::CitusCdc(Box::new(CdcProperties:: { props: properties, - source_type: "citus".to_string(), ..Default::default() }))), _ => Err(anyhow!("unexpected cdc connector '{}'", connector_name,)), @@ -329,9 +348,13 @@ impl ConnectorProperties { pub fn init_cdc_properties(&mut self, table_schema: Option) { match self { - ConnectorProperties::MySqlCdc(c) - | ConnectorProperties::PostgresCdc(c) - | ConnectorProperties::CitusCdc(c) => { + ConnectorProperties::MySqlCdc(c) => { + c.table_schema = table_schema; + } + ConnectorProperties::PostgresCdc(c) => { + c.table_schema = table_schema; + } + ConnectorProperties::CitusCdc(c) => { c.table_schema = table_schema; } _ => {} @@ -352,7 +375,7 @@ impl ConnectorProperties { } } -#[derive(Debug, Clone, Serialize, Deserialize, EnumAsInner, PartialEq, Hash)] +#[derive(Debug, Clone, EnumAsInner, PartialEq, Hash)] pub enum SplitImpl { Kafka(KafkaSplit), Pulsar(PulsarSplit), @@ -360,9 +383,9 @@ pub enum SplitImpl { Nexmark(NexmarkSplit), Datagen(DatagenSplit), GooglePubsub(PubsubSplit), - MySqlCdc(DebeziumCdcSplit), - PostgresCdc(DebeziumCdcSplit), - CitusCdc(DebeziumCdcSplit), + MySqlCdc(DebeziumCdcSplit), + PostgresCdc(DebeziumCdcSplit), + CitusCdc(DebeziumCdcSplit), S3(FsSplit), } @@ -384,33 +407,6 @@ impl SplitImpl { } } -pub enum SplitReaderImpl { - S3(Box), - Dummy(Box), - Kinesis(Box), - Kafka(Box), - Nexmark(Box), - Pulsar(Box), - Datagen(Box), - MySqlCdc(Box), - PostgresCdc(Box), - CitusCdc(Box), - GooglePubsub(Box), -} - -pub enum SplitEnumeratorImpl { - Kafka(KafkaSplitEnumerator), - Pulsar(PulsarSplitEnumerator), - Kinesis(KinesisSplitEnumerator), - Nexmark(NexmarkSplitEnumerator), - Datagen(DatagenSplitEnumerator), - MySqlCdc(DebeziumSplitEnumerator), - PostgresCdc(DebeziumSplitEnumerator), - CitusCdc(DebeziumSplitEnumerator), - GooglePubsub(PubsubSplitEnumerator), - S3(S3SplitEnumerator), -} - impl_connector_properties! { { Kafka, KAFKA_CONNECTOR }, { Pulsar, PULSAR_CONNECTOR }, @@ -418,25 +414,9 @@ impl_connector_properties! { { Nexmark, NEXMARK_CONNECTOR }, { Datagen, DATAGEN_CONNECTOR }, { S3, S3_CONNECTOR }, - { MySqlCdc, MYSQL_CDC_CONNECTOR }, - { PostgresCdc, POSTGRES_CDC_CONNECTOR }, - { CitusCdc, CITUS_CDC_CONNECTOR }, { GooglePubsub, GOOGLE_PUBSUB_CONNECTOR} } -impl_split_enumerator! { - { Kafka, KafkaSplitEnumerator }, - { Pulsar, PulsarSplitEnumerator }, - { Kinesis, KinesisSplitEnumerator }, - { Nexmark, NexmarkSplitEnumerator }, - { Datagen, DatagenSplitEnumerator }, - { MySqlCdc, DebeziumSplitEnumerator }, - { PostgresCdc, DebeziumSplitEnumerator }, - { CitusCdc, DebeziumSplitEnumerator }, - { GooglePubsub, PubsubSplitEnumerator}, - { S3, S3SplitEnumerator } -} - impl_split! { { Kafka, KAFKA_CONNECTOR, KafkaSplit }, { Pulsar, PULSAR_CONNECTOR, PulsarSplit }, @@ -444,26 +424,12 @@ impl_split! { { Nexmark, NEXMARK_CONNECTOR, NexmarkSplit }, { Datagen, DATAGEN_CONNECTOR, DatagenSplit }, { GooglePubsub, GOOGLE_PUBSUB_CONNECTOR, PubsubSplit }, - { MySqlCdc, MYSQL_CDC_CONNECTOR, DebeziumCdcSplit }, - { PostgresCdc, POSTGRES_CDC_CONNECTOR, DebeziumCdcSplit }, - { CitusCdc, CITUS_CDC_CONNECTOR, DebeziumCdcSplit }, + { MySqlCdc, MYSQL_CDC_CONNECTOR, DebeziumCdcSplit }, + { PostgresCdc, POSTGRES_CDC_CONNECTOR, DebeziumCdcSplit }, + { CitusCdc, CITUS_CDC_CONNECTOR, DebeziumCdcSplit }, { S3, S3_CONNECTOR, FsSplit } } -impl_split_reader! { - { S3, S3FileReader }, - { Kafka, KafkaSplitReader }, - { Pulsar, PulsarSplitReader }, - { Kinesis, KinesisSplitReader }, - { Nexmark, NexmarkSplitReader }, - { Datagen, DatagenSplitReader }, - { MySqlCdc, CdcSplitReader}, - { PostgresCdc, CdcSplitReader}, - { CitusCdc, CdcSplitReader }, - { GooglePubsub, PubsubSplitReader }, - { Dummy, DummySplitReader } -} - pub type DataType = risingwave_common::types::DataType; #[derive(Clone, Debug)] @@ -645,7 +611,6 @@ mod tests { let conn_props = ConnectorProperties::extract(user_props_mysql).unwrap(); if let ConnectorProperties::MySqlCdc(c) = conn_props { - assert_eq!(c.source_type, "mysql"); assert_eq!(c.props.get("connector_node_addr").unwrap(), "localhost"); assert_eq!(c.props.get("database.hostname").unwrap(), "127.0.0.1"); assert_eq!(c.props.get("database.port").unwrap(), "3306"); @@ -659,7 +624,6 @@ mod tests { let conn_props = ConnectorProperties::extract(user_props_postgres).unwrap(); if let ConnectorProperties::PostgresCdc(c) = conn_props { - assert_eq!(c.source_type, "postgres"); assert_eq!(c.props.get("connector_node_addr").unwrap(), "localhost"); assert_eq!(c.props.get("database.hostname").unwrap(), "127.0.0.1"); assert_eq!(c.props.get("database.port").unwrap(), "5432"); diff --git a/src/connector/src/source/cdc/enumerator/mod.rs b/src/connector/src/source/cdc/enumerator/mod.rs index 91d2bf0bb7bbf..6ced5ee79da1e 100644 --- a/src/connector/src/source/cdc/enumerator/mod.rs +++ b/src/connector/src/source/cdc/enumerator/mod.rs @@ -12,38 +12,43 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::marker::PhantomData; use std::str::FromStr; use anyhow::anyhow; use async_trait::async_trait; use itertools::Itertools; use risingwave_common::util::addr::HostAddr; -use risingwave_pb::connector_service::SourceType as PbSourceType; +use risingwave_pb::connector_service::SourceType; use crate::source::cdc::{ - CdcProperties, CdcSplitBase, DebeziumCdcSplit, MySqlCdcSplit, PostgresCdcSplit, + CdcProperties, CdcSourceTypeTrait, CdcSplitBase, Citus, DebeziumCdcSplit, MySqlCdcSplit, Mysql, + Postgres, PostgresCdcSplit, }; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; pub const DATABASE_SERVERS_KEY: &str = "database.servers"; #[derive(Debug)] -pub struct DebeziumSplitEnumerator { +pub struct DebeziumSplitEnumerator { /// The source_id in the catalog source_id: u32, - source_type: PbSourceType, worker_node_addrs: Vec, + _phantom: PhantomData, } #[async_trait] -impl SplitEnumerator for DebeziumSplitEnumerator { - type Properties = CdcProperties; - type Split = DebeziumCdcSplit; +impl SplitEnumerator for DebeziumSplitEnumerator +where + Self: ListCdcSplits, +{ + type Properties = CdcProperties; + type Split = DebeziumCdcSplit; async fn new( - props: CdcProperties, + props: CdcProperties, context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> anyhow::Result { let connector_client = context.connector_client.clone().ok_or_else(|| { anyhow!("connector node endpoint not specified or unable to connect to connector node") })?; @@ -59,12 +64,16 @@ impl SplitEnumerator for DebeziumSplitEnumerator { .transpose()? .unwrap_or_default(); - let source_type = props.get_source_type_pb()?; + assert_eq!( + props.get_source_type_pb(), + SourceType::from(T::source_type()) + ); + // validate connector properties connector_client .validate_source_properties( context.info.source_id as u64, - props.get_source_type_pb()?, + props.get_source_type_pb(), props.props, props.table_schema, ) @@ -73,54 +82,73 @@ impl SplitEnumerator for DebeziumSplitEnumerator { tracing::debug!("validate cdc source properties success"); Ok(Self { source_id: context.info.source_id, - source_type, worker_node_addrs: server_addrs, + _phantom: PhantomData, }) } - async fn list_splits(&mut self) -> anyhow::Result> { - match self.source_type { - PbSourceType::Mysql => { - // CDC source only supports single split - let split = MySqlCdcSplit { - inner: CdcSplitBase::new(self.source_id, None), - }; - let dbz_split = DebeziumCdcSplit { - mysql_split: Some(split), - pg_split: None, - }; - Ok(vec![dbz_split]) - } - PbSourceType::Postgres => { + async fn list_splits(&mut self) -> anyhow::Result>> { + Ok(self.list_cdc_splits()) + } +} + +pub trait ListCdcSplits { + type CdcSourceType: CdcSourceTypeTrait; + fn list_cdc_splits(&mut self) -> Vec>; +} + +impl ListCdcSplits for DebeziumSplitEnumerator { + type CdcSourceType = Mysql; + + fn list_cdc_splits(&mut self) -> Vec> { + // CDC source only supports single split + let split = MySqlCdcSplit { + inner: CdcSplitBase::new(self.source_id, None), + }; + let dbz_split = DebeziumCdcSplit { + mysql_split: Some(split), + pg_split: None, + _phantom: PhantomData, + }; + vec![dbz_split] + } +} + +impl ListCdcSplits for DebeziumSplitEnumerator { + type CdcSourceType = Postgres; + + fn list_cdc_splits(&mut self) -> Vec> { + let split = PostgresCdcSplit { + inner: CdcSplitBase::new(self.source_id, None), + server_addr: None, + }; + let dbz_split = DebeziumCdcSplit { + mysql_split: None, + pg_split: Some(split), + _phantom: Default::default(), + }; + vec![dbz_split] + } +} + +impl ListCdcSplits for DebeziumSplitEnumerator { + type CdcSourceType = Citus; + + fn list_cdc_splits(&mut self) -> Vec> { + self.worker_node_addrs + .iter() + .enumerate() + .map(|(id, addr)| { let split = PostgresCdcSplit { - inner: CdcSplitBase::new(self.source_id, None), - server_addr: None, + inner: CdcSplitBase::new(id as u32, None), + server_addr: Some(addr.to_string()), }; - let dbz_split = DebeziumCdcSplit { + DebeziumCdcSplit { mysql_split: None, pg_split: Some(split), - }; - Ok(vec![dbz_split]) - } - PbSourceType::Citus => { - let splits = self - .worker_node_addrs - .iter() - .enumerate() - .map(|(id, addr)| { - let split = PostgresCdcSplit { - inner: CdcSplitBase::new(id as u32, None), - server_addr: Some(addr.to_string()), - }; - DebeziumCdcSplit { - mysql_split: None, - pg_split: Some(split), - } - }) - .collect_vec(); - Ok(splits) - } - _ => Err(anyhow!("unexpected source type")), - } + _phantom: Default::default(), + } + }) + .collect_vec() } } diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index 308e6ed88206c..c9c9fae188615 100644 --- a/src/connector/src/source/cdc/mod.rs +++ b/src/connector/src/source/cdc/mod.rs @@ -15,34 +15,99 @@ pub mod enumerator; pub mod source; pub mod split; - use std::collections::HashMap; +use std::marker::PhantomData; -use anyhow::anyhow; pub use enumerator::*; -use risingwave_pb::connector_service::{SourceType, TableSchema}; -use serde::Deserialize; +use paste::paste; +use risingwave_pb::connector_service::{PbSourceType, SourceType, TableSchema}; pub use source::*; pub use split::*; -pub const MYSQL_CDC_CONNECTOR: &str = "mysql-cdc"; -pub const POSTGRES_CDC_CONNECTOR: &str = "postgres-cdc"; -pub const CITUS_CDC_CONNECTOR: &str = "citus-cdc"; +use crate::source::{SourceProperties, SplitImpl}; + +pub const MYSQL_CDC_CONNECTOR: &str = Mysql::CDC_CONNECTOR_NAME; +pub const POSTGRES_CDC_CONNECTOR: &str = Postgres::CDC_CONNECTOR_NAME; +pub const CITUS_CDC_CONNECTOR: &str = Citus::CDC_CONNECTOR_NAME; + +pub trait CdcSourceTypeTrait: Send + Sync + Clone + 'static { + const CDC_CONNECTOR_NAME: &'static str; + fn source_type() -> CdcSourceType; +} + +macro_rules! impl_cdc_source_type { + ($({$source_type:ident, $name:expr }),*) => { + $( + paste!{ + #[derive(Clone, Debug, Default, PartialEq, Eq, Hash)] + pub struct $source_type; + impl CdcSourceTypeTrait for $source_type { + const CDC_CONNECTOR_NAME: &'static str = concat!($name, "-cdc"); + fn source_type() -> CdcSourceType { + CdcSourceType::$source_type + } + } + + pub type [< $source_type DebeziumSplitEnumerator >] = DebeziumSplitEnumerator<$source_type>; + } + )* + + pub enum CdcSourceType { + $( + $source_type, + )* + } + + impl From for CdcSourceType { + fn from(value: PbSourceType) -> Self { + match value { + PbSourceType::Unspecified => unreachable!(), + $( + PbSourceType::$source_type => CdcSourceType::$source_type, + )* + } + } + } -#[derive(Clone, Debug, Deserialize, Default)] -pub struct CdcProperties { - /// Type of the cdc source, e.g. mysql, postgres - pub source_type: String, + impl From for PbSourceType { + fn from(this: CdcSourceType) -> PbSourceType { + match this { + $( + CdcSourceType::$source_type => PbSourceType::$source_type, + )* + } + } + } + } +} + +impl_cdc_source_type!({ Mysql, "mysql" }, { Postgres, "postgres" }, { Citus, "citus" }); + +#[derive(Clone, Debug, Default)] +pub struct CdcProperties { /// Properties specified in the WITH clause by user pub props: HashMap, /// Schema of the source specified by users pub table_schema: Option, + + pub _phantom: PhantomData, +} + +impl SourceProperties for CdcProperties +where + DebeziumCdcSplit: TryFrom + Into, + DebeziumSplitEnumerator: ListCdcSplits, +{ + type Split = DebeziumCdcSplit; + type SplitEnumerator = DebeziumSplitEnumerator; + type SplitReader = CdcSplitReader; + + const SOURCE_NAME: &'static str = T::CDC_CONNECTOR_NAME; } -impl CdcProperties { - pub fn get_source_type_pb(&self) -> anyhow::Result { - SourceType::from_str_name(&self.source_type.to_ascii_uppercase()) - .ok_or_else(|| anyhow!("unknown source type: {}", self.source_type)) +impl CdcProperties { + pub fn get_source_type_pb(&self) -> SourceType { + SourceType::from(T::source_type()) } } diff --git a/src/connector/src/source/cdc/source/reader.rs b/src/connector/src/source/cdc/source/reader.rs index 200c91a8a5051..d6b82a9d94391 100644 --- a/src/connector/src/source/cdc/source/reader.rs +++ b/src/connector/src/source/cdc/source/reader.rs @@ -16,28 +16,25 @@ use std::str::FromStr; use anyhow::{anyhow, Result}; use async_trait::async_trait; -use futures::{pin_mut, StreamExt, TryStreamExt}; +use futures::pin_mut; use futures_async_stream::try_stream; use risingwave_common::util::addr::HostAddr; use risingwave_pb::connector_service::GetEventStreamResponse; -use crate::impl_common_split_reader_logic; use crate::parser::ParserConfig; use crate::source::base::SourceMessage; -use crate::source::cdc::CdcProperties; +use crate::source::cdc::{CdcProperties, CdcSourceType, CdcSourceTypeTrait, DebeziumCdcSplit}; use crate::source::{ - BoxSourceWithStateStream, Column, SourceContextRef, SplitId, SplitImpl, SplitMetaData, - SplitReader, + into_chunk_stream, BoxSourceWithStateStream, Column, CommonSplitReader, SourceContextRef, + SplitId, SplitMetaData, SplitReader, }; -impl_common_split_reader_logic!(CdcSplitReader, CdcProperties); - -pub struct CdcSplitReader { +pub struct CdcSplitReader { source_id: u64, start_offset: Option, // host address of worker node for a Citus cluster server_addr: Option, - conn_props: CdcProperties, + conn_props: CdcProperties, split_id: SplitId, // whether the full snapshot phase is done @@ -47,13 +44,14 @@ pub struct CdcSplitReader { } #[async_trait] -impl SplitReader for CdcSplitReader { - type Properties = CdcProperties; +impl SplitReader for CdcSplitReader { + type Properties = CdcProperties; + type Split = DebeziumCdcSplit; #[allow(clippy::unused_async)] async fn new( - conn_props: CdcProperties, - splits: Vec, + conn_props: CdcProperties, + splits: Vec>, parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, @@ -61,8 +59,8 @@ impl SplitReader for CdcSplitReader { assert_eq!(splits.len(), 1); let split = splits.into_iter().next().unwrap(); let split_id = split.id(); - match split { - SplitImpl::MySqlCdc(split) | SplitImpl::PostgresCdc(split) => Ok(Self { + match T::source_type() { + CdcSourceType::Mysql | CdcSourceType::Postgres => Ok(Self { source_id: split.split_id() as u64, start_offset: split.start_offset().clone(), server_addr: None, @@ -72,7 +70,7 @@ impl SplitReader for CdcSplitReader { parser_config, source_ctx, }), - SplitImpl::CitusCdc(split) => Ok(Self { + CdcSourceType::Citus => Ok(Self { source_id: split.split_id() as u64, start_offset: split.start_offset().clone(), server_addr: split.server_addr().clone(), @@ -82,20 +80,18 @@ impl SplitReader for CdcSplitReader { parser_config, source_ctx, }), - - _ => Err(anyhow!( - "failed to create cdc split reader: invalid splis info" - )), } } fn into_stream(self) -> BoxSourceWithStateStream { - self.into_chunk_stream() + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) } } -impl CdcSplitReader { - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] +impl CommonSplitReader for CdcSplitReader { + #[try_stream(ok = Vec, error = anyhow::Error)] async fn into_data_stream(self) { let cdc_client = self.source_ctx.connector_client.clone().ok_or_else(|| { anyhow!("connector node endpoint not specified or unable to connect to connector node") @@ -122,7 +118,7 @@ impl CdcSplitReader { let cdc_stream = cdc_client .start_source_stream( self.source_id, - self.conn_props.get_source_type_pb()?, + self.conn_props.get_source_type_pb(), self.start_offset, properties, self.snapshot_done, diff --git a/src/connector/src/source/cdc/split.rs b/src/connector/src/source/cdc/split.rs index e72fe97b5658f..6535585a5309e 100644 --- a/src/connector/src/source/cdc/split.rs +++ b/src/connector/src/source/cdc/split.rs @@ -13,11 +13,13 @@ // limitations under the License. use std::collections::HashMap; +use std::marker::PhantomData; use anyhow::anyhow; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::source::cdc::CdcSourceTypeTrait; use crate::source::{SplitId, SplitMetaData}; /// The base states of a CDC split, which will be persisted to checkpoint. @@ -151,12 +153,15 @@ impl PostgresCdcSplit { } #[derive(Clone, Serialize, Deserialize, Debug, PartialEq, Hash)] -pub struct DebeziumCdcSplit { +pub struct DebeziumCdcSplit { pub mysql_split: Option, pub pg_split: Option, + + #[serde(skip)] + pub _phantom: PhantomData, } -impl SplitMetaData for DebeziumCdcSplit { +impl SplitMetaData for DebeziumCdcSplit { fn id(&self) -> SplitId { assert!(self.mysql_split.is_some() || self.pg_split.is_some()); if let Some(split) = &self.mysql_split { @@ -177,11 +182,12 @@ impl SplitMetaData for DebeziumCdcSplit { } } -impl DebeziumCdcSplit { +impl DebeziumCdcSplit { pub fn new(mysql_split: Option, pg_split: Option) -> Self { Self { mysql_split, pg_split, + _phantom: PhantomData, } } diff --git a/src/connector/src/source/common.rs b/src/connector/src/source/common.rs new file mode 100644 index 0000000000000..86ad60cc1b969 --- /dev/null +++ b/src/connector/src/source/common.rs @@ -0,0 +1,76 @@ +// 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 futures::{Stream, StreamExt, TryStreamExt}; +use futures_async_stream::try_stream; +use risingwave_common::error::RwError; + +use crate::parser::ParserConfig; +use crate::source::{SourceContextRef, SourceMessage, SplitReader, StreamChunkWithState}; + +pub(crate) trait CommonSplitReader: SplitReader + 'static { + fn into_data_stream( + self, + ) -> impl Stream, anyhow::Error>> + Send; +} + +#[try_stream(boxed, ok = StreamChunkWithState, error = RwError)] +pub(crate) async fn into_chunk_stream( + reader: impl CommonSplitReader, + parser_config: ParserConfig, + source_ctx: SourceContextRef, +) { + let actor_id = source_ctx.source_info.actor_id.to_string(); + let source_id = source_ctx.source_info.source_id.to_string(); + let metrics = source_ctx.metrics.clone(); + + let data_stream = reader.into_data_stream(); + + let data_stream = data_stream + .inspect_ok(move |data_batch| { + let mut by_split_id = std::collections::HashMap::new(); + + for msg in data_batch { + by_split_id + .entry(msg.split_id.as_ref()) + .or_insert_with(Vec::new) + .push(msg); + } + + for (split_id, msgs) in by_split_id { + metrics + .partition_input_count + .with_label_values(&[&actor_id, &source_id, split_id]) + .inc_by(msgs.len() as u64); + + let sum_bytes = msgs + .iter() + .flat_map(|msg| msg.payload.as_ref().map(|p| p.len() as u64)) + .sum(); + + metrics + .partition_input_bytes + .with_label_values(&[&actor_id, &source_id, split_id]) + .inc_by(sum_bytes); + } + }) + .boxed(); + + let parser = + crate::parser::ByteStreamSourceParserImpl::create(parser_config, source_ctx).await?; + #[for_await] + for msg_batch in parser.into_stream(data_stream) { + yield msg_batch?; + } +} diff --git a/src/connector/src/source/datagen/mod.rs b/src/connector/src/source/datagen/mod.rs index c0d9717db5366..af2dd2c388e92 100644 --- a/src/connector/src/source/datagen/mod.rs +++ b/src/connector/src/source/datagen/mod.rs @@ -24,6 +24,8 @@ use serde_with::{serde_as, DisplayFromStr}; pub use source::*; pub use split::*; +use crate::source::SourceProperties; + pub const DATAGEN_CONNECTOR: &str = "datagen"; #[serde_as] @@ -55,6 +57,14 @@ pub struct DatagenProperties { fields: HashMap, } +impl SourceProperties for DatagenProperties { + type Split = DatagenSplit; + type SplitEnumerator = DatagenSplitEnumerator; + type SplitReader = DatagenSplitReader; + + const SOURCE_NAME: &'static str = DATAGEN_CONNECTOR; +} + fn default_rows_per_second() -> u64 { 10 } diff --git a/src/connector/src/source/datagen/source/reader.rs b/src/connector/src/source/datagen/source/reader.rs index d040a3cb63f21..3840b0e1f5dbc 100644 --- a/src/connector/src/source/datagen/source/reader.rs +++ b/src/connector/src/source/datagen/source/reader.rs @@ -16,23 +16,19 @@ use std::collections::HashMap; use anyhow::{anyhow, Result}; use async_trait::async_trait; -use futures::{StreamExt, TryStreamExt}; -use futures_async_stream::try_stream; +use futures::{Stream, StreamExt, TryStreamExt}; use risingwave_common::field_generator::{FieldGeneratorImpl, VarcharProperty}; use super::generator::DatagenEventGenerator; -use crate::impl_common_split_reader_logic; use crate::parser::{EncodingProperties, ParserConfig, ProtocolProperties}; use crate::source::data_gen_util::spawn_data_generation_stream; use crate::source::datagen::source::SEQUENCE_FIELD_KIND; use crate::source::datagen::{DatagenProperties, DatagenSplit, FieldDesc}; use crate::source::{ - BoxSourceStream, BoxSourceWithStateStream, Column, DataType, SourceContextRef, SplitId, - SplitImpl, SplitMetaData, SplitReader, + into_chunk_stream, BoxSourceWithStateStream, Column, CommonSplitReader, DataType, + SourceContextRef, SourceMessage, SplitId, SplitMetaData, SplitReader, }; -impl_common_split_reader_logic!(DatagenSplitReader, DatagenProperties); - pub struct DatagenSplitReader { generator: DatagenEventGenerator, assigned_split: DatagenSplit, @@ -45,16 +41,16 @@ pub struct DatagenSplitReader { #[async_trait] impl SplitReader for DatagenSplitReader { type Properties = DatagenProperties; + type Split = DatagenSplit; #[allow(clippy::unused_async)] async fn new( properties: DatagenProperties, - splits: Vec, + splits: Vec, parser_config: ParserConfig, source_ctx: SourceContextRef, columns: Option>, ) -> Result { - let mut assigned_split = DatagenSplit::default(); let mut events_so_far = u64::default(); tracing::debug!("Splits for datagen found! {:?}", splits); @@ -62,14 +58,12 @@ impl SplitReader for DatagenSplitReader { let split = splits.into_iter().next().unwrap(); // TODO: currently, assume there's only on split in one reader let split_id = split.id(); - if let SplitImpl::Datagen(n) = split { - if let Some(s) = n.start_offset { - // start_offset in `SplitImpl` indicates the latest successfully generated - // index, so here we use start_offset+1 - events_so_far = s + 1; - }; - assigned_split = n; - } + let assigned_split = split; + if let Some(s) = assigned_split.start_offset { + // start_offset in `SplitImpl` indicates the latest successfully generated + // index, so here we use start_offset+1 + events_so_far = s + 1; + }; let split_index = assigned_split.split_index as u64; let split_num = assigned_split.split_num as u64; @@ -170,16 +164,20 @@ impl SplitReader for DatagenSplitReader { ) .boxed() } - _ => self.into_chunk_stream(), + _ => { + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) + } } } } -impl DatagenSplitReader { - pub(crate) fn into_data_stream(self) -> BoxSourceStream { +impl CommonSplitReader for DatagenSplitReader { + fn into_data_stream(self) -> impl Stream, anyhow::Error>> { // Will buffer at most 4 event chunks. const BUFFER_SIZE: usize = 4; - spawn_data_generation_stream(self.generator.into_msg_stream(), BUFFER_SIZE).boxed() + spawn_data_generation_stream(self.generator.into_msg_stream(), BUFFER_SIZE) } } @@ -345,11 +343,11 @@ mod tests { is_visible: true, }, ]; - let state = vec![SplitImpl::Datagen(DatagenSplit { + let state = vec![DatagenSplit { split_index: 0, split_num: 1, start_offset: None, - })]; + }]; let properties = DatagenProperties { split_num: None, rows_per_second: 10, @@ -423,11 +421,11 @@ mod tests { is_visible: true, }, ]; - let state = vec![SplitImpl::Datagen(DatagenSplit { + let state = vec![DatagenSplit { split_index: 0, split_num: 1, start_offset: None, - })]; + }]; let properties = DatagenProperties { split_num: None, rows_per_second: 10, @@ -453,11 +451,11 @@ mod tests { let v1 = stream.skip(1).next().await.unwrap()?; - let state = vec![SplitImpl::Datagen(DatagenSplit { + let state = vec![DatagenSplit { split_index: 0, split_num: 1, start_offset: Some(9), - })]; + }]; let mut stream = DatagenSplitReader::new( properties, state, diff --git a/src/connector/src/source/dummy_connector.rs b/src/connector/src/source/dummy_connector.rs deleted file mode 100644 index 3a5b8922fd29a..0000000000000 --- a/src/connector/src/source/dummy_connector.rs +++ /dev/null @@ -1,45 +0,0 @@ -// 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 anyhow::Result; -use async_trait::async_trait; -use futures::StreamExt; - -use super::{SourceContextRef, SplitImpl, SplitReader}; -use crate::parser::ParserConfig; -use crate::source::{BoxSourceWithStateStream, Column}; - -/// [`DummySplitReader`] is a placeholder for source executor that is assigned no split. It will -/// wait forever when calling `next`. -#[derive(Clone, Debug)] -pub struct DummySplitReader; - -#[async_trait] -impl SplitReader for DummySplitReader { - type Properties = (); - - async fn new( - _properties: Self::Properties, - _state: Vec, - _parser_config: ParserConfig, - _source_ctx: SourceContextRef, - _columns: Option>, - ) -> Result { - Ok(Self {}) - } - - fn into_stream(self) -> BoxSourceWithStateStream { - futures::stream::pending().boxed() - } -} diff --git a/src/connector/src/source/filesystem/s3/mod.rs b/src/connector/src/source/filesystem/s3/mod.rs index 62f6bcd922a80..12701087309e0 100644 --- a/src/connector/src/source/filesystem/s3/mod.rs +++ b/src/connector/src/source/filesystem/s3/mod.rs @@ -19,6 +19,8 @@ use serde::Deserialize; pub use source::S3FileReader; use crate::aws_auth::AwsAuthProps; +use crate::source::filesystem::FsSplit; +use crate::source::SourceProperties; pub const S3_CONNECTOR: &str = "s3"; @@ -38,6 +40,14 @@ pub struct S3Properties { endpoint_url: Option, } +impl SourceProperties for S3Properties { + type Split = FsSplit; + type SplitEnumerator = S3SplitEnumerator; + type SplitReader = S3FileReader; + + const SOURCE_NAME: &'static str = S3_CONNECTOR; +} + impl From<&S3Properties> for AwsAuthProps { fn from(props: &S3Properties) -> Self { Self { diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index b20822139d181..736e4493d3f55 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -37,7 +37,7 @@ use crate::source::filesystem::file_common::FsSplit; use crate::source::filesystem::nd_streaming; use crate::source::filesystem::s3::S3Properties; use crate::source::{ - BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SourceMeta, SplitImpl, + BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SourceMeta, StreamChunkWithState, }; const MAX_CHANNEL_BUFFER_SIZE: usize = 2048; @@ -164,10 +164,11 @@ impl S3FileReader { #[async_trait] impl SplitReader for S3FileReader { type Properties = S3Properties; + type Split = FsSplit; async fn new( props: S3Properties, - state: Vec, + splits: Vec, parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, @@ -179,10 +180,6 @@ impl SplitReader for S3FileReader { let bucket_name = props.bucket_name; let s3_client = s3_client(&sdk_config, Some(default_conn_config())); - let splits = state - .into_iter() - .map(|split| split.into_fs().expect("not a fs split")) - .collect(); let s3_file_reader = S3FileReader { split_offset: HashMap::new(), bucket_name, @@ -272,8 +269,6 @@ mod tests { let splits = enumerator.list_splits().await.unwrap(); println!("splits {:?}", splits); - let splits = splits.into_iter().map(SplitImpl::S3).collect(); - let descs = vec![ SourceColumnDesc::simple("id", DataType::Int64, 1.into()), SourceColumnDesc::simple("name", DataType::Varchar, 2.into()), diff --git a/src/connector/src/source/google_pubsub/mod.rs b/src/connector/src/source/google_pubsub/mod.rs index 0619edde5a8bd..1bec07a1aa4ae 100644 --- a/src/connector/src/source/google_pubsub/mod.rs +++ b/src/connector/src/source/google_pubsub/mod.rs @@ -23,6 +23,8 @@ use serde_with::{serde_as, DisplayFromStr}; pub use source::*; pub use split::*; +use crate::source::SourceProperties; + pub const GOOGLE_PUBSUB_CONNECTOR: &str = "google_pubsub"; #[serde_as] @@ -70,6 +72,14 @@ pub struct PubsubProperties { pub start_snapshot: Option, } +impl SourceProperties for PubsubProperties { + type Split = PubsubSplit; + type SplitEnumerator = PubsubSplitEnumerator; + type SplitReader = PubsubSplitReader; + + const SOURCE_NAME: &'static str = GOOGLE_PUBSUB_CONNECTOR; +} + impl PubsubProperties { /// `initialize_env` sets environment variables read by the `google-cloud-pubsub` crate pub(crate) fn initialize_env(&self) { diff --git a/src/connector/src/source/google_pubsub/source/reader.rs b/src/connector/src/source/google_pubsub/source/reader.rs index f5d4955ec9b20..18aeeecc050e4 100644 --- a/src/connector/src/source/google_pubsub/source/reader.rs +++ b/src/connector/src/source/google_pubsub/source/reader.rs @@ -15,7 +15,6 @@ use anyhow::{anyhow, ensure, Context, Result}; use async_trait::async_trait; use chrono::{NaiveDateTime, TimeZone, Utc}; -use futures::{StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use google_cloud_pubsub::client::Client; use google_cloud_pubsub::subscription::{SeekTo, Subscription}; @@ -23,18 +22,15 @@ use risingwave_common::bail; use tonic::Code; use super::TaggedReceivedMessage; -use crate::impl_common_split_reader_logic; use crate::parser::ParserConfig; -use crate::source::google_pubsub::PubsubProperties; +use crate::source::google_pubsub::{PubsubProperties, PubsubSplit}; use crate::source::{ - BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SplitId, SplitImpl, - SplitMetaData, SplitReader, + into_chunk_stream, BoxSourceWithStateStream, Column, CommonSplitReader, SourceContextRef, + SourceMessage, SplitId, SplitMetaData, SplitReader, }; const PUBSUB_MAX_FETCH_MESSAGES: usize = 1024; -impl_common_split_reader_logic!(PubsubSplitReader, PubsubProperties); - pub struct PubsubSplitReader { subscription: Subscription, stop_offset: Option, @@ -44,8 +40,8 @@ pub struct PubsubSplitReader { source_ctx: SourceContextRef, } -impl PubsubSplitReader { - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] +impl CommonSplitReader for PubsubSplitReader { + #[try_stream(ok = Vec, error = anyhow::Error)] async fn into_data_stream(self) { loop { let pull_result = self @@ -110,10 +106,11 @@ impl PubsubSplitReader { #[async_trait] impl SplitReader for PubsubSplitReader { type Properties = PubsubProperties; + type Split = PubsubSplit; async fn new( properties: PubsubProperties, - splits: Vec, + splits: Vec, parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, @@ -122,12 +119,7 @@ impl SplitReader for PubsubSplitReader { splits.len() == 1, "the pubsub reader only supports a single split" ); - let split = splits - .into_iter() - .next() - .unwrap() - .into_google_pubsub() - .unwrap(); + let split = splits.into_iter().next().unwrap(); // Set environment variables consumed by `google_cloud_pubsub` properties.initialize_env(); @@ -172,6 +164,8 @@ impl SplitReader for PubsubSplitReader { } fn into_stream(self) -> BoxSourceWithStateStream { - self.into_chunk_stream() + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) } } diff --git a/src/connector/src/source/kafka/mod.rs b/src/connector/src/source/kafka/mod.rs index 7b8e3a4d1e4eb..5a2d570e77eaf 100644 --- a/src/connector/src/source/kafka/mod.rs +++ b/src/connector/src/source/kafka/mod.rs @@ -27,6 +27,8 @@ pub use source::*; pub use split::*; use crate::common::KafkaCommon; +use crate::source::SourceProperties; + pub const KAFKA_CONNECTOR: &str = "kafka"; pub const KAFKA_PROPS_BROKER_KEY: &str = "properties.bootstrap.server"; pub const KAFKA_PROPS_BROKER_KEY_ALIAS: &str = "kafka.brokers"; @@ -115,6 +117,14 @@ pub struct KafkaProperties { pub rdkafka_properties: RdKafkaPropertiesConsumer, } +impl SourceProperties for KafkaProperties { + type Split = KafkaSplit; + type SplitEnumerator = KafkaSplitEnumerator; + type SplitReader = KafkaSplitReader; + + const SOURCE_NAME: &'static str = KAFKA_CONNECTOR; +} + impl KafkaProperties { pub fn set_client(&self, c: &mut rdkafka::ClientConfig) { self.common.set_client(c); diff --git a/src/connector/src/source/kafka/source/reader.rs b/src/connector/src/source/kafka/source/reader.rs index d443429857c21..3164fabe2bded 100644 --- a/src/connector/src/source/kafka/source/reader.rs +++ b/src/connector/src/source/kafka/source/reader.rs @@ -19,26 +19,23 @@ use std::time::{Duration, SystemTime, UNIX_EPOCH}; use anyhow::{anyhow, Result}; use async_trait::async_trait; -use futures::{StreamExt, TryStreamExt}; +use futures::StreamExt; use futures_async_stream::try_stream; use rdkafka::config::RDKafkaLogLevel; use rdkafka::consumer::{Consumer, StreamConsumer}; use rdkafka::error::KafkaError; use rdkafka::{ClientConfig, Message, Offset, TopicPartitionList}; -use crate::impl_common_split_reader_logic; use crate::parser::ParserConfig; use crate::source::base::SourceMessage; use crate::source::kafka::{ KafkaProperties, KafkaSplit, PrivateLinkConsumerContext, KAFKA_ISOLATION_LEVEL, }; use crate::source::{ - BoxSourceWithStateStream, Column, SourceContextRef, SplitId, SplitImpl, SplitMetaData, - SplitReader, + into_chunk_stream, BoxSourceWithStateStream, Column, CommonSplitReader, SourceContextRef, + SplitId, SplitMetaData, SplitReader, }; -impl_common_split_reader_logic!(KafkaSplitReader, KafkaProperties); - pub struct KafkaSplitReader { consumer: StreamConsumer, offsets: HashMap, Option)>, @@ -51,10 +48,11 @@ pub struct KafkaSplitReader { #[async_trait] impl SplitReader for KafkaSplitReader { type Properties = KafkaProperties; + type Split = KafkaSplit; async fn new( properties: KafkaProperties, - splits: Vec, + splits: Vec, parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, @@ -108,11 +106,6 @@ impl SplitReader for KafkaSplitReader { .await .map_err(|e| anyhow!("failed to create kafka consumer: {}", e))?; - let splits = splits - .into_iter() - .map(|split| split.into_kafka().unwrap()) - .collect::>(); - let mut tpl = TopicPartitionList::with_capacity(splits.len()); let mut offsets = HashMap::new(); @@ -159,7 +152,9 @@ impl SplitReader for KafkaSplitReader { } fn into_stream(self) -> BoxSourceWithStateStream { - self.into_chunk_stream() + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) } } @@ -176,9 +171,11 @@ impl KafkaSplitReader { ]) .set(offset); } +} - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] - pub async fn into_data_stream(self) { +impl CommonSplitReader for KafkaSplitReader { + #[try_stream(ok = Vec, error = anyhow::Error)] + async fn into_data_stream(self) { if self.offsets.values().all(|(start_offset, stop_offset)| { match (start_offset, stop_offset) { (Some(start), Some(stop)) if (*start + 1) >= *stop => true, diff --git a/src/connector/src/source/kinesis/mod.rs b/src/connector/src/source/kinesis/mod.rs index fc786f8f1b10d..6ad250fc93018 100644 --- a/src/connector/src/source/kinesis/mod.rs +++ b/src/connector/src/source/kinesis/mod.rs @@ -19,6 +19,10 @@ pub mod split; use serde::Deserialize; use crate::common::KinesisCommon; +use crate::source::kinesis::enumerator::client::KinesisSplitEnumerator; +use crate::source::kinesis::source::reader::KinesisSplitReader; +use crate::source::kinesis::split::KinesisSplit; +use crate::source::SourceProperties; pub const KINESIS_CONNECTOR: &str = "kinesis"; @@ -36,3 +40,11 @@ pub struct KinesisProperties { #[serde(flatten)] pub common: KinesisCommon, } + +impl SourceProperties for KinesisProperties { + type Split = KinesisSplit; + type SplitEnumerator = KinesisSplitEnumerator; + type SplitReader = KinesisSplitReader; + + const SOURCE_NAME: &'static str = KINESIS_CONNECTOR; +} diff --git a/src/connector/src/source/kinesis/source/reader.rs b/src/connector/src/source/kinesis/source/reader.rs index 8d714f3b79334..9ef6a179db9a5 100644 --- a/src/connector/src/source/kinesis/source/reader.rs +++ b/src/connector/src/source/kinesis/source/reader.rs @@ -20,22 +20,18 @@ use aws_sdk_kinesis::error::{DisplayErrorContext, SdkError}; use aws_sdk_kinesis::operation::get_records::{GetRecordsError, GetRecordsOutput}; use aws_sdk_kinesis::types::ShardIteratorType; use aws_sdk_kinesis::Client as KinesisClient; -use futures::{StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use tokio_retry; -use crate::impl_common_split_reader_logic; use crate::parser::ParserConfig; use crate::source::kinesis::source::message::KinesisMessage; -use crate::source::kinesis::split::KinesisOffset; +use crate::source::kinesis::split::{KinesisOffset, KinesisSplit}; use crate::source::kinesis::KinesisProperties; use crate::source::{ - BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SplitId, SplitImpl, - SplitMetaData, SplitReader, + into_chunk_stream, BoxSourceWithStateStream, Column, CommonSplitReader, SourceContextRef, + SourceMessage, SplitId, SplitMetaData, SplitReader, }; -impl_common_split_reader_logic!(KinesisSplitReader, KinesisProperties); - #[derive(Debug, Clone)] pub struct KinesisSplitReader { client: KinesisClient, @@ -54,17 +50,18 @@ pub struct KinesisSplitReader { #[async_trait] impl SplitReader for KinesisSplitReader { type Properties = KinesisProperties; + type Split = KinesisSplit; async fn new( properties: KinesisProperties, - splits: Vec, + splits: Vec, parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, ) -> Result { assert!(splits.len() == 1); - let split = splits.into_iter().next().unwrap().into_kinesis().unwrap(); + let split = splits.into_iter().next().unwrap(); let start_position = match &split.start_position { KinesisOffset::None => match &properties.scan_startup_mode { @@ -108,13 +105,15 @@ impl SplitReader for KinesisSplitReader { } fn into_stream(self) -> BoxSourceWithStateStream { - self.into_chunk_stream() + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) } } -impl KinesisSplitReader { - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] - pub(crate) async fn into_data_stream(mut self) { +impl CommonSplitReader for KinesisSplitReader { + #[try_stream(ok = Vec < SourceMessage >, error = anyhow::Error)] + async fn into_data_stream(mut self) { self.new_shard_iter().await?; loop { if self.shard_iter.is_none() { @@ -189,7 +188,8 @@ impl KinesisSplitReader { } } } - +} +impl KinesisSplitReader { async fn new_shard_iter(&mut self) -> Result<()> { let (starting_seq_num, iter_type) = if self.latest_offset.is_some() { ( @@ -269,7 +269,7 @@ impl KinesisSplitReader { #[cfg(test)] mod tests { - use futures::StreamExt; + use futures::{pin_mut, StreamExt}; use super::*; use crate::common::KinesisCommon; @@ -294,36 +294,38 @@ mod tests { seq_offset: None, }; - let mut trim_horizen_reader = KinesisSplitReader::new( + let trim_horizen_reader = KinesisSplitReader::new( properties.clone(), - vec![SplitImpl::Kinesis(KinesisSplit { + vec![KinesisSplit { shard_id: "shardId-000000000001".to_string().into(), start_position: KinesisOffset::Earliest, end_position: KinesisOffset::None, - })], + }], Default::default(), Default::default(), None, ) .await? .into_data_stream(); + pin_mut!(trim_horizen_reader); println!("{:?}", trim_horizen_reader.next().await.unwrap()?); - let mut offset_reader = KinesisSplitReader::new( + let offset_reader = KinesisSplitReader::new( properties.clone(), - vec![SplitImpl::Kinesis(KinesisSplit { + vec![KinesisSplit { shard_id: "shardId-000000000001".to_string().into(), start_position: KinesisOffset::SequenceNumber( "49629139817504901062972448413535783695568426186596941842".to_string(), ), end_position: KinesisOffset::None, - })], + }], Default::default(), Default::default(), None, ) .await? .into_data_stream(); + pin_mut!(offset_reader); println!("{:?}", offset_reader.next().await.unwrap()?); Ok(()) diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index ac10ded25c688..77c9803eb8e2c 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -16,7 +16,6 @@ pub mod base; pub mod cdc; pub mod data_gen_util; pub mod datagen; -pub mod dummy_connector; pub mod filesystem; pub mod google_pubsub; pub mod kafka; @@ -25,10 +24,13 @@ pub mod monitor; pub mod nexmark; pub mod pulsar; pub use base::*; +pub(crate) use common::*; pub use google_pubsub::GOOGLE_PUBSUB_CONNECTOR; pub use kafka::KAFKA_CONNECTOR; pub use kinesis::KINESIS_CONNECTOR; +mod common; mod manager; + pub use manager::SourceColumnDesc; pub use crate::source::nexmark::NEXMARK_CONNECTOR; diff --git a/src/connector/src/source/nexmark/mod.rs b/src/connector/src/source/nexmark/mod.rs index 679306cf96b22..e1f75ae1008e7 100644 --- a/src/connector/src/source/nexmark/mod.rs +++ b/src/connector/src/source/nexmark/mod.rs @@ -25,6 +25,9 @@ use serde::Deserialize; use serde_with::{serde_as, DisplayFromStr}; pub use split::*; +use crate::source::nexmark::source::reader::NexmarkSplitReader; +use crate::source::SourceProperties; + pub const NEXMARK_CONNECTOR: &str = "nexmark"; const fn identity_i32() -> i32 { @@ -217,6 +220,14 @@ pub struct NexmarkPropertiesInner { pub threads: Option, } +impl SourceProperties for NexmarkProperties { + type Split = NexmarkSplit; + type SplitEnumerator = NexmarkSplitEnumerator; + type SplitReader = NexmarkSplitReader; + + const SOURCE_NAME: &'static str = NEXMARK_CONNECTOR; +} + fn default_event_num() -> u64 { u64::MAX } diff --git a/src/connector/src/source/nexmark/source/reader.rs b/src/connector/src/source/nexmark/source/reader.rs index e7a9c03b5f9b6..6cc0dd3f49cb4 100644 --- a/src/connector/src/source/nexmark/source/reader.rs +++ b/src/connector/src/source/nexmark/source/reader.rs @@ -33,8 +33,8 @@ use crate::source::nexmark::source::combined_event::{ }; use crate::source::nexmark::{NexmarkProperties, NexmarkSplit}; use crate::source::{ - BoxSourceWithStateStream, Column, SourceContextRef, SplitId, SplitImpl, SplitMetaData, - SplitReader, StreamChunkWithState, + BoxSourceWithStateStream, Column, SourceContextRef, SplitId, SplitMetaData, SplitReader, + StreamChunkWithState, }; #[derive(Debug)] @@ -55,11 +55,12 @@ pub struct NexmarkSplitReader { #[async_trait] impl SplitReader for NexmarkSplitReader { type Properties = NexmarkProperties; + type Split = NexmarkSplit; #[allow(clippy::unused_async)] async fn new( properties: NexmarkProperties, - splits: Vec, + splits: Vec, parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, @@ -67,7 +68,7 @@ impl SplitReader for NexmarkSplitReader { tracing::debug!("Splits for nexmark found! {:?}", splits); assert!(splits.len() == 1); // TODO: currently, assume there's only one split in one reader - let split = splits.into_iter().next().unwrap().into_nexmark().unwrap(); + let split = splits.into_iter().next().unwrap(); let split_id = split.id(); let split_index = split.split_index as u64; @@ -182,7 +183,7 @@ mod tests { use super::*; use crate::source::nexmark::{NexmarkPropertiesInner, NexmarkSplitEnumerator}; - use crate::source::{SourceEnumeratorContext, SplitEnumerator, SplitImpl}; + use crate::source::{SourceEnumeratorContext, SplitEnumerator}; #[tokio::test] async fn test_nexmark_split_reader() -> Result<()> { @@ -197,12 +198,7 @@ mod tests { let mut enumerator = NexmarkSplitEnumerator::new(props.clone(), SourceEnumeratorContext::default().into()) .await?; - let list_splits_resp: Vec = enumerator - .list_splits() - .await? - .into_iter() - .map(SplitImpl::Nexmark) - .collect(); + let list_splits_resp: Vec<_> = enumerator.list_splits().await?.into_iter().collect(); assert_eq!(list_splits_resp.len(), 2); diff --git a/src/connector/src/source/pulsar/mod.rs b/src/connector/src/source/pulsar/mod.rs index 4c45e0110fa3d..4e6930abf7bd9 100644 --- a/src/connector/src/source/pulsar/mod.rs +++ b/src/connector/src/source/pulsar/mod.rs @@ -33,6 +33,8 @@ use url::Url; use crate::aws_auth::AwsAuthProps; use crate::aws_utils::load_file_descriptor_from_s3; +use crate::source::pulsar::source::reader::PulsarSplitReader; +use crate::source::SourceProperties; pub const PULSAR_CONNECTOR: &str = "pulsar"; @@ -76,6 +78,14 @@ pub struct PulsarProperties { pub oauth: Option, } +impl SourceProperties for PulsarProperties { + type Split = PulsarSplit; + type SplitEnumerator = PulsarSplitEnumerator; + type SplitReader = PulsarSplitReader; + + const SOURCE_NAME: &'static str = PULSAR_CONNECTOR; +} + impl PulsarProperties { pub async fn build_pulsar_client(&self) -> Result> { let mut pulsar_builder = Pulsar::builder(&self.service_url, TokioExecutor); diff --git a/src/connector/src/source/pulsar/source/reader.rs b/src/connector/src/source/pulsar/source/reader.rs index a8e801541f273..db6ccfedd726b 100644 --- a/src/connector/src/source/pulsar/source/reader.rs +++ b/src/connector/src/source/pulsar/source/reader.rs @@ -16,25 +16,21 @@ use std::time::{SystemTime, UNIX_EPOCH}; use anyhow::{anyhow, ensure, Result}; use async_trait::async_trait; -use futures::{StreamExt, TryStreamExt}; +use futures::StreamExt; use futures_async_stream::try_stream; use itertools::Itertools; use pulsar::consumer::InitialPosition; use pulsar::message::proto::MessageIdData; use pulsar::{Consumer, ConsumerBuilder, ConsumerOptions, Pulsar, SubType, TokioExecutor}; -use risingwave_common::try_match_expand; -use crate::impl_common_split_reader_logic; use crate::parser::ParserConfig; use crate::source::pulsar::split::PulsarSplit; use crate::source::pulsar::{PulsarEnumeratorOffset, PulsarProperties}; use crate::source::{ - BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SplitId, SplitImpl, - SplitMetaData, SplitReader, + into_chunk_stream, BoxSourceWithStateStream, Column, CommonSplitReader, SourceContextRef, + SourceMessage, SplitId, SplitMetaData, SplitReader, }; -impl_common_split_reader_logic!(PulsarSplitReader, PulsarProperties); - pub struct PulsarSplitReader { pulsar: Pulsar, consumer: Consumer, TokioExecutor>, @@ -90,16 +86,17 @@ fn parse_message_id(id: &str) -> Result { #[async_trait] impl SplitReader for PulsarSplitReader { type Properties = PulsarProperties; + type Split = PulsarSplit; async fn new( props: PulsarProperties, - splits: Vec, + splits: Vec, parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, ) -> Result { ensure!(splits.len() == 1, "only support single split"); - let split = try_match_expand!(splits.into_iter().next().unwrap(), SplitImpl::Pulsar)?; + let split = splits.into_iter().next().unwrap(); let pulsar = props.build_pulsar_client().await?; let topic = split.topic.to_string(); @@ -170,13 +167,15 @@ impl SplitReader for PulsarSplitReader { } fn into_stream(self) -> BoxSourceWithStateStream { - self.into_chunk_stream() + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) } } -impl PulsarSplitReader { - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] - pub(crate) async fn into_data_stream(self) { +impl CommonSplitReader for PulsarSplitReader { + #[try_stream(ok = Vec, error = anyhow::Error)] + async fn into_data_stream(self) { let max_chunk_size = self.source_ctx.source_ctrl_opts.chunk_size; #[for_await] for msgs in self.consumer.ready_chunks(max_chunk_size) { diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index ee40882c4cbf1..7fa512fcbb05a 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -27,8 +27,9 @@ use risingwave_common::catalog::TableDesc; use risingwave_common::error::RwError; use risingwave_common::hash::{ParallelUnitId, ParallelUnitMapping, VirtualNode}; use risingwave_common::util::scan_range::ScanRange; +use risingwave_connector::source::kafka::KafkaSplitEnumerator; use risingwave_connector::source::{ - ConnectorProperties, SourceEnumeratorContext, SplitEnumeratorImpl, SplitImpl, + ConnectorProperties, SourceEnumeratorContext, SplitEnumerator, SplitImpl, }; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{ExchangeInfo, ScanRange as ScanRangeProto}; @@ -266,19 +267,17 @@ impl SourceScanInfo { unreachable!("Never call complete when SourceScanInfo is already complete") } }; - let mut enumerator = SplitEnumeratorImpl::create( - fetch_info.connector, - SourceEnumeratorContext::default().into(), - ) - .await?; - let kafka_enumerator = match enumerator { - SplitEnumeratorImpl::Kafka(ref mut kafka_enumerator) => kafka_enumerator, + let kafka_prop = match fetch_info.connector { + ConnectorProperties::Kafka(prop) => *prop, _ => { return Err(SchedulerError::Internal(anyhow!( "Unsupported to query directly from this source" ))) } }; + let mut kafka_enumerator = + KafkaSplitEnumerator::new(kafka_prop, SourceEnumeratorContext::default().into()) + .await?; let split_info = kafka_enumerator .list_splits_batch(fetch_info.timebound.0, fetch_info.timebound.1) .await? diff --git a/src/meta/src/rpc/service/cloud_service.rs b/src/meta/src/rpc/service/cloud_service.rs index 435bfd830affb..104c523918d70 100644 --- a/src/meta/src/rpc/service/cloud_service.rs +++ b/src/meta/src/rpc/service/cloud_service.rs @@ -17,9 +17,10 @@ use std::sync::LazyLock; use async_trait::async_trait; use regex::Regex; +use risingwave_connector::dispatch_source_prop; use risingwave_connector::source::kafka::private_link::insert_privatelink_broker_rewrite_map; use risingwave_connector::source::{ - ConnectorProperties, SourceEnumeratorContext, SplitEnumeratorImpl, + ConnectorProperties, SourceEnumeratorContext, SourceProperties, SplitEnumerator, }; use risingwave_pb::catalog::connection::Info::PrivateLinkService; use risingwave_pb::cloud_service::cloud_service_server::CloudService; @@ -146,36 +147,43 @@ where e.to_string(), )); }; - let enumerator = - SplitEnumeratorImpl::create(props.unwrap(), SourceEnumeratorContext::default().into()) - .await; - if let Err(e) = enumerator { - return Ok(new_rwc_validate_fail_response( - ErrorType::KafkaInvalidProperties, - e.to_string(), - )); + + async fn new_enumerator( + props: P, + ) -> Result { + P::SplitEnumerator::new(props, SourceEnumeratorContext::default().into()).await } - if let Err(e) = enumerator.unwrap().list_splits().await { - let error_message = e.to_string(); - if error_message.contains("BrokerTransportFailure") { + + dispatch_source_prop!(props.unwrap(), props, { + let enumerator = new_enumerator(*props).await; + if let Err(e) = enumerator { return Ok(new_rwc_validate_fail_response( - ErrorType::KafkaBrokerUnreachable, + ErrorType::KafkaInvalidProperties, e.to_string(), )); } - static TOPIC_NOT_FOUND: LazyLock = - LazyLock::new(|| Regex::new(r"topic .* not found").unwrap()); - if TOPIC_NOT_FOUND.is_match(error_message.as_str()) { + if let Err(e) = enumerator.unwrap().list_splits().await { + let error_message = e.to_string(); + if error_message.contains("BrokerTransportFailure") { + return Ok(new_rwc_validate_fail_response( + ErrorType::KafkaBrokerUnreachable, + e.to_string(), + )); + } + static TOPIC_NOT_FOUND: LazyLock = + LazyLock::new(|| Regex::new(r"topic .* not found").unwrap()); + if TOPIC_NOT_FOUND.is_match(error_message.as_str()) { + return Ok(new_rwc_validate_fail_response( + ErrorType::KafkaTopicNotFound, + e.to_string(), + )); + } return Ok(new_rwc_validate_fail_response( - ErrorType::KafkaTopicNotFound, + ErrorType::KafkaOther, e.to_string(), )); } - return Ok(new_rwc_validate_fail_response( - ErrorType::KafkaOther, - e.to_string(), - )); - } + }); Ok(Response::new(RwCloudValidateSourceResponse { ok: true, error: None, diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index 1892de01b2aba..59de990bec6fd 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -16,15 +16,17 @@ use std::borrow::BorrowMut; use std::cmp::Ordering; use std::collections::hash_map::Entry; use std::collections::{BTreeMap, BTreeSet, BinaryHeap, HashMap, HashSet}; +use std::ops::Deref; use std::sync::Arc; use std::time::Duration; use anyhow::anyhow; use itertools::Itertools; use risingwave_common::catalog::TableId; +use risingwave_connector::dispatch_source_prop; use risingwave_connector::source::{ - ConnectorProperties, SourceEnumeratorContext, SourceEnumeratorInfo, SplitEnumeratorImpl, - SplitId, SplitImpl, SplitMetaData, + ConnectorProperties, SourceEnumeratorContext, SourceEnumeratorInfo, SourceProperties, + SplitEnumerator, SplitId, SplitImpl, SplitMetaData, }; use risingwave_pb::catalog::Source; use risingwave_pb::connector_service::PbTableSchema; @@ -62,23 +64,52 @@ struct SharedSplitMap { type SharedSplitMapRef = Arc>; -struct ConnectorSourceWorker { +struct ConnectorSourceWorker { source_id: SourceId, source_name: String, current_splits: SharedSplitMapRef, - enumerator: SplitEnumeratorImpl, + enumerator: P::SplitEnumerator, period: Duration, metrics: Arc, - connector_properties: ConnectorProperties, + connector_properties: P, connector_client: Option, fail_cnt: u32, } -impl ConnectorSourceWorker { - const DEFAULT_SOURCE_WORKER_TICK_INTERVAL: Duration = Duration::from_secs(30); +fn extract_prop_from_source(source: &Source) -> MetaResult { + let mut properties = ConnectorProperties::extract(source.properties.clone())?; + if properties.is_cdc_connector() { + let pk_indices = source + .pk_column_ids + .iter() + .map(|&id| { + source + .columns + .iter() + .position(|col| col.column_desc.as_ref().unwrap().column_id == id) + .unwrap() as u32 + }) + .collect_vec(); + let table_schema = PbTableSchema { + columns: source + .columns + .iter() + .flat_map(|col| &col.column_desc) + .cloned() + .collect(), + pk_indices, + }; + properties.init_cdc_properties(Some(table_schema)); + } + Ok(properties) +} + +const DEFAULT_SOURCE_WORKER_TICK_INTERVAL: Duration = Duration::from_secs(30); + +impl ConnectorSourceWorker

{ async fn refresh(&mut self) -> MetaResult<()> { - let enumerator = SplitEnumeratorImpl::create( + let enumerator = P::SplitEnumerator::new( self.connector_properties.clone(), Arc::new(SourceEnumeratorContext { metrics: self.metrics.source_enumerator_metrics.clone(), @@ -98,17 +129,13 @@ impl ConnectorSourceWorker { pub async fn create( connector_client: &Option, source: &Source, + connector_properties: P, period: Duration, splits: Arc>, metrics: Arc, ) -> MetaResult { - let mut properties = ConnectorProperties::extract(source.properties.clone())?; - if properties.is_cdc_connector() { - let table_schema = Self::extract_source_schema(source); - properties.init_cdc_properties(Some(table_schema)); - } - let enumerator = SplitEnumeratorImpl::create( - properties.clone(), + let enumerator = P::SplitEnumerator::new( + connector_properties.clone(), Arc::new(SourceEnumeratorContext { metrics: metrics.source_enumerator_metrics.clone(), info: SourceEnumeratorInfo { @@ -126,7 +153,7 @@ impl ConnectorSourceWorker { enumerator, period, metrics, - connector_properties: properties, + connector_properties, connector_client: connector_client.clone(), fail_cnt: 0, }) @@ -167,11 +194,18 @@ impl ConnectorSourceWorker { .with_label_values(&[self.source_id.to_string().as_str(), &self.source_name]) .set(res); }; - let splits = self.enumerator.list_splits().await.map_err(|e| { - source_is_up(0); - self.fail_cnt += 1; - e - })?; + let splits = self + .enumerator + .list_splits() + .await + .map_err(|e| { + source_is_up(0); + self.fail_cnt += 1; + e + })? + .into_iter() + .map(P::Split::into) + .collect_vec(); source_is_up(1); self.fail_cnt = 0; let mut current_splits = self.current_splits.lock().await; @@ -184,30 +218,6 @@ impl ConnectorSourceWorker { Ok(()) } - - fn extract_source_schema(source: &Source) -> PbTableSchema { - let pk_indices = source - .pk_column_ids - .iter() - .map(|&id| { - source - .columns - .iter() - .position(|col| col.column_desc.as_ref().unwrap().column_id == id) - .unwrap() as u32 - }) - .collect_vec(); - - PbTableSchema { - columns: source - .columns - .iter() - .flat_map(|col| &col.column_desc) - .cloned() - .collect(), - pk_indices, - } - } } struct ConnectorSourceWorkerHandle { @@ -534,7 +544,7 @@ where &mut managed_sources, metrics.clone(), ) - .await + .await? } } @@ -720,7 +730,7 @@ where source: Source, managed_sources: &mut HashMap, metrics: Arc, - ) { + ) -> MetaResult<()> { tracing::info!("spawning new watcher for source {}", source.id); let (sync_call_tx, sync_call_rx) = tokio::sync::mpsc::unbounded_channel(); @@ -729,32 +739,37 @@ where let current_splits_ref = splits.clone(); let source_id = source.id; + let connector_properties = extract_prop_from_source(&source)?; + let handle = tokio::spawn(async move { let mut ticker = time::interval(Self::DEFAULT_SOURCE_TICK_INTERVAL); ticker.set_missed_tick_behavior(MissedTickBehavior::Skip); - let mut worker = loop { - ticker.tick().await; - - match ConnectorSourceWorker::create( - &connector_client, - &source, - ConnectorSourceWorker::DEFAULT_SOURCE_WORKER_TICK_INTERVAL, - splits.clone(), - metrics.clone(), - ) - .await - { - Ok(worker) => { - break worker; - } - Err(e) => { - tracing::warn!("failed to create source worker: {}", e); + dispatch_source_prop!(connector_properties, prop, { + let mut worker = loop { + ticker.tick().await; + + match ConnectorSourceWorker::create( + &connector_client, + &source, + prop.deref().clone(), + DEFAULT_SOURCE_WORKER_TICK_INTERVAL, + splits.clone(), + metrics.clone(), + ) + .await + { + Ok(worker) => { + break worker; + } + Err(e) => { + tracing::warn!("failed to create source worker: {}", e); + } } - } - }; + }; - worker.run(sync_call_rx).await + worker.run(sync_call_rx).await + }); }); managed_sources.insert( @@ -765,6 +780,7 @@ where splits: current_splits_ref, }, ); + Ok(()) } async fn create_source_worker( @@ -775,38 +791,41 @@ where metrics: Arc, ) -> MetaResult<()> { let current_splits_ref = Arc::new(Mutex::new(SharedSplitMap { splits: None })); - let mut worker = ConnectorSourceWorker::create( - &connector_client, - source, - ConnectorSourceWorker::DEFAULT_SOURCE_WORKER_TICK_INTERVAL, - current_splits_ref.clone(), - metrics, - ) - .await?; - - tracing::info!("spawning new watcher for source {}", source.id); - - // don't force tick in process of recovery. One source down should not lead to meta recovery - // failure. - if force_tick { - // if fail to fetch meta info, will refuse to create source - - // todo: make the timeout configurable, longer than `properties.sync.call.timeout` in - // kafka - tokio::time::timeout(Self::DEFAULT_SOURCE_TICK_TIMEOUT, worker.tick()) - .await - .map_err(|_e| { - anyhow!( - "failed to fetch meta info for source {}, error: timeout {}", - source.id, - Self::DEFAULT_SOURCE_TICK_TIMEOUT.as_secs() - ) - })??; - } - + let connector_properties = extract_prop_from_source(source)?; let (sync_call_tx, sync_call_rx) = tokio::sync::mpsc::unbounded_channel(); + let handle = dispatch_source_prop!(connector_properties, prop, { + let mut worker = ConnectorSourceWorker::create( + &connector_client, + source, + *prop, + DEFAULT_SOURCE_WORKER_TICK_INTERVAL, + current_splits_ref.clone(), + metrics, + ) + .await?; + + tracing::info!("spawning new watcher for source {}", source.id); + + // don't force tick in process of recovery. One source down should not lead to meta + // recovery failure. + if force_tick { + // if fail to fetch meta info, will refuse to create source + + // todo: make the timeout configurable, longer than `properties.sync.call.timeout` + // in kafka + tokio::time::timeout(Self::DEFAULT_SOURCE_TICK_TIMEOUT, worker.tick()) + .await + .map_err(|_e| { + anyhow!( + "failed to fetch meta info for source {}, error: timeout {}", + source.id, + Self::DEFAULT_SOURCE_TICK_TIMEOUT.as_secs() + ) + })??; + } - let handle = tokio::spawn(async move { worker.run(sync_call_rx).await }); + tokio::spawn(async move { worker.run(sync_call_rx).await }) + }); managed_sources.insert( source.id, diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index df0dc8e147a59..376c7710a183d 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -16,16 +16,18 @@ use std::collections::HashMap; use std::sync::Arc; use futures::future::try_join_all; +use futures::stream::pending; use futures::StreamExt; use itertools::Itertools; use risingwave_common::catalog::ColumnId; use risingwave_common::error::ErrorCode::ConnectorError; use risingwave_common::error::{internal_error, Result}; use risingwave_common::util::select_all; +use risingwave_connector::dispatch_source_prop; use risingwave_connector::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; use risingwave_connector::source::{ - BoxSourceWithStateStream, Column, ConnectorProperties, ConnectorState, SourceColumnDesc, - SourceContext, SplitReaderImpl, + create_split_reader, BoxSourceWithStateStream, Column, ConnectorProperties, ConnectorState, + SourceColumnDesc, SourceContext, SplitReader, }; #[derive(Clone, Debug)] @@ -74,10 +76,14 @@ impl ConnectorSource { pub async fn stream_reader( &self, - splits: ConnectorState, + state: ConnectorState, column_ids: Vec, source_ctx: Arc, ) -> Result { + let Some(splits) = state + else { + return Ok(pending().boxed()); + }; let config = self.config.clone(); let columns = self.get_target_columns(column_ids)?; @@ -99,53 +105,46 @@ impl ConnectorSource { }, }; - let readers = if config.support_multiple_splits() { - tracing::debug!( - "spawning connector split reader for multiple splits {:?}", - splits - ); + let support_multiple_splits = config.support_multiple_splits(); - let reader = SplitReaderImpl::create( - config, - splits, - parser_config, - source_ctx, - data_gen_columns, - ) - .await?; + dispatch_source_prop!(config, prop, { + let readers = if support_multiple_splits { + tracing::debug!( + "spawning connector split reader for multiple splits {:?}", + splits + ); - vec![reader] - } else { - let to_reader_splits = match splits { - Some(vec_split_impl) => vec_split_impl - .into_iter() - .map(|split| Some(vec![split])) - .collect::>(), - None => vec![None], - }; + let reader = + create_split_reader(*prop, splits, parser_config, source_ctx, data_gen_columns) + .await?; - try_join_all(to_reader_splits.into_iter().map(|state| { - tracing::debug!("spawning connector split reader for split {:?}", state); - let props = config.clone(); - let data_gen_columns = data_gen_columns.clone(); - let parser_config = parser_config.clone(); - // TODO: is this reader split across multiple threads...? Realistically, we want - // source_ctx to live in a single actor. - let source_ctx = source_ctx.clone(); - async move { - SplitReaderImpl::create( - props, - state, - parser_config, - source_ctx, - data_gen_columns, - ) - .await - } - })) - .await? - }; + vec![reader] + } else { + let to_reader_splits = splits.into_iter().map(|split| vec![split]); - Ok(select_all(readers.into_iter().map(|r| r.into_stream())).boxed()) + try_join_all(to_reader_splits.into_iter().map(|splits| { + tracing::debug!("spawning connector split reader for split {:?}", splits); + let props = prop.clone(); + let data_gen_columns = data_gen_columns.clone(); + let parser_config = parser_config.clone(); + // TODO: is this reader split across multiple threads...? Realistically, we want + // source_ctx to live in a single actor. + let source_ctx = source_ctx.clone(); + async move { + create_split_reader( + *props, + splits, + parser_config, + source_ctx, + data_gen_columns, + ) + .await + } + })) + .await? + }; + + Ok(select_all(readers.into_iter().map(|r| r.into_stream())).boxed()) + }) } } diff --git a/src/source/src/fs_connector_source.rs b/src/source/src/fs_connector_source.rs index 94a90ff5f69e0..a3266b0cd348c 100644 --- a/src/source/src/fs_connector_source.rs +++ b/src/source/src/fs_connector_source.rs @@ -15,12 +15,16 @@ use std::collections::HashMap; use std::sync::Arc; +use futures::stream::pending; +use futures::StreamExt; use risingwave_common::catalog::ColumnId; use risingwave_common::error::ErrorCode::ConnectorError; -use risingwave_common::error::{internal_error, Result, RwError}; +use risingwave_common::error::{internal_error, Result}; +use risingwave_connector::dispatch_source_prop; use risingwave_connector::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; use risingwave_connector::source::{ - ConnectorProperties, ConnectorState, SourceColumnDesc, SourceContext, SplitReaderImpl, + create_split_reader, BoxSourceWithStateStream, ConnectorProperties, ConnectorState, + SourceColumnDesc, SourceContext, SplitReader, }; #[derive(Clone, Debug)] @@ -78,7 +82,7 @@ impl FsConnectorSource { state: ConnectorState, column_ids: Vec, source_ctx: Arc, - ) -> Result { + ) -> Result { let config = self.config.clone(); let columns = self.get_target_columns(column_ids)?; @@ -88,8 +92,16 @@ impl FsConnectorSource { rw_columns: columns, }, }; - SplitReaderImpl::create(config, state, parser_config, source_ctx, None) - .await - .map_err(RwError::from) + let stream = match state { + None => pending().boxed(), + Some(splits) => { + dispatch_source_prop!(config, prop, { + create_split_reader(*prop, splits, parser_config, source_ctx, None) + .await? + .into_stream() + }) + } + }; + Ok(stream) } } diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index dd70e908eff03..33ccb18468e8d 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -112,12 +112,11 @@ impl FsSourceExecutor { None, self.actor_ctx.error_suppressor.clone(), ); - let stream_reader = source_desc + source_desc .source .stream_reader(state, column_ids, Arc::new(source_ctx)) .await - .map_err(StreamExecutorError::connector_error)?; - Ok(stream_reader.into_stream()) + .map_err(StreamExecutorError::connector_error) } async fn apply_split_change( From a205e093e715fefe03214bba464c76e4fdf1f854 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 4 Sep 2023 18:53:36 +0800 Subject: [PATCH 2/9] refactor(source): specify cdc generic type parameter for different cdc source --- Cargo.lock | 1 + src/connector/Cargo.toml | 1 + src/connector/src/macros.rs | 77 +++------- src/connector/src/source/base.rs | 70 +++++----- .../src/source/cdc/enumerator/mod.rs | 132 +++++++++++------- src/connector/src/source/cdc/mod.rs | 81 +++++++++-- src/connector/src/source/cdc/source/reader.rs | 46 +++--- src/connector/src/source/cdc/split.rs | 12 +- src/connector/src/source/common.rs | 76 ++++++++++ .../src/source/datagen/source/reader.rs | 19 +-- .../src/source/google_pubsub/source/reader.rs | 13 +- .../src/source/kafka/source/reader.rs | 16 ++- .../src/source/kinesis/source/reader.rs | 26 ++-- src/connector/src/source/mod.rs | 2 + .../src/source/pulsar/source/reader.rs | 16 +-- 15 files changed, 361 insertions(+), 227 deletions(-) create mode 100644 src/connector/src/source/common.rs diff --git a/Cargo.lock b/Cargo.lock index dbf61ce31ee26..68242a7f36bb5 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6566,6 +6566,7 @@ dependencies = [ "num-bigint", "opendal", "parking_lot 0.12.1", + "paste", "prometheus", "prost", "prost-reflect", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 5ab001cfe7a11..315c32b677a8a 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -57,6 +57,7 @@ nexmark = { version = "0.2", features = ["serde"] } num-bigint = "0.4" opendal = "0.39" parking_lot = "0.12" +paste = "1" prometheus = { version = "0.13", features = ["process"] } prost = { version = "0.11.9", features = ["no-recursion-limit"] } prost-reflect = "0.11.4" diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index 70fb6130b8717..4291978e18117 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -37,7 +37,7 @@ macro_rules! impl_split_enumerator { .map(SplitImpl::$variant_name) .collect_vec() }) - .map_err(|e| ErrorCode::ConnectorError(e.into()).into()), + .map_err(|e| risingwave_common::error::ErrorCode::ConnectorError(e.into()).into()), )* } } @@ -55,6 +55,25 @@ macro_rules! impl_split { } } } + $( + impl TryFrom for $split { + type Error = anyhow::Error; + + fn try_from(split: SplitImpl) -> std::result::Result { + match split { + SplitImpl::$variant_name(inner) => Ok(inner), + other => Err(anyhow::anyhow!("expect {} but get {:?}", stringify!($split), other)) + } + } + } + + impl From<$split> for SplitImpl { + fn from(split: $split) -> SplitImpl { + SplitImpl::$variant_name(split) + } + } + + )* impl TryFrom<&ConnectorSplit> for SplitImpl { type Error = anyhow::Error; @@ -161,7 +180,7 @@ macro_rules! impl_connector_properties { pub fn extract(mut props: HashMap) -> Result { const UPSTREAM_SOURCE_KEY: &str = "connector"; let connector = props.remove(UPSTREAM_SOURCE_KEY).ok_or_else(|| anyhow!("Must specify 'connector' in WITH clause"))?; - if connector.ends_with("cdc") { + if connector.ends_with("-cdc") { ConnectorProperties::new_cdc_properties(&connector, props) } else { let json_value = serde_json::to_value(props).map_err(|e| anyhow!(e))?; @@ -180,57 +199,3 @@ macro_rules! impl_connector_properties { } } } - -#[macro_export] -macro_rules! impl_common_split_reader_logic { - ($reader:ty, $props:ty) => { - impl $reader { - #[try_stream(boxed, ok = $crate::source::StreamChunkWithState, error = risingwave_common::error::RwError)] - pub(crate) async fn into_chunk_stream(self) { - let parser_config = self.parser_config.clone(); - let actor_id = self.source_ctx.source_info.actor_id.to_string(); - let source_id = self.source_ctx.source_info.source_id.to_string(); - let metrics = self.source_ctx.metrics.clone(); - let source_ctx = self.source_ctx.clone(); - - let data_stream = self.into_data_stream(); - - let data_stream = data_stream - .inspect_ok(move |data_batch| { - let mut by_split_id = std::collections::HashMap::new(); - - for msg in data_batch { - by_split_id - .entry(msg.split_id.as_ref()) - .or_insert_with(Vec::new) - .push(msg); - } - - for (split_id, msgs) in by_split_id { - metrics - .partition_input_count - .with_label_values(&[&actor_id, &source_id, split_id]) - .inc_by(msgs.len() as u64); - - let sum_bytes = msgs - .iter() - .flat_map(|msg| msg.payload.as_ref().map(|p| p.len() as u64)) - .sum(); - - metrics - .partition_input_bytes - .with_label_values(&[&actor_id, &source_id, &split_id]) - .inc_by(sum_bytes); - } - }).boxed(); - - let parser = - $crate::parser::ByteStreamSourceParserImpl::create(parser_config, source_ctx).await?; - #[for_await] - for msg_batch in parser.into_stream(data_stream) { - yield msg_batch?; - } - } - } - }; -} diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 8b5b3287fb445..1178cf68939ec 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -25,28 +25,30 @@ use itertools::Itertools; use parking_lot::Mutex; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::TableId; -use risingwave_common::error::{ErrorCode, ErrorSuppressor, Result as RwResult, RwError}; +use risingwave_common::error::{ErrorSuppressor, Result as RwResult, RwError}; use risingwave_common::types::{JsonbVal, Scalar}; use risingwave_pb::connector_service::PbTableSchema; use risingwave_pb::source::ConnectorSplit; use risingwave_rpc_client::ConnectorClient; -use serde::{Deserialize, Serialize}; use super::datagen::DatagenMeta; -use super::filesystem::{FsSplit, S3FileReader, S3Properties, S3SplitEnumerator, S3_CONNECTOR}; +use super::filesystem::{FsSplit, S3Properties, S3_CONNECTOR}; use super::google_pubsub::GooglePubsubMeta; use super::kafka::KafkaMeta; use super::monitor::SourceMetrics; use super::nexmark::source::message::NexmarkMeta; use crate::parser::ParserConfig; use crate::source::cdc::{ - CdcProperties, CdcSplitReader, DebeziumCdcSplit, DebeziumSplitEnumerator, CITUS_CDC_CONNECTOR, - MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, + CdcProperties, CdcSplitReader, Citus, CitusDebeziumSplitEnumerator, DebeziumCdcSplit, + DebeziumSplitEnumerator, Mysql, MysqlDebeziumSplitEnumerator, Postgres, + PostgresDebeziumSplitEnumerator, CITUS_CDC_CONNECTOR, MYSQL_CDC_CONNECTOR, + POSTGRES_CDC_CONNECTOR, }; use crate::source::datagen::{ DatagenProperties, DatagenSplit, DatagenSplitEnumerator, DatagenSplitReader, DATAGEN_CONNECTOR, }; use crate::source::dummy_connector::DummySplitReader; +use crate::source::filesystem::{S3FileReader, S3SplitEnumerator}; use crate::source::google_pubsub::{ PubsubProperties, PubsubSplit, PubsubSplitEnumerator, PubsubSplitReader, GOOGLE_PUBSUB_CONNECTOR, @@ -287,7 +289,7 @@ pub trait SplitReader: Sized { fn into_stream(self) -> BoxSourceWithStateStream; } -#[derive(Clone, Debug, Deserialize)] +#[derive(Clone, Debug)] pub enum ConnectorProperties { Kafka(Box), Pulsar(Box), @@ -295,9 +297,9 @@ pub enum ConnectorProperties { Nexmark(Box), Datagen(Box), S3(Box), - MySqlCdc(Box), - PostgresCdc(Box), - CitusCdc(Box), + MySqlCdc(Box>), + PostgresCdc(Box>), + CitusCdc(Box>), GooglePubsub(Box), Dummy(Box<()>), } @@ -308,19 +310,16 @@ impl ConnectorProperties { properties: HashMap, ) -> Result { match connector_name { - MYSQL_CDC_CONNECTOR => Ok(Self::MySqlCdc(Box::new(CdcProperties { + MYSQL_CDC_CONNECTOR => Ok(Self::MySqlCdc(Box::new(CdcProperties:: { props: properties, - source_type: "mysql".to_string(), ..Default::default() }))), - POSTGRES_CDC_CONNECTOR => Ok(Self::PostgresCdc(Box::new(CdcProperties { + POSTGRES_CDC_CONNECTOR => Ok(Self::PostgresCdc(Box::new(CdcProperties:: { props: properties, - source_type: "postgres".to_string(), ..Default::default() }))), - CITUS_CDC_CONNECTOR => Ok(Self::CitusCdc(Box::new(CdcProperties { + CITUS_CDC_CONNECTOR => Ok(Self::CitusCdc(Box::new(CdcProperties:: { props: properties, - source_type: "citus".to_string(), ..Default::default() }))), _ => Err(anyhow!("unexpected cdc connector '{}'", connector_name,)), @@ -329,9 +328,13 @@ impl ConnectorProperties { pub fn init_cdc_properties(&mut self, table_schema: Option) { match self { - ConnectorProperties::MySqlCdc(c) - | ConnectorProperties::PostgresCdc(c) - | ConnectorProperties::CitusCdc(c) => { + ConnectorProperties::MySqlCdc(c) => { + c.table_schema = table_schema; + } + ConnectorProperties::PostgresCdc(c) => { + c.table_schema = table_schema; + } + ConnectorProperties::CitusCdc(c) => { c.table_schema = table_schema; } _ => {} @@ -352,7 +355,7 @@ impl ConnectorProperties { } } -#[derive(Debug, Clone, Serialize, Deserialize, EnumAsInner, PartialEq, Hash)] +#[derive(Debug, Clone, EnumAsInner, PartialEq, Hash)] pub enum SplitImpl { Kafka(KafkaSplit), Pulsar(PulsarSplit), @@ -360,9 +363,9 @@ pub enum SplitImpl { Nexmark(NexmarkSplit), Datagen(DatagenSplit), GooglePubsub(PubsubSplit), - MySqlCdc(DebeziumCdcSplit), - PostgresCdc(DebeziumCdcSplit), - CitusCdc(DebeziumCdcSplit), + MySqlCdc(DebeziumCdcSplit), + PostgresCdc(DebeziumCdcSplit), + CitusCdc(DebeziumCdcSplit), S3(FsSplit), } @@ -392,9 +395,9 @@ pub enum SplitReaderImpl { Nexmark(Box), Pulsar(Box), Datagen(Box), - MySqlCdc(Box), - PostgresCdc(Box), - CitusCdc(Box), + MySqlCdc(Box>), + PostgresCdc(Box>), + CitusCdc(Box>), GooglePubsub(Box), } @@ -404,9 +407,9 @@ pub enum SplitEnumeratorImpl { Kinesis(KinesisSplitEnumerator), Nexmark(NexmarkSplitEnumerator), Datagen(DatagenSplitEnumerator), - MySqlCdc(DebeziumSplitEnumerator), - PostgresCdc(DebeziumSplitEnumerator), - CitusCdc(DebeziumSplitEnumerator), + MySqlCdc(MysqlDebeziumSplitEnumerator), + PostgresCdc(PostgresDebeziumSplitEnumerator), + CitusCdc(CitusDebeziumSplitEnumerator), GooglePubsub(PubsubSplitEnumerator), S3(S3SplitEnumerator), } @@ -418,9 +421,6 @@ impl_connector_properties! { { Nexmark, NEXMARK_CONNECTOR }, { Datagen, DATAGEN_CONNECTOR }, { S3, S3_CONNECTOR }, - { MySqlCdc, MYSQL_CDC_CONNECTOR }, - { PostgresCdc, POSTGRES_CDC_CONNECTOR }, - { CitusCdc, CITUS_CDC_CONNECTOR }, { GooglePubsub, GOOGLE_PUBSUB_CONNECTOR} } @@ -444,9 +444,9 @@ impl_split! { { Nexmark, NEXMARK_CONNECTOR, NexmarkSplit }, { Datagen, DATAGEN_CONNECTOR, DatagenSplit }, { GooglePubsub, GOOGLE_PUBSUB_CONNECTOR, PubsubSplit }, - { MySqlCdc, MYSQL_CDC_CONNECTOR, DebeziumCdcSplit }, - { PostgresCdc, POSTGRES_CDC_CONNECTOR, DebeziumCdcSplit }, - { CitusCdc, CITUS_CDC_CONNECTOR, DebeziumCdcSplit }, + { MySqlCdc, MYSQL_CDC_CONNECTOR, DebeziumCdcSplit }, + { PostgresCdc, POSTGRES_CDC_CONNECTOR, DebeziumCdcSplit }, + { CitusCdc, CITUS_CDC_CONNECTOR, DebeziumCdcSplit }, { S3, S3_CONNECTOR, FsSplit } } @@ -645,7 +645,6 @@ mod tests { let conn_props = ConnectorProperties::extract(user_props_mysql).unwrap(); if let ConnectorProperties::MySqlCdc(c) = conn_props { - assert_eq!(c.source_type, "mysql"); assert_eq!(c.props.get("connector_node_addr").unwrap(), "localhost"); assert_eq!(c.props.get("database.hostname").unwrap(), "127.0.0.1"); assert_eq!(c.props.get("database.port").unwrap(), "3306"); @@ -659,7 +658,6 @@ mod tests { let conn_props = ConnectorProperties::extract(user_props_postgres).unwrap(); if let ConnectorProperties::PostgresCdc(c) = conn_props { - assert_eq!(c.source_type, "postgres"); assert_eq!(c.props.get("connector_node_addr").unwrap(), "localhost"); assert_eq!(c.props.get("database.hostname").unwrap(), "127.0.0.1"); assert_eq!(c.props.get("database.port").unwrap(), "5432"); diff --git a/src/connector/src/source/cdc/enumerator/mod.rs b/src/connector/src/source/cdc/enumerator/mod.rs index 91d2bf0bb7bbf..6ced5ee79da1e 100644 --- a/src/connector/src/source/cdc/enumerator/mod.rs +++ b/src/connector/src/source/cdc/enumerator/mod.rs @@ -12,38 +12,43 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::marker::PhantomData; use std::str::FromStr; use anyhow::anyhow; use async_trait::async_trait; use itertools::Itertools; use risingwave_common::util::addr::HostAddr; -use risingwave_pb::connector_service::SourceType as PbSourceType; +use risingwave_pb::connector_service::SourceType; use crate::source::cdc::{ - CdcProperties, CdcSplitBase, DebeziumCdcSplit, MySqlCdcSplit, PostgresCdcSplit, + CdcProperties, CdcSourceTypeTrait, CdcSplitBase, Citus, DebeziumCdcSplit, MySqlCdcSplit, Mysql, + Postgres, PostgresCdcSplit, }; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; pub const DATABASE_SERVERS_KEY: &str = "database.servers"; #[derive(Debug)] -pub struct DebeziumSplitEnumerator { +pub struct DebeziumSplitEnumerator { /// The source_id in the catalog source_id: u32, - source_type: PbSourceType, worker_node_addrs: Vec, + _phantom: PhantomData, } #[async_trait] -impl SplitEnumerator for DebeziumSplitEnumerator { - type Properties = CdcProperties; - type Split = DebeziumCdcSplit; +impl SplitEnumerator for DebeziumSplitEnumerator +where + Self: ListCdcSplits, +{ + type Properties = CdcProperties; + type Split = DebeziumCdcSplit; async fn new( - props: CdcProperties, + props: CdcProperties, context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> anyhow::Result { let connector_client = context.connector_client.clone().ok_or_else(|| { anyhow!("connector node endpoint not specified or unable to connect to connector node") })?; @@ -59,12 +64,16 @@ impl SplitEnumerator for DebeziumSplitEnumerator { .transpose()? .unwrap_or_default(); - let source_type = props.get_source_type_pb()?; + assert_eq!( + props.get_source_type_pb(), + SourceType::from(T::source_type()) + ); + // validate connector properties connector_client .validate_source_properties( context.info.source_id as u64, - props.get_source_type_pb()?, + props.get_source_type_pb(), props.props, props.table_schema, ) @@ -73,54 +82,73 @@ impl SplitEnumerator for DebeziumSplitEnumerator { tracing::debug!("validate cdc source properties success"); Ok(Self { source_id: context.info.source_id, - source_type, worker_node_addrs: server_addrs, + _phantom: PhantomData, }) } - async fn list_splits(&mut self) -> anyhow::Result> { - match self.source_type { - PbSourceType::Mysql => { - // CDC source only supports single split - let split = MySqlCdcSplit { - inner: CdcSplitBase::new(self.source_id, None), - }; - let dbz_split = DebeziumCdcSplit { - mysql_split: Some(split), - pg_split: None, - }; - Ok(vec![dbz_split]) - } - PbSourceType::Postgres => { + async fn list_splits(&mut self) -> anyhow::Result>> { + Ok(self.list_cdc_splits()) + } +} + +pub trait ListCdcSplits { + type CdcSourceType: CdcSourceTypeTrait; + fn list_cdc_splits(&mut self) -> Vec>; +} + +impl ListCdcSplits for DebeziumSplitEnumerator { + type CdcSourceType = Mysql; + + fn list_cdc_splits(&mut self) -> Vec> { + // CDC source only supports single split + let split = MySqlCdcSplit { + inner: CdcSplitBase::new(self.source_id, None), + }; + let dbz_split = DebeziumCdcSplit { + mysql_split: Some(split), + pg_split: None, + _phantom: PhantomData, + }; + vec![dbz_split] + } +} + +impl ListCdcSplits for DebeziumSplitEnumerator { + type CdcSourceType = Postgres; + + fn list_cdc_splits(&mut self) -> Vec> { + let split = PostgresCdcSplit { + inner: CdcSplitBase::new(self.source_id, None), + server_addr: None, + }; + let dbz_split = DebeziumCdcSplit { + mysql_split: None, + pg_split: Some(split), + _phantom: Default::default(), + }; + vec![dbz_split] + } +} + +impl ListCdcSplits for DebeziumSplitEnumerator { + type CdcSourceType = Citus; + + fn list_cdc_splits(&mut self) -> Vec> { + self.worker_node_addrs + .iter() + .enumerate() + .map(|(id, addr)| { let split = PostgresCdcSplit { - inner: CdcSplitBase::new(self.source_id, None), - server_addr: None, + inner: CdcSplitBase::new(id as u32, None), + server_addr: Some(addr.to_string()), }; - let dbz_split = DebeziumCdcSplit { + DebeziumCdcSplit { mysql_split: None, pg_split: Some(split), - }; - Ok(vec![dbz_split]) - } - PbSourceType::Citus => { - let splits = self - .worker_node_addrs - .iter() - .enumerate() - .map(|(id, addr)| { - let split = PostgresCdcSplit { - inner: CdcSplitBase::new(id as u32, None), - server_addr: Some(addr.to_string()), - }; - DebeziumCdcSplit { - mysql_split: None, - pg_split: Some(split), - } - }) - .collect_vec(); - Ok(splits) - } - _ => Err(anyhow!("unexpected source type")), - } + _phantom: Default::default(), + } + }) + .collect_vec() } } diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index 308e6ed88206c..e7bff4b4a7ec0 100644 --- a/src/connector/src/source/cdc/mod.rs +++ b/src/connector/src/source/cdc/mod.rs @@ -15,34 +15,85 @@ pub mod enumerator; pub mod source; pub mod split; - use std::collections::HashMap; +use std::marker::PhantomData; -use anyhow::anyhow; pub use enumerator::*; -use risingwave_pb::connector_service::{SourceType, TableSchema}; -use serde::Deserialize; +use paste::paste; +use risingwave_pb::connector_service::{PbSourceType, SourceType, TableSchema}; pub use source::*; pub use split::*; -pub const MYSQL_CDC_CONNECTOR: &str = "mysql-cdc"; -pub const POSTGRES_CDC_CONNECTOR: &str = "postgres-cdc"; -pub const CITUS_CDC_CONNECTOR: &str = "citus-cdc"; +pub const MYSQL_CDC_CONNECTOR: &str = Mysql::CDC_CONNECTOR_NAME; +pub const POSTGRES_CDC_CONNECTOR: &str = Postgres::CDC_CONNECTOR_NAME; +pub const CITUS_CDC_CONNECTOR: &str = Citus::CDC_CONNECTOR_NAME; + +pub trait CdcSourceTypeTrait: Send + Sync + Clone + 'static { + const CDC_CONNECTOR_NAME: &'static str; + fn source_type() -> CdcSourceType; +} + +macro_rules! impl_cdc_source_type { + ($({$source_type:ident, $name:expr }),*) => { + $( + paste!{ + #[derive(Clone, Debug, Default, PartialEq, Eq, Hash)] + pub struct $source_type; + impl CdcSourceTypeTrait for $source_type { + const CDC_CONNECTOR_NAME: &'static str = concat!($name, "-cdc"); + fn source_type() -> CdcSourceType { + CdcSourceType::$source_type + } + } + + pub type [< $source_type DebeziumSplitEnumerator >] = DebeziumSplitEnumerator<$source_type>; + } + )* + + pub enum CdcSourceType { + $( + $source_type, + )* + } -#[derive(Clone, Debug, Deserialize, Default)] -pub struct CdcProperties { - /// Type of the cdc source, e.g. mysql, postgres - pub source_type: String, + impl From for CdcSourceType { + fn from(value: PbSourceType) -> Self { + match value { + PbSourceType::Unspecified => unreachable!(), + $( + PbSourceType::$source_type => CdcSourceType::$source_type, + )* + } + } + } + + impl From for PbSourceType { + fn from(this: CdcSourceType) -> PbSourceType { + match this { + $( + CdcSourceType::$source_type => PbSourceType::$source_type, + )* + } + } + } + } +} + +impl_cdc_source_type!({ Mysql, "mysql" }, { Postgres, "postgres" }, { Citus, "citus" }); + +#[derive(Clone, Debug, Default)] +pub struct CdcProperties { /// Properties specified in the WITH clause by user pub props: HashMap, /// Schema of the source specified by users pub table_schema: Option, + + pub _phantom: PhantomData, } -impl CdcProperties { - pub fn get_source_type_pb(&self) -> anyhow::Result { - SourceType::from_str_name(&self.source_type.to_ascii_uppercase()) - .ok_or_else(|| anyhow!("unknown source type: {}", self.source_type)) +impl CdcProperties { + pub fn get_source_type_pb(&self) -> SourceType { + SourceType::from(T::source_type()) } } diff --git a/src/connector/src/source/cdc/source/reader.rs b/src/connector/src/source/cdc/source/reader.rs index 200c91a8a5051..974ec8877d2f6 100644 --- a/src/connector/src/source/cdc/source/reader.rs +++ b/src/connector/src/source/cdc/source/reader.rs @@ -16,28 +16,26 @@ use std::str::FromStr; use anyhow::{anyhow, Result}; use async_trait::async_trait; -use futures::{pin_mut, StreamExt, TryStreamExt}; +use futures::pin_mut; use futures_async_stream::try_stream; use risingwave_common::util::addr::HostAddr; use risingwave_pb::connector_service::GetEventStreamResponse; -use crate::impl_common_split_reader_logic; use crate::parser::ParserConfig; use crate::source::base::SourceMessage; -use crate::source::cdc::CdcProperties; +use crate::source::cdc::{CdcProperties, CdcSourceType, CdcSourceTypeTrait, DebeziumCdcSplit}; +use crate::source::common::{into_chunk_stream, CommonSplitReader}; use crate::source::{ BoxSourceWithStateStream, Column, SourceContextRef, SplitId, SplitImpl, SplitMetaData, SplitReader, }; -impl_common_split_reader_logic!(CdcSplitReader, CdcProperties); - -pub struct CdcSplitReader { +pub struct CdcSplitReader { source_id: u64, start_offset: Option, // host address of worker node for a Citus cluster server_addr: Option, - conn_props: CdcProperties, + conn_props: CdcProperties, split_id: SplitId, // whether the full snapshot phase is done @@ -47,22 +45,25 @@ pub struct CdcSplitReader { } #[async_trait] -impl SplitReader for CdcSplitReader { - type Properties = CdcProperties; +impl SplitReader for CdcSplitReader +where + DebeziumCdcSplit: TryFrom, +{ + type Properties = CdcProperties; #[allow(clippy::unused_async)] async fn new( - conn_props: CdcProperties, + conn_props: CdcProperties, splits: Vec, parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, ) -> Result { assert_eq!(splits.len(), 1); - let split = splits.into_iter().next().unwrap(); + let split = DebeziumCdcSplit::::try_from(splits.into_iter().next().unwrap())?; let split_id = split.id(); - match split { - SplitImpl::MySqlCdc(split) | SplitImpl::PostgresCdc(split) => Ok(Self { + match T::source_type() { + CdcSourceType::Mysql | CdcSourceType::Postgres => Ok(Self { source_id: split.split_id() as u64, start_offset: split.start_offset().clone(), server_addr: None, @@ -72,7 +73,7 @@ impl SplitReader for CdcSplitReader { parser_config, source_ctx, }), - SplitImpl::CitusCdc(split) => Ok(Self { + CdcSourceType::Citus => Ok(Self { source_id: split.split_id() as u64, start_offset: split.start_offset().clone(), server_addr: split.server_addr().clone(), @@ -82,20 +83,21 @@ impl SplitReader for CdcSplitReader { parser_config, source_ctx, }), - - _ => Err(anyhow!( - "failed to create cdc split reader: invalid splis info" - )), } } fn into_stream(self) -> BoxSourceWithStateStream { - self.into_chunk_stream() + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) } } -impl CdcSplitReader { - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] +impl CommonSplitReader for CdcSplitReader +where + Self: SplitReader, +{ + #[try_stream(ok = Vec, error = anyhow::Error)] async fn into_data_stream(self) { let cdc_client = self.source_ctx.connector_client.clone().ok_or_else(|| { anyhow!("connector node endpoint not specified or unable to connect to connector node") @@ -122,7 +124,7 @@ impl CdcSplitReader { let cdc_stream = cdc_client .start_source_stream( self.source_id, - self.conn_props.get_source_type_pb()?, + self.conn_props.get_source_type_pb(), self.start_offset, properties, self.snapshot_done, diff --git a/src/connector/src/source/cdc/split.rs b/src/connector/src/source/cdc/split.rs index e72fe97b5658f..6535585a5309e 100644 --- a/src/connector/src/source/cdc/split.rs +++ b/src/connector/src/source/cdc/split.rs @@ -13,11 +13,13 @@ // limitations under the License. use std::collections::HashMap; +use std::marker::PhantomData; use anyhow::anyhow; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::source::cdc::CdcSourceTypeTrait; use crate::source::{SplitId, SplitMetaData}; /// The base states of a CDC split, which will be persisted to checkpoint. @@ -151,12 +153,15 @@ impl PostgresCdcSplit { } #[derive(Clone, Serialize, Deserialize, Debug, PartialEq, Hash)] -pub struct DebeziumCdcSplit { +pub struct DebeziumCdcSplit { pub mysql_split: Option, pub pg_split: Option, + + #[serde(skip)] + pub _phantom: PhantomData, } -impl SplitMetaData for DebeziumCdcSplit { +impl SplitMetaData for DebeziumCdcSplit { fn id(&self) -> SplitId { assert!(self.mysql_split.is_some() || self.pg_split.is_some()); if let Some(split) = &self.mysql_split { @@ -177,11 +182,12 @@ impl SplitMetaData for DebeziumCdcSplit { } } -impl DebeziumCdcSplit { +impl DebeziumCdcSplit { pub fn new(mysql_split: Option, pg_split: Option) -> Self { Self { mysql_split, pg_split, + _phantom: PhantomData, } } diff --git a/src/connector/src/source/common.rs b/src/connector/src/source/common.rs new file mode 100644 index 0000000000000..02f1cbde3de38 --- /dev/null +++ b/src/connector/src/source/common.rs @@ -0,0 +1,76 @@ +// 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 futures::{Stream, StreamExt, TryStreamExt}; +use futures_async_stream::try_stream; +use risingwave_common::error::RwError; + +use crate::parser::ParserConfig; +use crate::source::{SourceContextRef, SourceMessage, SplitReader, StreamChunkWithState}; + +pub(crate) trait CommonSplitReader: SplitReader + 'static { + fn into_data_stream( + self, + ) -> impl Stream, anyhow::Error>> + Send; +} + +#[try_stream(boxed, ok = StreamChunkWithState, error = RwError)] +pub(crate) async fn into_chunk_stream( + reader: impl CommonSplitReader + Send, + parser_config: ParserConfig, + source_ctx: SourceContextRef, +) { + let actor_id = source_ctx.source_info.actor_id.to_string(); + let source_id = source_ctx.source_info.source_id.to_string(); + let metrics = source_ctx.metrics.clone(); + + let data_stream = reader.into_data_stream(); + + let data_stream = data_stream + .inspect_ok(move |data_batch| { + let mut by_split_id = std::collections::HashMap::new(); + + for msg in data_batch { + by_split_id + .entry(msg.split_id.as_ref()) + .or_insert_with(Vec::new) + .push(msg); + } + + for (split_id, msgs) in by_split_id { + metrics + .partition_input_count + .with_label_values(&[&actor_id, &source_id, split_id]) + .inc_by(msgs.len() as u64); + + let sum_bytes = msgs + .iter() + .flat_map(|msg| msg.payload.as_ref().map(|p| p.len() as u64)) + .sum(); + + metrics + .partition_input_bytes + .with_label_values(&[&actor_id, &source_id, split_id]) + .inc_by(sum_bytes); + } + }) + .boxed(); + + let parser = + crate::parser::ByteStreamSourceParserImpl::create(parser_config, source_ctx).await?; + #[for_await] + for msg_batch in parser.into_stream(data_stream) { + yield msg_batch?; + } +} diff --git a/src/connector/src/source/datagen/source/reader.rs b/src/connector/src/source/datagen/source/reader.rs index d040a3cb63f21..30d1bfae10c4e 100644 --- a/src/connector/src/source/datagen/source/reader.rs +++ b/src/connector/src/source/datagen/source/reader.rs @@ -16,23 +16,20 @@ use std::collections::HashMap; use anyhow::{anyhow, Result}; use async_trait::async_trait; -use futures::{StreamExt, TryStreamExt}; -use futures_async_stream::try_stream; +use futures::{Stream, StreamExt, TryStreamExt}; use risingwave_common::field_generator::{FieldGeneratorImpl, VarcharProperty}; use super::generator::DatagenEventGenerator; -use crate::impl_common_split_reader_logic; use crate::parser::{EncodingProperties, ParserConfig, ProtocolProperties}; +use crate::source::common::{into_chunk_stream, CommonSplitReader}; use crate::source::data_gen_util::spawn_data_generation_stream; use crate::source::datagen::source::SEQUENCE_FIELD_KIND; use crate::source::datagen::{DatagenProperties, DatagenSplit, FieldDesc}; use crate::source::{ - BoxSourceStream, BoxSourceWithStateStream, Column, DataType, SourceContextRef, SplitId, + BoxSourceWithStateStream, Column, DataType, SourceContextRef, SourceMessage, SplitId, SplitImpl, SplitMetaData, SplitReader, }; -impl_common_split_reader_logic!(DatagenSplitReader, DatagenProperties); - pub struct DatagenSplitReader { generator: DatagenEventGenerator, assigned_split: DatagenSplit, @@ -170,13 +167,17 @@ impl SplitReader for DatagenSplitReader { ) .boxed() } - _ => self.into_chunk_stream(), + _ => { + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) + } } } } -impl DatagenSplitReader { - pub(crate) fn into_data_stream(self) -> BoxSourceStream { +impl CommonSplitReader for DatagenSplitReader { + fn into_data_stream(self) -> impl Stream, anyhow::Error>> { // Will buffer at most 4 event chunks. const BUFFER_SIZE: usize = 4; spawn_data_generation_stream(self.generator.into_msg_stream(), BUFFER_SIZE).boxed() diff --git a/src/connector/src/source/google_pubsub/source/reader.rs b/src/connector/src/source/google_pubsub/source/reader.rs index f5d4955ec9b20..d4fa8a9ab5c98 100644 --- a/src/connector/src/source/google_pubsub/source/reader.rs +++ b/src/connector/src/source/google_pubsub/source/reader.rs @@ -15,7 +15,6 @@ use anyhow::{anyhow, ensure, Context, Result}; use async_trait::async_trait; use chrono::{NaiveDateTime, TimeZone, Utc}; -use futures::{StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use google_cloud_pubsub::client::Client; use google_cloud_pubsub::subscription::{SeekTo, Subscription}; @@ -23,8 +22,8 @@ use risingwave_common::bail; use tonic::Code; use super::TaggedReceivedMessage; -use crate::impl_common_split_reader_logic; use crate::parser::ParserConfig; +use crate::source::common::{into_chunk_stream, CommonSplitReader}; use crate::source::google_pubsub::PubsubProperties; use crate::source::{ BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SplitId, SplitImpl, @@ -33,8 +32,6 @@ use crate::source::{ const PUBSUB_MAX_FETCH_MESSAGES: usize = 1024; -impl_common_split_reader_logic!(PubsubSplitReader, PubsubProperties); - pub struct PubsubSplitReader { subscription: Subscription, stop_offset: Option, @@ -44,8 +41,8 @@ pub struct PubsubSplitReader { source_ctx: SourceContextRef, } -impl PubsubSplitReader { - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] +impl CommonSplitReader for PubsubSplitReader { + #[try_stream(ok = Vec, error = anyhow::Error)] async fn into_data_stream(self) { loop { let pull_result = self @@ -172,6 +169,8 @@ impl SplitReader for PubsubSplitReader { } fn into_stream(self) -> BoxSourceWithStateStream { - self.into_chunk_stream() + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) } } diff --git a/src/connector/src/source/kafka/source/reader.rs b/src/connector/src/source/kafka/source/reader.rs index d443429857c21..103255a882ed7 100644 --- a/src/connector/src/source/kafka/source/reader.rs +++ b/src/connector/src/source/kafka/source/reader.rs @@ -19,16 +19,16 @@ use std::time::{Duration, SystemTime, UNIX_EPOCH}; use anyhow::{anyhow, Result}; use async_trait::async_trait; -use futures::{StreamExt, TryStreamExt}; +use futures::StreamExt; use futures_async_stream::try_stream; use rdkafka::config::RDKafkaLogLevel; use rdkafka::consumer::{Consumer, StreamConsumer}; use rdkafka::error::KafkaError; use rdkafka::{ClientConfig, Message, Offset, TopicPartitionList}; -use crate::impl_common_split_reader_logic; use crate::parser::ParserConfig; use crate::source::base::SourceMessage; +use crate::source::common::{into_chunk_stream, CommonSplitReader}; use crate::source::kafka::{ KafkaProperties, KafkaSplit, PrivateLinkConsumerContext, KAFKA_ISOLATION_LEVEL, }; @@ -37,8 +37,6 @@ use crate::source::{ SplitReader, }; -impl_common_split_reader_logic!(KafkaSplitReader, KafkaProperties); - pub struct KafkaSplitReader { consumer: StreamConsumer, offsets: HashMap, Option)>, @@ -159,7 +157,9 @@ impl SplitReader for KafkaSplitReader { } fn into_stream(self) -> BoxSourceWithStateStream { - self.into_chunk_stream() + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) } } @@ -176,9 +176,11 @@ impl KafkaSplitReader { ]) .set(offset); } +} - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] - pub async fn into_data_stream(self) { +impl CommonSplitReader for KafkaSplitReader { + #[try_stream(ok = Vec, error = anyhow::Error)] + async fn into_data_stream(self) { if self.offsets.values().all(|(start_offset, stop_offset)| { match (start_offset, stop_offset) { (Some(start), Some(stop)) if (*start + 1) >= *stop => true, diff --git a/src/connector/src/source/kinesis/source/reader.rs b/src/connector/src/source/kinesis/source/reader.rs index 8d714f3b79334..2939908d4aef5 100644 --- a/src/connector/src/source/kinesis/source/reader.rs +++ b/src/connector/src/source/kinesis/source/reader.rs @@ -20,12 +20,11 @@ use aws_sdk_kinesis::error::{DisplayErrorContext, SdkError}; use aws_sdk_kinesis::operation::get_records::{GetRecordsError, GetRecordsOutput}; use aws_sdk_kinesis::types::ShardIteratorType; use aws_sdk_kinesis::Client as KinesisClient; -use futures::{StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use tokio_retry; -use crate::impl_common_split_reader_logic; use crate::parser::ParserConfig; +use crate::source::common::{into_chunk_stream, CommonSplitReader}; use crate::source::kinesis::source::message::KinesisMessage; use crate::source::kinesis::split::KinesisOffset; use crate::source::kinesis::KinesisProperties; @@ -34,8 +33,6 @@ use crate::source::{ SplitMetaData, SplitReader, }; -impl_common_split_reader_logic!(KinesisSplitReader, KinesisProperties); - #[derive(Debug, Clone)] pub struct KinesisSplitReader { client: KinesisClient, @@ -108,13 +105,15 @@ impl SplitReader for KinesisSplitReader { } fn into_stream(self) -> BoxSourceWithStateStream { - self.into_chunk_stream() + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) } } -impl KinesisSplitReader { - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] - pub(crate) async fn into_data_stream(mut self) { +impl CommonSplitReader for KinesisSplitReader { + #[try_stream(ok = Vec < SourceMessage >, error = anyhow::Error)] + async fn into_data_stream(mut self) { self.new_shard_iter().await?; loop { if self.shard_iter.is_none() { @@ -189,7 +188,8 @@ impl KinesisSplitReader { } } } - +} +impl KinesisSplitReader { async fn new_shard_iter(&mut self) -> Result<()> { let (starting_seq_num, iter_type) = if self.latest_offset.is_some() { ( @@ -269,7 +269,7 @@ impl KinesisSplitReader { #[cfg(test)] mod tests { - use futures::StreamExt; + use futures::{pin_mut, StreamExt}; use super::*; use crate::common::KinesisCommon; @@ -294,7 +294,7 @@ mod tests { seq_offset: None, }; - let mut trim_horizen_reader = KinesisSplitReader::new( + let trim_horizen_reader = KinesisSplitReader::new( properties.clone(), vec![SplitImpl::Kinesis(KinesisSplit { shard_id: "shardId-000000000001".to_string().into(), @@ -307,9 +307,10 @@ mod tests { ) .await? .into_data_stream(); + pin_mut!(trim_horizen_reader); println!("{:?}", trim_horizen_reader.next().await.unwrap()?); - let mut offset_reader = KinesisSplitReader::new( + let offset_reader = KinesisSplitReader::new( properties.clone(), vec![SplitImpl::Kinesis(KinesisSplit { shard_id: "shardId-000000000001".to_string().into(), @@ -324,6 +325,7 @@ mod tests { ) .await? .into_data_stream(); + pin_mut!(offset_reader); println!("{:?}", offset_reader.next().await.unwrap()?); Ok(()) diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index ac10ded25c688..b53fd0f924164 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -28,7 +28,9 @@ pub use base::*; pub use google_pubsub::GOOGLE_PUBSUB_CONNECTOR; pub use kafka::KAFKA_CONNECTOR; pub use kinesis::KINESIS_CONNECTOR; +mod common; mod manager; + pub use manager::SourceColumnDesc; pub use crate::source::nexmark::NEXMARK_CONNECTOR; diff --git a/src/connector/src/source/pulsar/source/reader.rs b/src/connector/src/source/pulsar/source/reader.rs index a8e801541f273..9c749c27d616a 100644 --- a/src/connector/src/source/pulsar/source/reader.rs +++ b/src/connector/src/source/pulsar/source/reader.rs @@ -16,7 +16,7 @@ use std::time::{SystemTime, UNIX_EPOCH}; use anyhow::{anyhow, ensure, Result}; use async_trait::async_trait; -use futures::{StreamExt, TryStreamExt}; +use futures::StreamExt; use futures_async_stream::try_stream; use itertools::Itertools; use pulsar::consumer::InitialPosition; @@ -24,8 +24,8 @@ use pulsar::message::proto::MessageIdData; use pulsar::{Consumer, ConsumerBuilder, ConsumerOptions, Pulsar, SubType, TokioExecutor}; use risingwave_common::try_match_expand; -use crate::impl_common_split_reader_logic; use crate::parser::ParserConfig; +use crate::source::common::{into_chunk_stream, CommonSplitReader}; use crate::source::pulsar::split::PulsarSplit; use crate::source::pulsar::{PulsarEnumeratorOffset, PulsarProperties}; use crate::source::{ @@ -33,8 +33,6 @@ use crate::source::{ SplitMetaData, SplitReader, }; -impl_common_split_reader_logic!(PulsarSplitReader, PulsarProperties); - pub struct PulsarSplitReader { pulsar: Pulsar, consumer: Consumer, TokioExecutor>, @@ -170,13 +168,15 @@ impl SplitReader for PulsarSplitReader { } fn into_stream(self) -> BoxSourceWithStateStream { - self.into_chunk_stream() + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) } } -impl PulsarSplitReader { - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] - pub(crate) async fn into_data_stream(self) { +impl CommonSplitReader for PulsarSplitReader { + #[try_stream(ok = Vec, error = anyhow::Error)] + async fn into_data_stream(self) { let max_chunk_size = self.source_ctx.source_ctrl_opts.chunk_size; #[for_await] for msgs in self.consumer.ready_chunks(max_chunk_size) { From 074a57c884588f356e50d40858f3139b67cf53a8 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 4 Sep 2023 19:10:59 +0800 Subject: [PATCH 3/9] refactor(source): implement the common split reader as a generic function --- src/connector/src/macros.rs | 54 ------------- src/connector/src/source/cdc/source/reader.rs | 14 ++-- src/connector/src/source/common.rs | 76 +++++++++++++++++++ .../src/source/datagen/source/reader.rs | 19 ++--- .../src/source/google_pubsub/source/reader.rs | 13 ++-- .../src/source/kafka/source/reader.rs | 16 ++-- .../src/source/kinesis/source/reader.rs | 26 ++++--- src/connector/src/source/mod.rs | 2 + .../src/source/pulsar/source/reader.rs | 16 ++-- 9 files changed, 132 insertions(+), 104 deletions(-) create mode 100644 src/connector/src/source/common.rs diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index 70fb6130b8717..ea42c63aa6c54 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -180,57 +180,3 @@ macro_rules! impl_connector_properties { } } } - -#[macro_export] -macro_rules! impl_common_split_reader_logic { - ($reader:ty, $props:ty) => { - impl $reader { - #[try_stream(boxed, ok = $crate::source::StreamChunkWithState, error = risingwave_common::error::RwError)] - pub(crate) async fn into_chunk_stream(self) { - let parser_config = self.parser_config.clone(); - let actor_id = self.source_ctx.source_info.actor_id.to_string(); - let source_id = self.source_ctx.source_info.source_id.to_string(); - let metrics = self.source_ctx.metrics.clone(); - let source_ctx = self.source_ctx.clone(); - - let data_stream = self.into_data_stream(); - - let data_stream = data_stream - .inspect_ok(move |data_batch| { - let mut by_split_id = std::collections::HashMap::new(); - - for msg in data_batch { - by_split_id - .entry(msg.split_id.as_ref()) - .or_insert_with(Vec::new) - .push(msg); - } - - for (split_id, msgs) in by_split_id { - metrics - .partition_input_count - .with_label_values(&[&actor_id, &source_id, split_id]) - .inc_by(msgs.len() as u64); - - let sum_bytes = msgs - .iter() - .flat_map(|msg| msg.payload.as_ref().map(|p| p.len() as u64)) - .sum(); - - metrics - .partition_input_bytes - .with_label_values(&[&actor_id, &source_id, &split_id]) - .inc_by(sum_bytes); - } - }).boxed(); - - let parser = - $crate::parser::ByteStreamSourceParserImpl::create(parser_config, source_ctx).await?; - #[for_await] - for msg_batch in parser.into_stream(data_stream) { - yield msg_batch?; - } - } - } - }; -} diff --git a/src/connector/src/source/cdc/source/reader.rs b/src/connector/src/source/cdc/source/reader.rs index 200c91a8a5051..32e0c27ca2856 100644 --- a/src/connector/src/source/cdc/source/reader.rs +++ b/src/connector/src/source/cdc/source/reader.rs @@ -16,22 +16,20 @@ use std::str::FromStr; use anyhow::{anyhow, Result}; use async_trait::async_trait; -use futures::{pin_mut, StreamExt, TryStreamExt}; +use futures::pin_mut; use futures_async_stream::try_stream; use risingwave_common::util::addr::HostAddr; use risingwave_pb::connector_service::GetEventStreamResponse; -use crate::impl_common_split_reader_logic; use crate::parser::ParserConfig; use crate::source::base::SourceMessage; use crate::source::cdc::CdcProperties; +use crate::source::common::{into_chunk_stream, CommonSplitReader}; use crate::source::{ BoxSourceWithStateStream, Column, SourceContextRef, SplitId, SplitImpl, SplitMetaData, SplitReader, }; -impl_common_split_reader_logic!(CdcSplitReader, CdcProperties); - pub struct CdcSplitReader { source_id: u64, start_offset: Option, @@ -90,12 +88,14 @@ impl SplitReader for CdcSplitReader { } fn into_stream(self) -> BoxSourceWithStateStream { - self.into_chunk_stream() + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) } } -impl CdcSplitReader { - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] +impl CommonSplitReader for CdcSplitReader { + #[try_stream(ok = Vec, error = anyhow::Error)] async fn into_data_stream(self) { let cdc_client = self.source_ctx.connector_client.clone().ok_or_else(|| { anyhow!("connector node endpoint not specified or unable to connect to connector node") diff --git a/src/connector/src/source/common.rs b/src/connector/src/source/common.rs new file mode 100644 index 0000000000000..02f1cbde3de38 --- /dev/null +++ b/src/connector/src/source/common.rs @@ -0,0 +1,76 @@ +// 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 futures::{Stream, StreamExt, TryStreamExt}; +use futures_async_stream::try_stream; +use risingwave_common::error::RwError; + +use crate::parser::ParserConfig; +use crate::source::{SourceContextRef, SourceMessage, SplitReader, StreamChunkWithState}; + +pub(crate) trait CommonSplitReader: SplitReader + 'static { + fn into_data_stream( + self, + ) -> impl Stream, anyhow::Error>> + Send; +} + +#[try_stream(boxed, ok = StreamChunkWithState, error = RwError)] +pub(crate) async fn into_chunk_stream( + reader: impl CommonSplitReader + Send, + parser_config: ParserConfig, + source_ctx: SourceContextRef, +) { + let actor_id = source_ctx.source_info.actor_id.to_string(); + let source_id = source_ctx.source_info.source_id.to_string(); + let metrics = source_ctx.metrics.clone(); + + let data_stream = reader.into_data_stream(); + + let data_stream = data_stream + .inspect_ok(move |data_batch| { + let mut by_split_id = std::collections::HashMap::new(); + + for msg in data_batch { + by_split_id + .entry(msg.split_id.as_ref()) + .or_insert_with(Vec::new) + .push(msg); + } + + for (split_id, msgs) in by_split_id { + metrics + .partition_input_count + .with_label_values(&[&actor_id, &source_id, split_id]) + .inc_by(msgs.len() as u64); + + let sum_bytes = msgs + .iter() + .flat_map(|msg| msg.payload.as_ref().map(|p| p.len() as u64)) + .sum(); + + metrics + .partition_input_bytes + .with_label_values(&[&actor_id, &source_id, split_id]) + .inc_by(sum_bytes); + } + }) + .boxed(); + + let parser = + crate::parser::ByteStreamSourceParserImpl::create(parser_config, source_ctx).await?; + #[for_await] + for msg_batch in parser.into_stream(data_stream) { + yield msg_batch?; + } +} diff --git a/src/connector/src/source/datagen/source/reader.rs b/src/connector/src/source/datagen/source/reader.rs index d040a3cb63f21..30d1bfae10c4e 100644 --- a/src/connector/src/source/datagen/source/reader.rs +++ b/src/connector/src/source/datagen/source/reader.rs @@ -16,23 +16,20 @@ use std::collections::HashMap; use anyhow::{anyhow, Result}; use async_trait::async_trait; -use futures::{StreamExt, TryStreamExt}; -use futures_async_stream::try_stream; +use futures::{Stream, StreamExt, TryStreamExt}; use risingwave_common::field_generator::{FieldGeneratorImpl, VarcharProperty}; use super::generator::DatagenEventGenerator; -use crate::impl_common_split_reader_logic; use crate::parser::{EncodingProperties, ParserConfig, ProtocolProperties}; +use crate::source::common::{into_chunk_stream, CommonSplitReader}; use crate::source::data_gen_util::spawn_data_generation_stream; use crate::source::datagen::source::SEQUENCE_FIELD_KIND; use crate::source::datagen::{DatagenProperties, DatagenSplit, FieldDesc}; use crate::source::{ - BoxSourceStream, BoxSourceWithStateStream, Column, DataType, SourceContextRef, SplitId, + BoxSourceWithStateStream, Column, DataType, SourceContextRef, SourceMessage, SplitId, SplitImpl, SplitMetaData, SplitReader, }; -impl_common_split_reader_logic!(DatagenSplitReader, DatagenProperties); - pub struct DatagenSplitReader { generator: DatagenEventGenerator, assigned_split: DatagenSplit, @@ -170,13 +167,17 @@ impl SplitReader for DatagenSplitReader { ) .boxed() } - _ => self.into_chunk_stream(), + _ => { + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) + } } } } -impl DatagenSplitReader { - pub(crate) fn into_data_stream(self) -> BoxSourceStream { +impl CommonSplitReader for DatagenSplitReader { + fn into_data_stream(self) -> impl Stream, anyhow::Error>> { // Will buffer at most 4 event chunks. const BUFFER_SIZE: usize = 4; spawn_data_generation_stream(self.generator.into_msg_stream(), BUFFER_SIZE).boxed() diff --git a/src/connector/src/source/google_pubsub/source/reader.rs b/src/connector/src/source/google_pubsub/source/reader.rs index f5d4955ec9b20..d4fa8a9ab5c98 100644 --- a/src/connector/src/source/google_pubsub/source/reader.rs +++ b/src/connector/src/source/google_pubsub/source/reader.rs @@ -15,7 +15,6 @@ use anyhow::{anyhow, ensure, Context, Result}; use async_trait::async_trait; use chrono::{NaiveDateTime, TimeZone, Utc}; -use futures::{StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use google_cloud_pubsub::client::Client; use google_cloud_pubsub::subscription::{SeekTo, Subscription}; @@ -23,8 +22,8 @@ use risingwave_common::bail; use tonic::Code; use super::TaggedReceivedMessage; -use crate::impl_common_split_reader_logic; use crate::parser::ParserConfig; +use crate::source::common::{into_chunk_stream, CommonSplitReader}; use crate::source::google_pubsub::PubsubProperties; use crate::source::{ BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SplitId, SplitImpl, @@ -33,8 +32,6 @@ use crate::source::{ const PUBSUB_MAX_FETCH_MESSAGES: usize = 1024; -impl_common_split_reader_logic!(PubsubSplitReader, PubsubProperties); - pub struct PubsubSplitReader { subscription: Subscription, stop_offset: Option, @@ -44,8 +41,8 @@ pub struct PubsubSplitReader { source_ctx: SourceContextRef, } -impl PubsubSplitReader { - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] +impl CommonSplitReader for PubsubSplitReader { + #[try_stream(ok = Vec, error = anyhow::Error)] async fn into_data_stream(self) { loop { let pull_result = self @@ -172,6 +169,8 @@ impl SplitReader for PubsubSplitReader { } fn into_stream(self) -> BoxSourceWithStateStream { - self.into_chunk_stream() + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) } } diff --git a/src/connector/src/source/kafka/source/reader.rs b/src/connector/src/source/kafka/source/reader.rs index d443429857c21..103255a882ed7 100644 --- a/src/connector/src/source/kafka/source/reader.rs +++ b/src/connector/src/source/kafka/source/reader.rs @@ -19,16 +19,16 @@ use std::time::{Duration, SystemTime, UNIX_EPOCH}; use anyhow::{anyhow, Result}; use async_trait::async_trait; -use futures::{StreamExt, TryStreamExt}; +use futures::StreamExt; use futures_async_stream::try_stream; use rdkafka::config::RDKafkaLogLevel; use rdkafka::consumer::{Consumer, StreamConsumer}; use rdkafka::error::KafkaError; use rdkafka::{ClientConfig, Message, Offset, TopicPartitionList}; -use crate::impl_common_split_reader_logic; use crate::parser::ParserConfig; use crate::source::base::SourceMessage; +use crate::source::common::{into_chunk_stream, CommonSplitReader}; use crate::source::kafka::{ KafkaProperties, KafkaSplit, PrivateLinkConsumerContext, KAFKA_ISOLATION_LEVEL, }; @@ -37,8 +37,6 @@ use crate::source::{ SplitReader, }; -impl_common_split_reader_logic!(KafkaSplitReader, KafkaProperties); - pub struct KafkaSplitReader { consumer: StreamConsumer, offsets: HashMap, Option)>, @@ -159,7 +157,9 @@ impl SplitReader for KafkaSplitReader { } fn into_stream(self) -> BoxSourceWithStateStream { - self.into_chunk_stream() + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) } } @@ -176,9 +176,11 @@ impl KafkaSplitReader { ]) .set(offset); } +} - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] - pub async fn into_data_stream(self) { +impl CommonSplitReader for KafkaSplitReader { + #[try_stream(ok = Vec, error = anyhow::Error)] + async fn into_data_stream(self) { if self.offsets.values().all(|(start_offset, stop_offset)| { match (start_offset, stop_offset) { (Some(start), Some(stop)) if (*start + 1) >= *stop => true, diff --git a/src/connector/src/source/kinesis/source/reader.rs b/src/connector/src/source/kinesis/source/reader.rs index 8d714f3b79334..2939908d4aef5 100644 --- a/src/connector/src/source/kinesis/source/reader.rs +++ b/src/connector/src/source/kinesis/source/reader.rs @@ -20,12 +20,11 @@ use aws_sdk_kinesis::error::{DisplayErrorContext, SdkError}; use aws_sdk_kinesis::operation::get_records::{GetRecordsError, GetRecordsOutput}; use aws_sdk_kinesis::types::ShardIteratorType; use aws_sdk_kinesis::Client as KinesisClient; -use futures::{StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use tokio_retry; -use crate::impl_common_split_reader_logic; use crate::parser::ParserConfig; +use crate::source::common::{into_chunk_stream, CommonSplitReader}; use crate::source::kinesis::source::message::KinesisMessage; use crate::source::kinesis::split::KinesisOffset; use crate::source::kinesis::KinesisProperties; @@ -34,8 +33,6 @@ use crate::source::{ SplitMetaData, SplitReader, }; -impl_common_split_reader_logic!(KinesisSplitReader, KinesisProperties); - #[derive(Debug, Clone)] pub struct KinesisSplitReader { client: KinesisClient, @@ -108,13 +105,15 @@ impl SplitReader for KinesisSplitReader { } fn into_stream(self) -> BoxSourceWithStateStream { - self.into_chunk_stream() + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) } } -impl KinesisSplitReader { - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] - pub(crate) async fn into_data_stream(mut self) { +impl CommonSplitReader for KinesisSplitReader { + #[try_stream(ok = Vec < SourceMessage >, error = anyhow::Error)] + async fn into_data_stream(mut self) { self.new_shard_iter().await?; loop { if self.shard_iter.is_none() { @@ -189,7 +188,8 @@ impl KinesisSplitReader { } } } - +} +impl KinesisSplitReader { async fn new_shard_iter(&mut self) -> Result<()> { let (starting_seq_num, iter_type) = if self.latest_offset.is_some() { ( @@ -269,7 +269,7 @@ impl KinesisSplitReader { #[cfg(test)] mod tests { - use futures::StreamExt; + use futures::{pin_mut, StreamExt}; use super::*; use crate::common::KinesisCommon; @@ -294,7 +294,7 @@ mod tests { seq_offset: None, }; - let mut trim_horizen_reader = KinesisSplitReader::new( + let trim_horizen_reader = KinesisSplitReader::new( properties.clone(), vec![SplitImpl::Kinesis(KinesisSplit { shard_id: "shardId-000000000001".to_string().into(), @@ -307,9 +307,10 @@ mod tests { ) .await? .into_data_stream(); + pin_mut!(trim_horizen_reader); println!("{:?}", trim_horizen_reader.next().await.unwrap()?); - let mut offset_reader = KinesisSplitReader::new( + let offset_reader = KinesisSplitReader::new( properties.clone(), vec![SplitImpl::Kinesis(KinesisSplit { shard_id: "shardId-000000000001".to_string().into(), @@ -324,6 +325,7 @@ mod tests { ) .await? .into_data_stream(); + pin_mut!(offset_reader); println!("{:?}", offset_reader.next().await.unwrap()?); Ok(()) diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index ac10ded25c688..b53fd0f924164 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -28,7 +28,9 @@ pub use base::*; pub use google_pubsub::GOOGLE_PUBSUB_CONNECTOR; pub use kafka::KAFKA_CONNECTOR; pub use kinesis::KINESIS_CONNECTOR; +mod common; mod manager; + pub use manager::SourceColumnDesc; pub use crate::source::nexmark::NEXMARK_CONNECTOR; diff --git a/src/connector/src/source/pulsar/source/reader.rs b/src/connector/src/source/pulsar/source/reader.rs index a8e801541f273..9c749c27d616a 100644 --- a/src/connector/src/source/pulsar/source/reader.rs +++ b/src/connector/src/source/pulsar/source/reader.rs @@ -16,7 +16,7 @@ use std::time::{SystemTime, UNIX_EPOCH}; use anyhow::{anyhow, ensure, Result}; use async_trait::async_trait; -use futures::{StreamExt, TryStreamExt}; +use futures::StreamExt; use futures_async_stream::try_stream; use itertools::Itertools; use pulsar::consumer::InitialPosition; @@ -24,8 +24,8 @@ use pulsar::message::proto::MessageIdData; use pulsar::{Consumer, ConsumerBuilder, ConsumerOptions, Pulsar, SubType, TokioExecutor}; use risingwave_common::try_match_expand; -use crate::impl_common_split_reader_logic; use crate::parser::ParserConfig; +use crate::source::common::{into_chunk_stream, CommonSplitReader}; use crate::source::pulsar::split::PulsarSplit; use crate::source::pulsar::{PulsarEnumeratorOffset, PulsarProperties}; use crate::source::{ @@ -33,8 +33,6 @@ use crate::source::{ SplitMetaData, SplitReader, }; -impl_common_split_reader_logic!(PulsarSplitReader, PulsarProperties); - pub struct PulsarSplitReader { pulsar: Pulsar, consumer: Consumer, TokioExecutor>, @@ -170,13 +168,15 @@ impl SplitReader for PulsarSplitReader { } fn into_stream(self) -> BoxSourceWithStateStream { - self.into_chunk_stream() + let parser_config = self.parser_config.clone(); + let source_context = self.source_ctx.clone(); + into_chunk_stream(self, parser_config, source_context) } } -impl PulsarSplitReader { - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] - pub(crate) async fn into_data_stream(self) { +impl CommonSplitReader for PulsarSplitReader { + #[try_stream(ok = Vec, error = anyhow::Error)] + async fn into_data_stream(self) { let max_chunk_size = self.source_ctx.source_ctrl_opts.chunk_size; #[for_await] for msgs in self.consumer.ready_chunks(max_chunk_size) { From 8cdc90ebd99d4faa8938f2a658628694f02a5fd6 Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 6 Sep 2023 01:48:29 +0800 Subject: [PATCH 4/9] extract more common cdc logic --- src/connector/src/macros.rs | 89 ++++++++++++++++++- src/connector/src/source/base.rs | 64 ++----------- src/connector/src/source/cdc/mod.rs | 53 ++--------- .../src/executor/backfill/cdc_backfill.rs | 2 +- 4 files changed, 104 insertions(+), 104 deletions(-) diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index 4291978e18117..a30ad58ce07ce 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -180,7 +180,8 @@ macro_rules! impl_connector_properties { pub fn extract(mut props: HashMap) -> Result { const UPSTREAM_SOURCE_KEY: &str = "connector"; let connector = props.remove(UPSTREAM_SOURCE_KEY).ok_or_else(|| anyhow!("Must specify 'connector' in WITH clause"))?; - if connector.ends_with("-cdc") { + use $crate::source::cdc::CDC_CONNECTOR_NAME_SUFFIX; + if connector.ends_with(CDC_CONNECTOR_NAME_SUFFIX) { ConnectorProperties::new_cdc_properties(&connector, props) } else { let json_value = serde_json::to_value(props).map_err(|e| anyhow!(e))?; @@ -199,3 +200,89 @@ macro_rules! impl_connector_properties { } } } + +#[macro_export] +macro_rules! impl_cdc_source_type { + ($({$source_type:ident, $name:expr }),*) => { + $( + paste!{ + #[derive(Clone, Debug, Default, PartialEq, Eq, Hash)] + pub struct $source_type; + impl CdcSourceTypeTrait for $source_type { + const CDC_CONNECTOR_NAME: &'static str = concat!($name, "-cdc"); + fn source_type() -> CdcSourceType { + CdcSourceType::$source_type + } + } + + pub type [< $source_type DebeziumSplitEnumerator >] = DebeziumSplitEnumerator<$source_type>; + } + )* + + pub enum CdcSourceType { + $( + $source_type, + )* + } + + impl From for CdcSourceType { + fn from(value: PbSourceType) -> Self { + match value { + PbSourceType::Unspecified => unreachable!(), + $( + PbSourceType::$source_type => CdcSourceType::$source_type, + )* + } + } + } + + impl From for PbSourceType { + fn from(this: CdcSourceType) -> PbSourceType { + match this { + $( + CdcSourceType::$source_type => PbSourceType::$source_type, + )* + } + } + } + + impl ConnectorProperties { + pub(crate) fn new_cdc_properties( + connector_name: &str, + properties: HashMap, + ) -> std::result::Result { + match connector_name { + $( + $source_type::CDC_CONNECTOR_NAME => paste! { + Ok(Self::[< $source_type Cdc >](Box::new(CdcProperties::<$source_type> { + props: properties, + ..Default::default() + }))) + }, + )* + _ => Err(anyhow::anyhow!("unexpected cdc connector '{}'", connector_name,)), + } + } + + pub fn init_cdc_properties(&mut self, table_schema: PbTableSchema) { + match self { + $( + paste! {ConnectorProperties:: [< $source_type Cdc >](c)} => { + c.table_schema = table_schema; + } + )* + _ => {} + } + } + + pub fn is_cdc_connector(&self) -> bool { + match self { + $( + paste! {ConnectorProperties:: [< $source_type Cdc >](_)} => true, + )* + _ => false, + } + } + } + } +} diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index cb39c8dcc9b41..5a94b36a6b8e8 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -27,7 +27,6 @@ use risingwave_common::array::StreamChunk; use risingwave_common::catalog::TableId; use risingwave_common::error::{ErrorSuppressor, RwError}; use risingwave_common::types::{JsonbVal, Scalar}; -use risingwave_pb::connector_service::PbTableSchema; use risingwave_pb::source::ConnectorSplit; use risingwave_rpc_client::ConnectorClient; @@ -299,7 +298,7 @@ pub enum ConnectorProperties { Nexmark(Box), Datagen(Box), S3(Box), - MySqlCdc(Box>), + MysqlCdc(Box>), PostgresCdc(Box>), CitusCdc(Box>), GooglePubsub(Box), @@ -308,51 +307,6 @@ pub enum ConnectorProperties { } impl ConnectorProperties { - fn new_cdc_properties( - connector_name: &str, - properties: HashMap, - ) -> Result { - match connector_name { - MYSQL_CDC_CONNECTOR => Ok(Self::MySqlCdc(Box::new(CdcProperties:: { - props: properties, - ..Default::default() - }))), - POSTGRES_CDC_CONNECTOR => Ok(Self::PostgresCdc(Box::new(CdcProperties:: { - props: properties, - ..Default::default() - }))), - CITUS_CDC_CONNECTOR => Ok(Self::CitusCdc(Box::new(CdcProperties:: { - props: properties, - ..Default::default() - }))), - _ => Err(anyhow!("unexpected cdc connector '{}'", connector_name,)), - } - } - - pub fn init_cdc_properties(&mut self, table_schema: PbTableSchema) { - match self { - ConnectorProperties::MySqlCdc(c) => { - c.table_schema = table_schema; - } - ConnectorProperties::PostgresCdc(c) => { - c.table_schema = table_schema; - } - ConnectorProperties::CitusCdc(c) => { - c.table_schema = table_schema; - } - _ => {} - } - } - - pub fn is_cdc_connector(&self) -> bool { - matches!( - self, - ConnectorProperties::MySqlCdc(_) - | ConnectorProperties::PostgresCdc(_) - | ConnectorProperties::CitusCdc(_) - ) - } - pub fn support_multiple_splits(&self) -> bool { matches!(self, ConnectorProperties::Kafka(_)) } @@ -366,7 +320,7 @@ pub enum SplitImpl { Nexmark(NexmarkSplit), Datagen(DatagenSplit), GooglePubsub(PubsubSplit), - MySqlCdc(DebeziumCdcSplit), + MysqlCdc(DebeziumCdcSplit), PostgresCdc(DebeziumCdcSplit), CitusCdc(DebeziumCdcSplit), Nats(NatsSplit), @@ -399,7 +353,7 @@ pub enum SplitReaderImpl { Nexmark(Box), Pulsar(Box), Datagen(Box), - MySqlCdc(Box>), + MysqlCdc(Box>), PostgresCdc(Box>), CitusCdc(Box>), GooglePubsub(Box), @@ -412,7 +366,7 @@ pub enum SplitEnumeratorImpl { Kinesis(KinesisSplitEnumerator), Nexmark(NexmarkSplitEnumerator), Datagen(DatagenSplitEnumerator), - MySqlCdc(MysqlDebeziumSplitEnumerator), + MysqlCdc(MysqlDebeziumSplitEnumerator), PostgresCdc(PostgresDebeziumSplitEnumerator), CitusCdc(CitusDebeziumSplitEnumerator), GooglePubsub(PubsubSplitEnumerator), @@ -437,7 +391,7 @@ impl_split_enumerator! { { Kinesis, KinesisSplitEnumerator }, { Nexmark, NexmarkSplitEnumerator }, { Datagen, DatagenSplitEnumerator }, - { MySqlCdc, DebeziumSplitEnumerator }, + { MysqlCdc, DebeziumSplitEnumerator }, { PostgresCdc, DebeziumSplitEnumerator }, { CitusCdc, DebeziumSplitEnumerator }, { GooglePubsub, PubsubSplitEnumerator}, @@ -452,7 +406,7 @@ impl_split! { { Nexmark, NEXMARK_CONNECTOR, NexmarkSplit }, { Datagen, DATAGEN_CONNECTOR, DatagenSplit }, { GooglePubsub, GOOGLE_PUBSUB_CONNECTOR, PubsubSplit }, - { MySqlCdc, MYSQL_CDC_CONNECTOR, DebeziumCdcSplit }, + { MysqlCdc, MYSQL_CDC_CONNECTOR, DebeziumCdcSplit }, { PostgresCdc, POSTGRES_CDC_CONNECTOR, DebeziumCdcSplit }, { CitusCdc, CITUS_CDC_CONNECTOR, DebeziumCdcSplit }, { S3, S3_CONNECTOR, FsSplit }, @@ -466,7 +420,7 @@ impl_split_reader! { { Kinesis, KinesisSplitReader }, { Nexmark, NexmarkSplitReader }, { Datagen, DatagenSplitReader }, - { MySqlCdc, CdcSplitReader}, + { MysqlCdc, CdcSplitReader}, { PostgresCdc, CdcSplitReader}, { CitusCdc, CdcSplitReader }, { GooglePubsub, PubsubSplitReader }, @@ -565,7 +519,7 @@ mod tests { let offset_str = "{\"sourcePartition\":{\"server\":\"RW_CDC_mydb.products\"},\"sourceOffset\":{\"transaction_id\":null,\"ts_sec\":1670407377,\"file\":\"binlog.000001\",\"pos\":98587,\"row\":2,\"server_id\":1,\"event\":2}}"; let mysql_split = MySqlCdcSplit::new(1001, offset_str.to_string()); let split = DebeziumCdcSplit::new(Some(mysql_split), None); - let split_impl = SplitImpl::MySqlCdc(split); + let split_impl = SplitImpl::MysqlCdc(split); let encoded_split = split_impl.encode_to_bytes(); let restored_split_impl = SplitImpl::restore_from_bytes(encoded_split.as_ref())?; assert_eq!( @@ -653,7 +607,7 @@ mod tests { )); let conn_props = ConnectorProperties::extract(user_props_mysql).unwrap(); - if let ConnectorProperties::MySqlCdc(c) = conn_props { + if let ConnectorProperties::MysqlCdc(c) = conn_props { assert_eq!(c.props.get("connector_node_addr").unwrap(), "localhost"); assert_eq!(c.props.get("database.hostname").unwrap(), "127.0.0.1"); assert_eq!(c.props.get("database.port").unwrap(), "3306"); diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index 0205ecfd9147b..86a8b16adec02 100644 --- a/src/connector/src/source/cdc/mod.rs +++ b/src/connector/src/source/cdc/mod.rs @@ -21,10 +21,15 @@ use std::marker::PhantomData; pub use enumerator::*; use paste::paste; use risingwave_common::catalog::{ColumnDesc, Field, Schema}; -use risingwave_pb::connector_service::{PbSourceType, SourceType, TableSchema}; +use risingwave_pb::connector_service::{PbSourceType, PbTableSchema, SourceType, TableSchema}; pub use source::*; pub use split::*; +use crate::impl_cdc_source_type; +use crate::source::ConnectorProperties; + +pub const CDC_CONNECTOR_NAME_SUFFIX: &str = "-cdc"; + pub const MYSQL_CDC_CONNECTOR: &str = Mysql::CDC_CONNECTOR_NAME; pub const POSTGRES_CDC_CONNECTOR: &str = Postgres::CDC_CONNECTOR_NAME; pub const CITUS_CDC_CONNECTOR: &str = Citus::CDC_CONNECTOR_NAME; @@ -34,52 +39,6 @@ pub trait CdcSourceTypeTrait: Send + Sync + Clone + 'static { fn source_type() -> CdcSourceType; } -macro_rules! impl_cdc_source_type { - ($({$source_type:ident, $name:expr }),*) => { - $( - paste!{ - #[derive(Clone, Debug, Default, PartialEq, Eq, Hash)] - pub struct $source_type; - impl CdcSourceTypeTrait for $source_type { - const CDC_CONNECTOR_NAME: &'static str = concat!($name, "-cdc"); - fn source_type() -> CdcSourceType { - CdcSourceType::$source_type - } - } - - pub type [< $source_type DebeziumSplitEnumerator >] = DebeziumSplitEnumerator<$source_type>; - } - )* - - pub enum CdcSourceType { - $( - $source_type, - )* - } - - impl From for CdcSourceType { - fn from(value: PbSourceType) -> Self { - match value { - PbSourceType::Unspecified => unreachable!(), - $( - PbSourceType::$source_type => CdcSourceType::$source_type, - )* - } - } - } - - impl From for PbSourceType { - fn from(this: CdcSourceType) -> PbSourceType { - match this { - $( - CdcSourceType::$source_type => PbSourceType::$source_type, - )* - } - } - } - } -} - impl_cdc_source_type!({ Mysql, "mysql" }, { Postgres, "postgres" }, { Citus, "citus" }); #[derive(Clone, Debug, Default)] diff --git a/src/stream/src/executor/backfill/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc_backfill.rs index 55742348bbf03..2f522ae8eeb0c 100644 --- a/src/stream/src/executor/backfill/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc_backfill.rs @@ -492,7 +492,7 @@ impl CdcBackfillExecutor { .set(key.into(), JsonbVal::from(Value::Bool(true))) .await?; - if let Some(SplitImpl::MySqlCdc(split)) = cdc_split.as_mut() + if let Some(SplitImpl::MysqlCdc(split)) = cdc_split.as_mut() && let Some(s) = split.mysql_split.as_mut() { let start_offset = last_binlog_offset.as_ref().map(|cdc_offset| { From bdb20b4ffce0e9d5087a25810ce17bc65d20be34 Mon Sep 17 00:00:00 2001 From: William Wen Date: Fri, 8 Sep 2023 18:01:18 +0800 Subject: [PATCH 5/9] make code compile --- .../src/source/nats/source/reader.rs | 4 ---- src/meta/src/stream/source_manager.rs | 19 ++++++------------- 2 files changed, 6 insertions(+), 17 deletions(-) diff --git a/src/connector/src/source/nats/source/reader.rs b/src/connector/src/source/nats/source/reader.rs index eb090047d1798..d958b5a898495 100644 --- a/src/connector/src/source/nats/source/reader.rs +++ b/src/connector/src/source/nats/source/reader.rs @@ -47,10 +47,6 @@ impl SplitReader for NatsSplitReader { ) -> Result { // TODO: to simplify the logic, return 1 split for first version assert!(splits.len() == 1); - let splits = splits - .into_iter() - .map(|split| split.into_nats().unwrap()) - .collect::>(); let consumer = properties .common .build_consumer(0, splits[0].start_sequence) diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index 501f41b0fba59..1c0bd9f0e5a0b 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -193,25 +193,18 @@ impl ConnectorSourceWorker

{ .with_label_values(&[self.source_id.to_string().as_str(), &self.source_name]) .set(res); }; - let splits = self - .enumerator - .list_splits() - .await - .map_err(|e| { - source_is_up(0); - self.fail_cnt += 1; - e - })? - .into_iter() - .map(P::Split::into) - .collect_vec(); + let splits = self.enumerator.list_splits().await.map_err(|e| { + source_is_up(0); + self.fail_cnt += 1; + e + })?; source_is_up(1); self.fail_cnt = 0; let mut current_splits = self.current_splits.lock().await; current_splits.splits.replace( splits .into_iter() - .map(|split| (split.id(), split)) + .map(|split| (split.id(), P::Split::into(split))) .collect(), ); From 73ae14cded102a4fcd53fdc64dae0f69e62ba076 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 11 Sep 2023 13:18:11 +0800 Subject: [PATCH 6/9] refactor(source): refine source macro --- src/connector/src/lib.rs | 2 + src/connector/src/macros.rs | 318 +++++++++++++++------------- src/connector/src/source/base.rs | 216 +++++++++++-------- src/connector/src/source/cdc/mod.rs | 22 +- 4 files changed, 326 insertions(+), 232 deletions(-) diff --git a/src/connector/src/lib.rs b/src/connector/src/lib.rs index 75a895a5f80cd..2894df85db534 100644 --- a/src/connector/src/lib.rs +++ b/src/connector/src/lib.rs @@ -49,6 +49,8 @@ pub mod source; pub mod common; +pub use paste::paste; + #[derive(Clone, Debug, Default)] pub struct ConnectorParams { pub connector_client: Option, diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index f18bb18462749..792e2066abcca 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -13,152 +13,225 @@ // limitations under the License. #[macro_export] -macro_rules! impl_split { - ($({ $variant_name:ident, $connector_name:ident, $split:ty} ),*) => { - impl From<&SplitImpl> for ConnectorSplit { - fn from(split: &SplitImpl) -> Self { - match split { - $( SplitImpl::$variant_name(inner) => ConnectorSplit { split_type: String::from($connector_name), encoded_split: inner.encode_to_bytes().to_vec() }, )* - } +macro_rules! for_all_classified_sources { + ($macro:path $(,$extra_args:tt)*) => { + $macro! { + // cdc sources + { + { Mysql }, + { Postgres }, + { Citus } + }, + // other sources + { + { Kafka, $crate::source::kafka::KafkaProperties, $crate::source::kafka::KafkaSplit }, + { Pulsar, $crate::source::pulsar::PulsarProperties, $crate::source::pulsar::PulsarSplit }, + { Kinesis, $crate::source::kinesis::KinesisProperties, $crate::source::kinesis::split::KinesisSplit }, + { Nexmark, $crate::source::nexmark::NexmarkProperties, $crate::source::nexmark::NexmarkSplit }, + { Datagen, $crate::source::datagen::DatagenProperties, $crate::source::datagen::DatagenSplit }, + { GooglePubsub, $crate::source::google_pubsub::PubsubProperties, $crate::source::google_pubsub::PubsubSplit }, + { Nats, $crate::source::nats::NatsProperties, $crate::source::nats::split::NatsSplit }, + { S3, $crate::source::filesystem::S3Properties, $crate::source::filesystem::FsSplit } } + $( + ,$extra_args + )* } - $( - impl TryFrom for $split { - type Error = anyhow::Error; + }; +} - fn try_from(split: SplitImpl) -> std::result::Result { - match split { - SplitImpl::$variant_name(inner) => Ok(inner), - other => Err(anyhow::anyhow!("expect {} but get {:?}", stringify!($split), other)) - } +#[macro_export] +macro_rules! for_all_sources_inner { + ( + {$({ $cdc_source_type:ident }),* }, + { $({ $source_variant:ident, $prop_name:ty, $split:ty }),* }, + $macro:tt $(, $extra_args:tt)* + ) => { + $crate::paste! { + $macro! { + { + $( + { + [< $cdc_source_type Cdc >], + $crate::source::cdc::[< $cdc_source_type CdcProperties >], + $crate::source::cdc::DebeziumCdcSplit<$crate::source::cdc::$cdc_source_type> + }, + )* + $( + { $source_variant, $prop_name, $split } + ),* } + $(,$extra_args)* } + } + }; +} - impl From<$split> for SplitImpl { - fn from(split: $split) -> SplitImpl { - SplitImpl::$variant_name(split) - } - } +#[macro_export] +macro_rules! for_all_sources { + ($macro:path $(, $arg:tt )*) => { + $crate::for_all_classified_sources! {$crate::for_all_sources_inner, $macro $(,$arg)* } + }; +} - )* +#[macro_export] +macro_rules! dispatch_source_enum_inner { + ( + {$({$source_variant:ident, $prop_name:ty, $split:ty }),*}, + $enum_name:ident, + $impl:tt, + {$inner_name:ident, $prop_type_name:ident, $split_type_name:ident}, + $body:expr + ) => {{ + match $impl { + $( + $enum_name::$source_variant($inner_name) => { + type $prop_type_name = $prop_name; + type $split_type_name = $split; + { + $body + } + }, + )* + } + }} +} - impl TryFrom<&ConnectorSplit> for SplitImpl { - type Error = anyhow::Error; +#[macro_export] +macro_rules! dispatch_source_enum { + ($enum_name:ident, $impl:expr, $inner_name:tt, $body:expr) => {{ + $crate::for_all_sources! {$crate::dispatch_source_enum_inner, $enum_name, { $impl }, $inner_name, $body} + }}; +} - fn try_from(split: &ConnectorSplit) -> std::result::Result { - match split.split_type.to_lowercase().as_str() { - $( $connector_name => <$split>::restore_from_bytes(split.encoded_split.as_ref()).map(SplitImpl::$variant_name), )* - other => { - Err(anyhow!("connector '{}' is not supported", other)) +#[macro_export] +macro_rules! match_source_name_str_inner { + ( + {$({$source_variant:ident, $prop_name:ty, $split:ty }),*}, + $source_name_str:expr, + $prop_type_name:ident, + $body:expr, + $on_other_closure:expr + ) => {{ + match $source_name_str { + $( + <$prop_name>::SOURCE_NAME => { + type $prop_type_name = $prop_name; + { + $body } - } - } + }, + )* + other => ($on_other_closure)(other), } + }} +} - impl SplitMetaData for SplitImpl { - fn id(&self) -> SplitId { - match self { - $( Self::$variant_name(inner) => inner.id(), )* - } - } +#[macro_export] +macro_rules! match_source_name_str { + ($source_name_str:expr, $prop_type_name:ident, $body:expr, $on_other_closure:expr) => {{ + $crate::for_all_sources! { + $crate::match_source_name_str_inner, + { $source_name_str }, + $prop_type_name, + { $body }, + { $on_other_closure } + } + }}; +} - fn encode_to_json(&self) -> JsonbVal { - use serde_json::json; - let inner = self.encode_to_json_inner().take(); - json!({ SPLIT_TYPE_FIELD: self.get_type(), SPLIT_INFO_FIELD: inner}).into() - } +#[macro_export] +macro_rules! dispatch_split_impl { + ($impl:expr, $inner_name:ident, $prop_type_name:ident, $body:expr) => {{ + use $crate::source::SplitImpl; + $crate::dispatch_source_enum! {SplitImpl, { $impl }, {$inner_name, $prop_type_name, IgnoreSplitType}, $body} + }}; +} - fn restore_from_json(value: JsonbVal) -> Result { - let mut value = value.take(); - let json_obj = value.as_object_mut().unwrap(); - let split_type = json_obj.remove(SPLIT_TYPE_FIELD).unwrap().as_str().unwrap().to_string(); - let inner_value = json_obj.remove(SPLIT_INFO_FIELD).unwrap(); - Self::restore_from_json_inner(&split_type, inner_value.into()) - } +#[macro_export] +macro_rules! impl_split { + ({$({ $variant_name:ident, $prop_name:ty, $split:ty}),*}) => { + + #[derive(Debug, Clone, EnumAsInner, PartialEq, Hash)] + pub enum SplitImpl { + $( + $variant_name($split), + )* } - impl SplitImpl { - pub fn get_type(&self) -> String { - match self { - $( Self::$variant_name(_) => $connector_name, )* - } - .to_string() - } + $( + impl TryFrom for $split { + type Error = anyhow::Error; - pub fn update_in_place(&mut self, start_offset: String) -> anyhow::Result<()> { - match self { - $( Self::$variant_name(inner) => inner.update_with_offset(start_offset)?, )* + fn try_from(split: SplitImpl) -> std::result::Result { + match split { + SplitImpl::$variant_name(inner) => Ok(inner), + other => Err(anyhow::anyhow!("expect {} but get {:?}", stringify!($split), other)) + } } - Ok(()) } - pub fn encode_to_json_inner(&self) -> JsonbVal { - match self { - $( Self::$variant_name(inner) => inner.encode_to_json(), )* + impl From<$split> for SplitImpl { + fn from(split: $split) -> SplitImpl { + SplitImpl::$variant_name(split) } } - fn restore_from_json_inner(split_type: &str, value: JsonbVal) -> Result { - match split_type.to_lowercase().as_str() { - $( $connector_name => <$split>::restore_from_json(value).map(SplitImpl::$variant_name), )* - other => { - Err(anyhow!("connector '{}' is not supported", other)) - } - } - } - } + )* } } +#[macro_export] +macro_rules! dispatch_source_prop { + ($impl:expr, $source_prop:tt, $body:expr) => {{ + use $crate::source::ConnectorProperties; + $crate::dispatch_source_enum! {ConnectorProperties, { $impl }, {$source_prop, IgnorePropType, IgnoreSplitType}, {$body}} + }}; +} + #[macro_export] macro_rules! impl_connector_properties { - ($({ $variant_name:ident, $connector_name:ident } ),*) => { - impl ConnectorProperties { - pub fn extract(mut props: HashMap) -> Result { - const UPSTREAM_SOURCE_KEY: &str = "connector"; - let connector = props.remove(UPSTREAM_SOURCE_KEY).ok_or_else(|| anyhow!("Must specify 'connector' in WITH clause"))?; - use $crate::source::cdc::CDC_CONNECTOR_NAME_SUFFIX; - if connector.ends_with(CDC_CONNECTOR_NAME_SUFFIX) { - ConnectorProperties::new_cdc_properties(&connector, props) - } else { - let json_value = serde_json::to_value(props).map_err(|e| anyhow!(e))?; - match connector.to_lowercase().as_str() { - $( - $connector_name => { - serde_json::from_value(json_value).map_err(|e| anyhow!(e.to_string())).map(Self::$variant_name) - }, - )* - _ => { - Err(anyhow!("connector '{}' is not supported", connector,)) - } - } + ({$({ $variant_name:ident, $prop_name:ty, $split:ty}),*}) => { + #[derive(Clone, Debug)] + pub enum ConnectorProperties { + $( + $variant_name(Box<$prop_name>), + )* + } + + $( + impl From<$prop_name> for ConnectorProperties { + fn from(prop: $prop_name) -> ConnectorProperties { + ConnectorProperties::$variant_name(Box::new(prop)) } } - } + )* } } #[macro_export] macro_rules! impl_cdc_source_type { - ($({$source_type:ident, $name:expr }),*) => { + ( + {$({$cdc_source_type:tt}),*}, + {$($_ignore:tt),*} + ) => { $( - paste!{ + $crate::paste!{ #[derive(Clone, Debug, Default, PartialEq, Eq, Hash)] - pub struct $source_type; - impl CdcSourceTypeTrait for $source_type { - const CDC_CONNECTOR_NAME: &'static str = concat!($name, "-cdc"); + pub struct $cdc_source_type; + impl CdcSourceTypeTrait for $cdc_source_type { + const CDC_CONNECTOR_NAME: &'static str = concat!(stringify!([<$cdc_source_type:lower>]), "-cdc"); fn source_type() -> CdcSourceType { - CdcSourceType::$source_type + CdcSourceType::$cdc_source_type } } - - pub type [< $source_type DebeziumSplitEnumerator >] = DebeziumSplitEnumerator<$source_type>; + pub type [<$cdc_source_type CdcProperties>] = CdcProperties<$cdc_source_type>; } )* pub enum CdcSourceType { $( - $source_type, + $cdc_source_type, )* } @@ -167,7 +240,7 @@ macro_rules! impl_cdc_source_type { match value { PbSourceType::Unspecified => unreachable!(), $( - PbSourceType::$source_type => CdcSourceType::$source_type, + PbSourceType::$cdc_source_type => CdcSourceType::$cdc_source_type, )* } } @@ -177,47 +250,8 @@ macro_rules! impl_cdc_source_type { fn from(this: CdcSourceType) -> PbSourceType { match this { $( - CdcSourceType::$source_type => PbSourceType::$source_type, - )* - } - } - } - - impl ConnectorProperties { - pub(crate) fn new_cdc_properties( - connector_name: &str, - properties: HashMap, - ) -> std::result::Result { - match connector_name { - $( - $source_type::CDC_CONNECTOR_NAME => paste! { - Ok(Self::[< $source_type Cdc >](Box::new(CdcProperties::<$source_type> { - props: properties, - ..Default::default() - }))) - }, - )* - _ => Err(anyhow::anyhow!("unexpected cdc connector '{}'", connector_name,)), - } - } - - pub fn init_cdc_properties(&mut self, table_schema: PbTableSchema) { - match self { - $( - paste! {ConnectorProperties:: [< $source_type Cdc >](c)} => { - c.table_schema = table_schema; - } - )* - _ => {} - } - } - - pub fn is_cdc_connector(&self) -> bool { - match self { - $( - paste! {ConnectorProperties:: [< $source_type Cdc >](_)} => true, + CdcSourceType::$cdc_source_type => PbSourceType::$cdc_source_type, )* - _ => false, } } } diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 2e87243d40af1..c6e565d20fa90 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::collections::HashMap; +use std::ops::Deref; use std::sync::Arc; use anyhow::{anyhow, Result}; @@ -27,41 +28,47 @@ use risingwave_common::array::StreamChunk; use risingwave_common::catalog::TableId; use risingwave_common::error::{ErrorSuppressor, RwError}; use risingwave_common::types::{JsonbVal, Scalar}; +use risingwave_pb::connector_service::PbTableSchema; use risingwave_pb::source::ConnectorSplit; use risingwave_rpc_client::ConnectorClient; +use serde::de::DeserializeOwned; use super::datagen::DatagenMeta; -use super::filesystem::{FsSplit, S3Properties, S3_CONNECTOR}; +use super::filesystem::FsSplit; use super::google_pubsub::GooglePubsubMeta; use super::kafka::KafkaMeta; use super::monitor::SourceMetrics; use super::nexmark::source::message::NexmarkMeta; use crate::parser::ParserConfig; -use crate::source::cdc::{ - CdcProperties, Citus, DebeziumCdcSplit, Mysql, Postgres, CITUS_CDC_CONNECTOR, - MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, -}; pub(crate) use crate::source::common::CommonSplitReader; -use crate::source::datagen::{DatagenProperties, DatagenSplit, DATAGEN_CONNECTOR}; -use crate::source::google_pubsub::{PubsubProperties, PubsubSplit, GOOGLE_PUBSUB_CONNECTOR}; -use crate::source::kafka::{KafkaProperties, KafkaSplit, KAFKA_CONNECTOR}; -use crate::source::kinesis::split::KinesisSplit; -use crate::source::kinesis::{KinesisProperties, KINESIS_CONNECTOR}; use crate::source::monitor::EnumeratorMetrics; -use crate::source::nats::source::NatsSplit; -use crate::source::nats::{NatsProperties, NATS_CONNECTOR}; -use crate::source::nexmark::{NexmarkProperties, NexmarkSplit, NEXMARK_CONNECTOR}; -use crate::source::pulsar::{PulsarProperties, PulsarSplit, PULSAR_CONNECTOR}; -use crate::{impl_connector_properties, impl_split}; +use crate::{ + dispatch_source_prop, dispatch_split_impl, for_all_sources, impl_connector_properties, + impl_split, match_source_name_str, +}; const SPLIT_TYPE_FIELD: &str = "split_type"; const SPLIT_INFO_FIELD: &str = "split_info"; -pub trait SourceProperties: Clone { +pub trait TryFromHashmap: Sized { + fn try_from_hashmap(props: HashMap) -> Result; +} + +pub trait SourceProperties: TryFromHashmap + Clone { const SOURCE_NAME: &'static str; + const IS_CDC_SOURCE: bool = false; type Split: SplitMetaData + TryFrom + Into; type SplitEnumerator: SplitEnumerator; type SplitReader: SplitReader; + + fn set_schema(&mut self, _table_schema: PbTableSchema) {} +} + +impl TryFromHashmap for P { + fn try_from_hashmap(props: HashMap) -> Result { + let json_value = serde_json::to_value(props).map_err(|e| anyhow!(e))?; + serde_json::from_value::

(json_value).map_err(|e| anyhow!(e.to_string())) + } } pub async fn create_split_reader( @@ -289,61 +296,67 @@ pub trait SplitReader: Sized + Send { fn into_stream(self) -> BoxSourceWithStateStream; } -#[derive(Clone, Debug)] -pub enum ConnectorProperties { - Kafka(Box), - Pulsar(Box), - Kinesis(Box), - Nexmark(Box), - Datagen(Box), - S3(Box), - MysqlCdc(Box>), - PostgresCdc(Box>), - CitusCdc(Box>), - GooglePubsub(Box), - Nats(Box), -} - -#[macro_export] -macro_rules! dispatch_source_prop { - ($impl:expr, $source_prop:ident, $body:tt) => {{ - use $crate::source::base::ConnectorProperties; - - match $impl { - ConnectorProperties::Kafka($source_prop) => $body, - ConnectorProperties::Pulsar($source_prop) => $body, - ConnectorProperties::Kinesis($source_prop) => $body, - ConnectorProperties::Nexmark($source_prop) => $body, - ConnectorProperties::Datagen($source_prop) => $body, - ConnectorProperties::S3($source_prop) => $body, - ConnectorProperties::MysqlCdc($source_prop) => $body, - ConnectorProperties::PostgresCdc($source_prop) => $body, - ConnectorProperties::CitusCdc($source_prop) => $body, - ConnectorProperties::GooglePubsub($source_prop) => $body, - ConnectorProperties::Nats($source_prop) => $body, - } - }}; -} +for_all_sources!(impl_connector_properties); impl ConnectorProperties { + pub fn extract(mut props: HashMap) -> Result { + const UPSTREAM_SOURCE_KEY: &str = "connector"; + let connector = props + .remove(UPSTREAM_SOURCE_KEY) + .ok_or_else(|| anyhow!("Must specify 'connector' in WITH clause"))?; + match_source_name_str!( + connector.to_lowercase().as_str(), + PropType, + PropType::try_from_hashmap(props).map(ConnectorProperties::from), + |other| Err(anyhow!("connector '{}' is not supported", other)) + ) + } + + pub fn init_cdc_properties(&mut self, table_schema: PbTableSchema) { + dispatch_source_prop!(self, prop, prop.set_schema(table_schema)); + } + + pub fn is_cdc_connector(&self) -> bool { + fn is_cdc_source(_prop: &P) -> bool { + P::IS_CDC_SOURCE + } + dispatch_source_prop!(self, prop, is_cdc_source(prop.deref())) + } + pub fn support_multiple_splits(&self) -> bool { matches!(self, ConnectorProperties::Kafka(_)) } } -#[derive(Debug, Clone, EnumAsInner, PartialEq, Hash)] -pub enum SplitImpl { - Kafka(KafkaSplit), - Pulsar(PulsarSplit), - Kinesis(KinesisSplit), - Nexmark(NexmarkSplit), - Datagen(DatagenSplit), - GooglePubsub(PubsubSplit), - MysqlCdc(DebeziumCdcSplit), - PostgresCdc(DebeziumCdcSplit), - CitusCdc(DebeziumCdcSplit), - Nats(NatsSplit), - S3(FsSplit), +for_all_sources!(impl_split); + +impl From<&SplitImpl> for ConnectorSplit { + fn from(split: &SplitImpl) -> Self { + dispatch_split_impl!(split, inner, SourcePropType, { + ConnectorSplit { + split_type: String::from(SourcePropType::SOURCE_NAME), + encoded_split: inner.encode_to_bytes().to_vec(), + } + }) + } +} + +impl TryFrom<&ConnectorSplit> for SplitImpl { + type Error = anyhow::Error; + + fn try_from(split: &ConnectorSplit) -> std::result::Result { + match_source_name_str!( + split.split_type.to_lowercase().as_str(), + PropType, + { + ::Split::restore_from_bytes( + split.encoded_split.as_ref(), + ) + .map(Into::into) + }, + |other| Err(anyhow!("connector '{}' is not supported", other)) + ) + } } // for the `FsSourceExecutor` @@ -364,29 +377,59 @@ impl SplitImpl { } } -impl_connector_properties! { - { Kafka, KAFKA_CONNECTOR }, - { Pulsar, PULSAR_CONNECTOR }, - { Kinesis, KINESIS_CONNECTOR }, - { Nexmark, NEXMARK_CONNECTOR }, - { Datagen, DATAGEN_CONNECTOR }, - { S3, S3_CONNECTOR }, - { GooglePubsub, GOOGLE_PUBSUB_CONNECTOR}, - { Nats, NATS_CONNECTOR } +impl SplitImpl { + fn restore_from_json_inner(split_type: &str, value: JsonbVal) -> Result { + match_source_name_str!( + split_type.to_lowercase().as_str(), + PropType, + ::Split::restore_from_json(value).map(Into::into), + |other| Err(anyhow!("connector '{}' is not supported", other)) + ) + } +} + +impl SplitMetaData for SplitImpl { + fn id(&self) -> SplitId { + dispatch_split_impl!(self, inner, IgnoreType, inner.id()) + } + + fn encode_to_json(&self) -> JsonbVal { + use serde_json::json; + let inner = self.encode_to_json_inner().take(); + json!({ SPLIT_TYPE_FIELD: self.get_type(), SPLIT_INFO_FIELD: inner}).into() + } + + fn restore_from_json(value: JsonbVal) -> Result { + let mut value = value.take(); + let json_obj = value.as_object_mut().unwrap(); + let split_type = json_obj + .remove(SPLIT_TYPE_FIELD) + .unwrap() + .as_str() + .unwrap() + .to_string(); + let inner_value = json_obj.remove(SPLIT_INFO_FIELD).unwrap(); + Self::restore_from_json_inner(&split_type, inner_value.into()) + } } -impl_split! { - { Kafka, KAFKA_CONNECTOR, KafkaSplit }, - { Pulsar, PULSAR_CONNECTOR, PulsarSplit }, - { Kinesis, KINESIS_CONNECTOR, KinesisSplit }, - { Nexmark, NEXMARK_CONNECTOR, NexmarkSplit }, - { Datagen, DATAGEN_CONNECTOR, DatagenSplit }, - { GooglePubsub, GOOGLE_PUBSUB_CONNECTOR, PubsubSplit }, - { MysqlCdc, MYSQL_CDC_CONNECTOR, DebeziumCdcSplit }, - { PostgresCdc, POSTGRES_CDC_CONNECTOR, DebeziumCdcSplit }, - { CitusCdc, CITUS_CDC_CONNECTOR, DebeziumCdcSplit }, - { S3, S3_CONNECTOR, FsSplit }, - { Nats, NATS_CONNECTOR, NatsSplit } +impl SplitImpl { + pub fn get_type(&self) -> String { + dispatch_split_impl!(self, _ignored, PropType, { + PropType::SOURCE_NAME.to_string() + }) + } + + pub fn update_in_place(&mut self, start_offset: String) -> Result<()> { + dispatch_split_impl!(self, inner, IgnoreType, { + inner.update_with_offset(start_offset)? + }); + Ok(()) + } + + pub fn encode_to_json_inner(&self) -> JsonbVal { + dispatch_split_impl!(self, inner, IgnoreType, inner.encode_to_json()) + } } pub type DataType = risingwave_common::types::DataType; @@ -461,7 +504,8 @@ mod tests { use nexmark::event::EventType; use super::*; - use crate::source::cdc::MySqlCdcSplit; + use crate::source::cdc::{DebeziumCdcSplit, MySqlCdcSplit}; + use crate::source::kafka::KafkaSplit; #[test] fn test_split_impl_get_fn() -> Result<()> { diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index e7054060a6f2b..d80dfb50948b8 100644 --- a/src/connector/src/source/cdc/mod.rs +++ b/src/connector/src/source/cdc/mod.rs @@ -19,14 +19,13 @@ use std::collections::HashMap; use std::marker::PhantomData; pub use enumerator::*; -use paste::paste; use risingwave_common::catalog::{ColumnDesc, Field, Schema}; use risingwave_pb::connector_service::{PbSourceType, PbTableSchema, SourceType, TableSchema}; pub use source::*; pub use split::*; -use crate::impl_cdc_source_type; -use crate::source::{ConnectorProperties, SourceProperties, SplitImpl}; +use crate::source::{SourceProperties, SplitImpl, TryFromHashmap}; +use crate::{for_all_classified_sources, impl_cdc_source_type}; pub const CDC_CONNECTOR_NAME_SUFFIX: &str = "-cdc"; @@ -39,7 +38,7 @@ pub trait CdcSourceTypeTrait: Send + Sync + Clone + 'static { fn source_type() -> CdcSourceType; } -impl_cdc_source_type!({ Mysql, "mysql" }, { Postgres, "postgres" }, { Citus, "citus" }); +for_all_classified_sources!(impl_cdc_source_type); #[derive(Clone, Debug, Default)] pub struct CdcProperties { @@ -52,6 +51,16 @@ pub struct CdcProperties { pub _phantom: PhantomData, } +impl TryFromHashmap for CdcProperties { + fn try_from_hashmap(props: HashMap) -> anyhow::Result { + Ok(CdcProperties { + props, + table_schema: Default::default(), + _phantom: PhantomData, + }) + } +} + impl SourceProperties for CdcProperties where DebeziumCdcSplit: TryFrom + Into, @@ -61,7 +70,12 @@ where type SplitEnumerator = DebeziumSplitEnumerator; type SplitReader = CdcSplitReader; + const IS_CDC_SOURCE: bool = true; const SOURCE_NAME: &'static str = T::CDC_CONNECTOR_NAME; + + fn set_schema(&mut self, table_schema: PbTableSchema) { + self.table_schema = table_schema; + } } impl CdcProperties { From 2206d071df7ae3c772626053267b62991d6ca70b Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 11 Sep 2023 14:30:37 +0800 Subject: [PATCH 7/9] fmt --- src/source/src/connector_source.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 376c7710a183d..445bf0f6dbb90 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -80,8 +80,7 @@ impl ConnectorSource { column_ids: Vec, source_ctx: Arc, ) -> Result { - let Some(splits) = state - else { + let Some(splits) = state else { return Ok(pending().boxed()); }; let config = self.config.clone(); From 0fc9b65f9e0cd18fc6a605b8b12eb56852a2eb9c Mon Sep 17 00:00:00 2001 From: William Wen Date: Tue, 19 Sep 2023 12:55:03 +0800 Subject: [PATCH 8/9] extract update_with_offset to split trait --- src/connector/src/source/base.rs | 10 ++++++++ src/connector/src/source/cdc/split.rs | 22 +++++++++-------- src/connector/src/source/datagen/split.rs | 10 ++++---- .../src/source/filesystem/file_common.rs | 12 +++++----- .../src/source/google_pubsub/split.rs | 12 ++++------ src/connector/src/source/kafka/split.rs | 10 ++++---- src/connector/src/source/kinesis/split.rs | 22 ++++++++--------- src/connector/src/source/nats/split.rs | 10 ++++---- src/connector/src/source/nexmark/split.rs | 10 ++++---- src/connector/src/source/pulsar/split.rs | 24 +++++++++---------- src/meta/src/stream/source_manager.rs | 4 ++++ 11 files changed, 79 insertions(+), 67 deletions(-) diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 2176368d0e824..d0f858bf108fa 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -411,6 +411,15 @@ impl SplitMetaData for SplitImpl { let inner_value = json_obj.remove(SPLIT_INFO_FIELD).unwrap(); Self::restore_from_json_inner(&split_type, inner_value.into()) } + + fn update_with_offset(&mut self, start_offset: String) -> Result<()> { + dispatch_split_impl!( + self, + inner, + IgnoreType, + inner.update_with_offset(start_offset) + ) + } } impl SplitImpl { @@ -490,6 +499,7 @@ pub trait SplitMetaData: Sized { fn encode_to_json(&self) -> JsonbVal; fn restore_from_json(value: JsonbVal) -> Result; + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()>; } /// [`ConnectorState`] maintains the consuming splits' info. In specific split readers, diff --git a/src/connector/src/source/cdc/split.rs b/src/connector/src/source/cdc/split.rs index 9d13a3c5a6eac..3bd24992f75f0 100644 --- a/src/connector/src/source/cdc/split.rs +++ b/src/connector/src/source/cdc/split.rs @@ -132,6 +132,7 @@ pub struct DebeziumCdcSplit { impl SplitMetaData for DebeziumCdcSplit { fn id(&self) -> SplitId { + // TODO: may check T to get the specific cdc type assert!(self.mysql_split.is_some() || self.pg_split.is_some()); if let Some(split) = &self.mysql_split { return format!("{}", split.inner.split_id).into(); @@ -149,6 +150,17 @@ impl SplitMetaData for DebeziumCdcSplit { fn restore_from_json(value: JsonbVal) -> anyhow::Result { serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) } + + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + // TODO: may check T to get the specific cdc type + assert!(self.mysql_split.is_some() || self.pg_split.is_some()); + if let Some(split) = &mut self.mysql_split { + split.update_with_offset(start_offset)? + } else if let Some(split) = &mut self.pg_split { + split.update_with_offset(start_offset)? + } + Ok(()) + } } impl DebeziumCdcSplit { @@ -196,14 +208,4 @@ impl DebeziumCdcSplit { } unreachable!("invalid debezium split") } - - pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { - assert!(self.mysql_split.is_some() || self.pg_split.is_some()); - if let Some(split) = &mut self.mysql_split { - split.update_with_offset(start_offset)? - } else if let Some(split) = &mut self.pg_split { - split.update_with_offset(start_offset)? - } - Ok(()) - } } diff --git a/src/connector/src/source/datagen/split.rs b/src/connector/src/source/datagen/split.rs index 08babee97fcb9..617b933728837 100644 --- a/src/connector/src/source/datagen/split.rs +++ b/src/connector/src/source/datagen/split.rs @@ -39,6 +39,11 @@ impl SplitMetaData for DatagenSplit { fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } + + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + self.start_offset = Some(start_offset.as_str().parse::().unwrap()); + Ok(()) + } } impl DatagenSplit { @@ -49,9 +54,4 @@ impl DatagenSplit { start_offset, } } - - pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { - self.start_offset = Some(start_offset.as_str().parse::().unwrap()); - Ok(()) - } } diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index 4711a5080c5ae..d4328289b547f 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -38,6 +38,12 @@ impl SplitMetaData for FsSplit { fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } + + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + let offset = start_offset.parse().unwrap(); + self.offset = offset; + Ok(()) + } } impl FsSplit { @@ -48,10 +54,4 @@ impl FsSplit { size, } } - - pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { - let offset = start_offset.parse().unwrap(); - self.offset = offset; - Ok(()) - } } diff --git a/src/connector/src/source/google_pubsub/split.rs b/src/connector/src/source/google_pubsub/split.rs index d623a9337cf33..1f598eb6852d4 100644 --- a/src/connector/src/source/google_pubsub/split.rs +++ b/src/connector/src/source/google_pubsub/split.rs @@ -35,13 +35,6 @@ pub struct PubsubSplit { pub(crate) stop_offset: Option, } -impl PubsubSplit { - pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { - self.start_offset = Some(start_offset); - Ok(()) - } -} - impl SplitMetaData for PubsubSplit { fn restore_from_json(value: JsonbVal) -> anyhow::Result { serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) @@ -54,4 +47,9 @@ impl SplitMetaData for PubsubSplit { fn id(&self) -> SplitId { format!("{}-{}", self.subscription, self.index).into() } + + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + self.start_offset = Some(start_offset); + Ok(()) + } } diff --git a/src/connector/src/source/kafka/split.rs b/src/connector/src/source/kafka/split.rs index 9d67b4496fe52..31c834d5f1609 100644 --- a/src/connector/src/source/kafka/split.rs +++ b/src/connector/src/source/kafka/split.rs @@ -39,6 +39,11 @@ impl SplitMetaData for KafkaSplit { fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } + + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + self.start_offset = Some(start_offset.as_str().parse::().unwrap()); + Ok(()) + } } impl KafkaSplit { @@ -56,11 +61,6 @@ impl KafkaSplit { } } - pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { - self.start_offset = Some(start_offset.as_str().parse::().unwrap()); - Ok(()) - } - pub fn get_topic_and_partition(&self) -> (String, i32) { (self.topic.clone(), self.partition) } diff --git a/src/connector/src/source/kinesis/split.rs b/src/connector/src/source/kinesis/split.rs index 09138d842b8ea..e03ecb59bfcd0 100644 --- a/src/connector/src/source/kinesis/split.rs +++ b/src/connector/src/source/kinesis/split.rs @@ -46,6 +46,17 @@ impl SplitMetaData for KinesisSplit { fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } + + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + let start_offset = if start_offset.is_empty() { + KinesisOffset::Earliest + } else { + KinesisOffset::SequenceNumber(start_offset) + }; + + self.start_position = start_offset; + Ok(()) + } } impl KinesisSplit { @@ -60,15 +71,4 @@ impl KinesisSplit { end_position, } } - - pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { - let start_offset = if start_offset.is_empty() { - KinesisOffset::Earliest - } else { - KinesisOffset::SequenceNumber(start_offset) - }; - - self.start_position = start_offset; - Ok(()) - } } diff --git a/src/connector/src/source/nats/split.rs b/src/connector/src/source/nats/split.rs index f0fcfaff35481..5a5d85f8f6cbe 100644 --- a/src/connector/src/source/nats/split.rs +++ b/src/connector/src/source/nats/split.rs @@ -41,6 +41,11 @@ impl SplitMetaData for NatsSplit { fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } + + fn update_with_offset(&mut self, start_sequence: String) -> anyhow::Result<()> { + self.start_sequence = Some(start_sequence.as_str().parse::().unwrap()); + Ok(()) + } } impl NatsSplit { @@ -51,9 +56,4 @@ impl NatsSplit { start_sequence, } } - - pub fn update_with_offset(&mut self, start_sequence: String) -> anyhow::Result<()> { - self.start_sequence = Some(start_sequence.as_str().parse::().unwrap()); - Ok(()) - } } diff --git a/src/connector/src/source/nexmark/split.rs b/src/connector/src/source/nexmark/split.rs index e1730993f5fbc..221fa20cbfe48 100644 --- a/src/connector/src/source/nexmark/split.rs +++ b/src/connector/src/source/nexmark/split.rs @@ -38,6 +38,11 @@ impl SplitMetaData for NexmarkSplit { fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } + + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + self.start_offset = Some(start_offset.as_str().parse::().unwrap()); + Ok(()) + } } impl NexmarkSplit { @@ -48,9 +53,4 @@ impl NexmarkSplit { start_offset, } } - - pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { - self.start_offset = Some(start_offset.as_str().parse::().unwrap()); - Ok(()) - } } diff --git a/src/connector/src/source/pulsar/split.rs b/src/connector/src/source/pulsar/split.rs index 3f7f1424f0ea3..5e546c0473519 100644 --- a/src/connector/src/source/pulsar/split.rs +++ b/src/connector/src/source/pulsar/split.rs @@ -26,19 +26,6 @@ pub struct PulsarSplit { pub(crate) start_offset: PulsarEnumeratorOffset, } -impl PulsarSplit { - pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { - let start_offset = if start_offset.is_empty() { - PulsarEnumeratorOffset::Earliest - } else { - PulsarEnumeratorOffset::MessageId(start_offset) - }; - - self.start_offset = start_offset; - Ok(()) - } -} - impl SplitMetaData for PulsarSplit { fn id(&self) -> SplitId { // TODO: should avoid constructing a string every time @@ -52,4 +39,15 @@ impl SplitMetaData for PulsarSplit { fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } + + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + let start_offset = if start_offset.is_empty() { + PulsarEnumeratorOffset::Earliest + } else { + PulsarEnumeratorOffset::MessageId(start_offset) + }; + + self.start_offset = start_offset; + Ok(()) + } } diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index cae431aa5b188..4e2263b8843f1 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -936,6 +936,10 @@ mod tests { fn restore_from_json(value: JsonbVal) -> anyhow::Result { serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) } + + fn update_with_offset(&mut self, _start_offset: String) -> anyhow::Result<()> { + Ok(()) + } } fn check_all_splits( From 9a3b9b09061a4e40405c41e7d0462eab1508e92a Mon Sep 17 00:00:00 2001 From: William Wen Date: Tue, 19 Sep 2023 13:07:57 +0800 Subject: [PATCH 9/9] extract init_from_pb_source --- src/connector/src/source/base.rs | 17 ++++------------- src/connector/src/source/cdc/mod.rs | 26 ++++++++++++++++++++++++-- src/meta/src/stream/source_manager.rs | 26 +------------------------- 3 files changed, 29 insertions(+), 40 deletions(-) diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index d0f858bf108fa..d02645385b81f 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::collections::HashMap; -use std::ops::Deref; use std::sync::Arc; use anyhow::{anyhow, Result}; @@ -28,7 +27,7 @@ use risingwave_common::array::StreamChunk; use risingwave_common::catalog::TableId; use risingwave_common::error::{ErrorSuppressor, RwError}; use risingwave_common::types::{JsonbVal, Scalar}; -use risingwave_pb::connector_service::PbTableSchema; +use risingwave_pb::catalog::PbSource; use risingwave_pb::source::ConnectorSplit; use risingwave_rpc_client::ConnectorClient; use serde::de::DeserializeOwned; @@ -56,12 +55,11 @@ pub trait TryFromHashmap: Sized { pub trait SourceProperties: TryFromHashmap + Clone { const SOURCE_NAME: &'static str; - const IS_CDC_SOURCE: bool = false; type Split: SplitMetaData + TryFrom + Into; type SplitEnumerator: SplitEnumerator; type SplitReader: SplitReader; - fn set_schema(&mut self, _table_schema: PbTableSchema) {} + fn init_from_pb_source(&mut self, _source: &PbSource) {} } impl TryFromHashmap for P { @@ -312,15 +310,8 @@ impl ConnectorProperties { ) } - pub fn init_cdc_properties(&mut self, table_schema: PbTableSchema) { - dispatch_source_prop!(self, prop, prop.set_schema(table_schema)); - } - - pub fn is_cdc_connector(&self) -> bool { - fn is_cdc_source(_prop: &P) -> bool { - P::IS_CDC_SOURCE - } - dispatch_source_prop!(self, prop, is_cdc_source(prop.deref())) + pub fn init_from_pb_source(&mut self, source: &PbSource) { + dispatch_source_prop!(self, prop, prop.init_from_pb_source(source)) } pub fn support_multiple_splits(&self) -> bool { diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index d80dfb50948b8..1d795a7141e84 100644 --- a/src/connector/src/source/cdc/mod.rs +++ b/src/connector/src/source/cdc/mod.rs @@ -19,7 +19,9 @@ use std::collections::HashMap; use std::marker::PhantomData; pub use enumerator::*; +use itertools::Itertools; use risingwave_common::catalog::{ColumnDesc, Field, Schema}; +use risingwave_pb::catalog::PbSource; use risingwave_pb::connector_service::{PbSourceType, PbTableSchema, SourceType, TableSchema}; pub use source::*; pub use split::*; @@ -70,10 +72,30 @@ where type SplitEnumerator = DebeziumSplitEnumerator; type SplitReader = CdcSplitReader; - const IS_CDC_SOURCE: bool = true; const SOURCE_NAME: &'static str = T::CDC_CONNECTOR_NAME; - fn set_schema(&mut self, table_schema: PbTableSchema) { + fn init_from_pb_source(&mut self, source: &PbSource) { + let pk_indices = source + .pk_column_ids + .iter() + .map(|&id| { + source + .columns + .iter() + .position(|col| col.column_desc.as_ref().unwrap().column_id == id) + .unwrap() as u32 + }) + .collect_vec(); + + let table_schema = PbTableSchema { + columns: source + .columns + .iter() + .flat_map(|col| &col.column_desc) + .cloned() + .collect(), + pk_indices, + }; self.table_schema = table_schema; } } diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index 4e2263b8843f1..d6a7377f19928 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -29,7 +29,6 @@ use risingwave_connector::source::{ SplitEnumerator, SplitId, SplitImpl, SplitMetaData, }; use risingwave_pb::catalog::Source; -use risingwave_pb::connector_service::PbTableSchema; use risingwave_pb::source::{ConnectorSplit, ConnectorSplits}; use risingwave_rpc_client::ConnectorClient; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; @@ -77,30 +76,7 @@ struct ConnectorSourceWorker { fn extract_prop_from_source(source: &Source) -> MetaResult { let mut properties = ConnectorProperties::extract(source.properties.clone())?; - if properties.is_cdc_connector() { - let pk_indices = source - .pk_column_ids - .iter() - .map(|&id| { - source - .columns - .iter() - .position(|col| col.column_desc.as_ref().unwrap().column_id == id) - .unwrap() as u32 - }) - .collect_vec(); - - let table_schema = PbTableSchema { - columns: source - .columns - .iter() - .flat_map(|col| &col.column_desc) - .cloned() - .collect(), - pk_indices, - }; - properties.init_cdc_properties(table_schema); - } + properties.init_from_pb_source(source); Ok(properties) }