Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

refactor(source): refine some code of split reader #7644

Merged
merged 3 commits into from
Feb 2, 2023
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
30 changes: 1 addition & 29 deletions src/connector/src/macros.rs
Original file line number Diff line number Diff line change
Expand Up @@ -235,7 +235,7 @@ 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_msg_stream(self) {
pub(crate) async fn into_chunk_stream(self) {
let parser_config = self.parser_config.clone();
let actor_id = self.source_info.actor_id.to_string();
let source_id = self.source_info.source_id.to_string();
Expand Down Expand Up @@ -272,33 +272,5 @@ macro_rules! impl_common_split_reader_logic {
}
}
}

#[async_trait]
impl $crate::source::SplitReaderV2 for $reader {
type Properties = $props;

async fn new(
properties: $props,
state: Vec<SplitImpl>,
parser_config: ParserConfig,
metrics: Arc<SourceMetrics>,
source_info: SourceInfo,
columns: Option<Vec<Column>>,
) -> Result<Self> {
Self::new(
properties,
state,
parser_config,
metrics,
source_info,
columns,
)
.await
}

fn into_stream(self) -> $crate::source::BoxSourceWithStateStream {
self.into_msg_stream()
}
}
};
}
13 changes: 11 additions & 2 deletions src/connector/src/source/cdc/source/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,9 @@ use crate::parser::ParserConfig;
use crate::source::base::SourceMessage;
use crate::source::cdc::CdcProperties;
use crate::source::monitor::SourceMetrics;
use crate::source::{Column, SourceInfo, SplitId, SplitImpl, SplitMetaData};
use crate::source::{
BoxSourceWithStateStream, Column, SourceInfo, SplitId, SplitImpl, SplitMetaData, SplitReaderV2,
};

impl_common_split_reader_logic!(CdcSplitReader, CdcProperties);

Expand All @@ -43,7 +45,10 @@ pub struct CdcSplitReader {
source_info: SourceInfo,
}

impl CdcSplitReader {
#[async_trait]
impl SplitReaderV2 for CdcSplitReader {
type Properties = CdcProperties;

#[allow(clippy::unused_async)]
async fn new(
conn_props: CdcProperties,
Expand Down Expand Up @@ -71,6 +76,10 @@ impl CdcSplitReader {
)),
}
}

fn into_stream(self) -> BoxSourceWithStateStream {
self.into_chunk_stream()
}
}

impl CdcSplitReader {
Expand Down
14 changes: 12 additions & 2 deletions src/connector/src/source/datagen/source/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,8 @@ use crate::source::datagen::source::SEQUENCE_FIELD_KIND;
use crate::source::datagen::{DatagenProperties, DatagenSplit};
use crate::source::monitor::SourceMetrics;
use crate::source::{
BoxSourceStream, Column, DataType, SourceInfo, SplitId, SplitImpl, SplitMetaData,
BoxSourceStream, BoxSourceWithStateStream, Column, DataType, SourceInfo, SplitId, SplitImpl,
SplitMetaData, SplitReaderV2,
};

impl_common_split_reader_logic!(DatagenSplitReader, DatagenProperties);
Expand All @@ -45,7 +46,10 @@ pub struct DatagenSplitReader {
source_info: SourceInfo,
}

impl DatagenSplitReader {
#[async_trait]
impl SplitReaderV2 for DatagenSplitReader {
type Properties = DatagenProperties;

#[allow(clippy::unused_async)]
async fn new(
properties: DatagenProperties,
Expand Down Expand Up @@ -132,6 +136,12 @@ impl DatagenSplitReader {
})
}

fn into_stream(self) -> BoxSourceWithStateStream {
self.into_chunk_stream()
}
}

impl DatagenSplitReader {
pub(crate) fn into_data_stream(self) -> BoxSourceStream {
// Will buffer at most 4 event chunks.
const BUFFER_SIZE: usize = 4;
Expand Down
14 changes: 12 additions & 2 deletions src/connector/src/source/google_pubsub/source/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,10 @@ use crate::impl_common_split_reader_logic;
use crate::parser::ParserConfig;
use crate::source::google_pubsub::PubsubProperties;
use crate::source::monitor::SourceMetrics;
use crate::source::{Column, SourceInfo, SourceMessage, SplitId, SplitImpl, SplitMetaData};
use crate::source::{
BoxSourceWithStateStream, Column, SourceInfo, SourceMessage, SplitId, SplitImpl, SplitMetaData,
SplitReaderV2,
};

const PUBSUB_MAX_FETCH_MESSAGES: usize = 1024;

Expand Down Expand Up @@ -109,7 +112,10 @@ impl PubsubSplitReader {
}
}

impl PubsubSplitReader {
#[async_trait]
impl SplitReaderV2 for PubsubSplitReader {
type Properties = PubsubProperties;

async fn new(
properties: PubsubProperties,
splits: Vec<SplitImpl>,
Expand Down Expand Up @@ -169,4 +175,8 @@ impl PubsubSplitReader {
source_info,
})
}

fn into_stream(self) -> BoxSourceWithStateStream {
self.into_chunk_stream()
}
}
13 changes: 11 additions & 2 deletions src/connector/src/source/kafka/source/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,9 @@ use crate::parser::ParserConfig;
use crate::source::base::{SourceMessage, MAX_CHUNK_SIZE};
use crate::source::kafka::KafkaProperties;
use crate::source::monitor::SourceMetrics;
use crate::source::{Column, SourceInfo, SplitId, SplitImpl, SplitMetaData};
use crate::source::{
BoxSourceWithStateStream, Column, SourceInfo, SplitId, SplitImpl, SplitMetaData, SplitReaderV2,
};

impl_common_split_reader_logic!(KafkaSplitReader, KafkaProperties);

Expand All @@ -46,7 +48,10 @@ pub struct KafkaSplitReader {
source_info: SourceInfo,
}

impl KafkaSplitReader {
#[async_trait]
impl SplitReaderV2 for KafkaSplitReader {
type Properties = KafkaProperties;

async fn new(
properties: KafkaProperties,
splits: Vec<SplitImpl>,
Expand Down Expand Up @@ -136,6 +141,10 @@ impl KafkaSplitReader {
source_info,
})
}

fn into_stream(self) -> BoxSourceWithStateStream {
self.into_chunk_stream()
}
}

impl KafkaSplitReader {
Expand Down
14 changes: 12 additions & 2 deletions src/connector/src/source/kinesis/source/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,10 @@ use crate::source::kinesis::source::message::KinesisMessage;
use crate::source::kinesis::split::KinesisOffset;
use crate::source::kinesis::KinesisProperties;
use crate::source::monitor::SourceMetrics;
use crate::source::{Column, SourceInfo, SourceMessage, SplitId, SplitImpl, SplitMetaData};
use crate::source::{
BoxSourceWithStateStream, Column, SourceInfo, SourceMessage, SplitId, SplitImpl, SplitMetaData,
SplitReaderV2,
};

impl_common_split_reader_logic!(KinesisSplitReader, KinesisProperties);

Expand All @@ -52,7 +55,10 @@ pub struct KinesisSplitReader {
source_info: SourceInfo,
}

impl KinesisSplitReader {
#[async_trait]
impl SplitReaderV2 for KinesisSplitReader {
type Properties = KinesisProperties;

async fn new(
properties: KinesisProperties,
splits: Vec<SplitImpl>,
Expand Down Expand Up @@ -106,6 +112,10 @@ impl KinesisSplitReader {
source_info,
})
}

fn into_stream(self) -> BoxSourceWithStateStream {
self.into_chunk_stream()
}
}

impl KinesisSplitReader {
Expand Down
12 changes: 10 additions & 2 deletions src/connector/src/source/nexmark/source/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,8 @@ use crate::source::monitor::SourceMetrics;
use crate::source::nexmark::source::message::NexmarkMessage;
use crate::source::nexmark::{NexmarkProperties, NexmarkSplit};
use crate::source::{
BoxSourceStream, Column, SourceInfo, SourceMessage, SplitId, SplitImpl, SplitMetaData,
BoxSourceStream, BoxSourceWithStateStream, Column, SourceInfo, SourceMessage, SplitId,
SplitImpl, SplitMetaData, SplitReaderV2,
};

impl_common_split_reader_logic!(NexmarkSplitReader, NexmarkProperties);
Expand All @@ -52,7 +53,10 @@ pub struct NexmarkSplitReader {
source_info: SourceInfo,
}

impl NexmarkSplitReader {
#[async_trait]
impl SplitReaderV2 for NexmarkSplitReader {
type Properties = NexmarkProperties;

#[allow(clippy::unused_async)]
async fn new(
properties: NexmarkProperties,
Expand Down Expand Up @@ -96,6 +100,10 @@ impl NexmarkSplitReader {
source_info,
})
}

fn into_stream(self) -> BoxSourceWithStateStream {
self.into_chunk_stream()
}
}

impl NexmarkSplitReader {
Expand Down
12 changes: 10 additions & 2 deletions src/connector/src/source/pulsar/source/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,8 @@ use crate::source::monitor::SourceMetrics;
use crate::source::pulsar::split::PulsarSplit;
use crate::source::pulsar::{PulsarEnumeratorOffset, PulsarProperties};
use crate::source::{
Column, SourceInfo, SourceMessage, SplitId, SplitImpl, SplitMetaData, MAX_CHUNK_SIZE,
BoxSourceWithStateStream, Column, SourceInfo, SourceMessage, SplitId, SplitImpl, SplitMetaData,
SplitReaderV2, MAX_CHUNK_SIZE,
};

impl_common_split_reader_logic!(PulsarSplitReader, PulsarProperties);
Expand Down Expand Up @@ -89,7 +90,10 @@ fn parse_message_id(id: &str) -> Result<MessageIdData> {
Ok(message_id)
}

impl PulsarSplitReader {
#[async_trait]
impl SplitReaderV2 for PulsarSplitReader {
type Properties = PulsarProperties;

async fn new(
props: PulsarProperties,
splits: Vec<SplitImpl>,
Expand Down Expand Up @@ -157,6 +161,10 @@ impl PulsarSplitReader {
source_info,
})
}

fn into_stream(self) -> BoxSourceWithStateStream {
self.into_chunk_stream()
}
}

impl PulsarSplitReader {
Expand Down