From 976ed2a56e102d121d83821d2ac9953828965304 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Thu, 11 Jan 2024 18:01:29 +0800 Subject: [PATCH 01/42] add offset/partition cols upon building SourceDesc --- src/source/src/source_desc.rs | 58 +++++++++++++++++++++++++++++++++-- 1 file changed, 55 insertions(+), 3 deletions(-) diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index aed602aa71ff8..369ac9930a96c 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -15,15 +15,16 @@ use std::collections::HashMap; use std::sync::Arc; -use risingwave_common::catalog::ColumnDesc; +use risingwave_common::catalog::{ColumnDesc, ColumnId}; use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; +use risingwave_common::types::DataType; use risingwave_connector::parser::{EncodingProperties, ProtocolProperties, SpecificParserConfig}; use risingwave_connector::source::monitor::SourceMetrics; use risingwave_connector::source::{SourceColumnDesc, SourceColumnType}; use risingwave_connector::ConnectorParams; use risingwave_pb::catalog::PbStreamSourceInfo; -use risingwave_pb::plan_common::PbColumnCatalog; +use risingwave_pb::plan_common::{AdditionalColumnType, ColumnCatalog, PbColumnCatalog}; use crate::connector_source::ConnectorSource; use crate::fs_connector_source::FsConnectorSource; @@ -83,10 +84,61 @@ impl SourceDescBuilder { } fn column_catalogs_to_source_column_descs(&self) -> Vec { + let mut columns_exist = [false; 2]; + let last_column_id = self + .columns + .last() + .map(|c| c.column_desc.as_ref().unwrap().column_id.into()) + .unwrap_or(ColumnId::new(0)); + + let additional_columns = [ + ColumnCatalog { + column_desc: Some( + ColumnDesc::named_with_additional_column( + "_rw_partition", + last_column_id.next(), + DataType::Varchar, + AdditionalColumnType::Partition, + ) + .to_protobuf(), + ), + is_hidden: true, + }, + ColumnCatalog { + column_desc: Some( + ColumnDesc::named_with_additional_column( + "_rw_offset", + last_column_id.next().next(), + DataType::Varchar, + AdditionalColumnType::Offset, + ) + .to_protobuf(), + ), + is_hidden: true, + }, + ]; + let mut columns: Vec<_> = self .columns .iter() - .map(|c| SourceColumnDesc::from(&ColumnDesc::from(c.column_desc.as_ref().unwrap()))) + .chain(additional_columns.iter()) + .filter_map(|c| { + if match c.column_desc.as_ref().unwrap().get_additional_column_type() { + Ok(AdditionalColumnType::Partition) => { + std::mem::replace(&mut columns_exist[0], true) + } + Ok(AdditionalColumnType::Offset) => { + std::mem::replace(&mut columns_exist[1], true) + } + _ => false, + } { + None + } else { + Some(SourceColumnDesc::from(&ColumnDesc::from( + c.column_desc.as_ref().unwrap(), + ))) + } + }) .collect(); if let Some(row_id_index) = self.row_id_index { columns[row_id_index].column_type = SourceColumnType::RowId; From f2d9466c6749c501de8c8fd6546995dd09dacb24 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Fri, 12 Jan 2024 12:18:33 +0800 Subject: [PATCH 02/42] batch rename --- src/connector/benches/nexmark_integration.rs | 8 +++--- src/connector/src/parser/mod.rs | 17 ++++++------ src/connector/src/source/base.rs | 27 +++---------------- src/connector/src/source/cdc/source/reader.rs | 4 +-- src/connector/src/source/common.rs | 5 ++-- .../src/source/datagen/source/generator.rs | 6 ++--- .../src/source/datagen/source/reader.rs | 4 +-- .../opendal_source/opendal_reader.rs | 9 ++++--- .../src/source/filesystem/s3/source/reader.rs | 8 +++--- .../src/source/google_pubsub/source/reader.rs | 4 +-- .../src/source/kafka/source/reader.rs | 4 +-- .../src/source/kinesis/source/reader.rs | 4 +-- .../src/source/nats/source/reader.rs | 4 +-- .../src/source/nexmark/source/reader.rs | 11 ++++---- .../src/source/pulsar/source/reader.rs | 16 +++++------ src/connector/src/source/test_source.rs | 8 +++--- src/source/src/connector_source.rs | 4 +-- src/source/src/fs_connector_source.rs | 4 +-- src/source/src/table.rs | 4 +-- .../src/executor/source/fetch_executor.rs | 12 ++++----- .../src/executor/source/fs_source_executor.rs | 14 +++++----- .../src/executor/source/source_executor.rs | 16 +++++------ src/stream/src/executor/stream_reader.rs | 8 +++--- .../tests/integration_tests/sink/utils.rs | 8 +++--- 24 files changed, 95 insertions(+), 114 deletions(-) diff --git a/src/connector/benches/nexmark_integration.rs b/src/connector/benches/nexmark_integration.rs index 951d42e594732..85eb4756d3430 100644 --- a/src/connector/benches/nexmark_integration.rs +++ b/src/connector/benches/nexmark_integration.rs @@ -25,8 +25,8 @@ use risingwave_connector::parser::{ ByteStreamSourceParser, JsonParser, SourceParserIntoStreamExt, SpecificParserConfig, }; use risingwave_connector::source::{ - BoxSourceStream, BoxSourceWithStateStream, SourceColumnDesc, SourceMessage, SourceMeta, - StreamChunkWithState, + BoxSourceStream, BoxChunkedSourceStream, SourceColumnDesc, SourceMessage, SourceMeta, + StreamChunk, }; use tracing::Level; use tracing_subscriber::prelude::*; @@ -90,8 +90,8 @@ fn make_parser() -> impl ByteStreamSourceParser { JsonParser::new(props, columns, Default::default()).unwrap() } -fn make_stream_iter() -> impl Iterator { - let mut stream: BoxSourceWithStateStream = +fn make_stream_iter() -> impl Iterator { + let mut stream: BoxChunkedSourceStream = make_parser().into_stream(make_data_stream()).boxed(); std::iter::from_fn(move || { diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index cb82c443ba0a7..5c62edc1b4ee9 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -54,8 +54,7 @@ use crate::schema::schema_registry::SchemaRegistryAuth; use crate::source::monitor::GLOBAL_SOURCE_METRICS; use crate::source::{ extract_source_struct, BoxSourceStream, SourceColumnDesc, SourceColumnType, SourceContext, - SourceContextRef, SourceEncode, SourceFormat, SourceMeta, SourceWithStateStream, SplitId, - StreamChunkWithState, + SourceContextRef, SourceEncode, SourceFormat, SourceMeta, ChunkedSourceStream, SplitId, }; pub mod additional_columns; @@ -551,8 +550,8 @@ impl P { /// /// # Returns /// - /// A [`SourceWithStateStream`] which is a stream of parsed messages. - pub fn into_stream(self, data_stream: BoxSourceStream) -> impl SourceWithStateStream { + /// A [`ChunkedSourceStream`] which is a stream of parsed messages. + pub fn into_stream(self, data_stream: BoxSourceStream) -> impl ChunkedSourceStream { // Enable tracing to provide more information for parsing failures. let source_info = self.source_ctx().source_info; @@ -574,7 +573,7 @@ const MAX_ROWS_FOR_TRANSACTION: usize = 4096; // TODO: when upsert is disabled, how to filter those empty payload // Currently, an err is returned for non upsert with empty payload -#[try_stream(ok = StreamChunkWithState, error = RwError)] +#[try_stream(ok = StreamChunk, error = RwError)] async fn into_chunk_stream(mut parser: P, data_stream: BoxSourceStream) { let columns = parser.columns().to_vec(); @@ -604,7 +603,7 @@ async fn into_chunk_stream(mut parser: P, data_stream ); *len = 0; // reset `len` while keeping `id` yield_asap = false; - yield StreamChunkWithState { + yield StreamChunk { chunk: builder.take(batch_len), split_offset_mapping: Some(std::mem::take(&mut split_offset_mapping)), }; @@ -711,7 +710,7 @@ async fn into_chunk_stream(mut parser: P, data_stream // chunk now. if current_transaction.is_none() && yield_asap { yield_asap = false; - yield StreamChunkWithState { + yield StreamChunk { chunk: builder.take(batch_len - (i + 1)), split_offset_mapping: Some(std::mem::take(&mut split_offset_mapping)), }; @@ -724,7 +723,7 @@ async fn into_chunk_stream(mut parser: P, data_stream if current_transaction.is_none() { yield_asap = false; - yield StreamChunkWithState { + yield StreamChunk { chunk: builder.take(0), split_offset_mapping: Some(std::mem::take(&mut split_offset_mapping)), }; @@ -799,7 +798,7 @@ pub enum ByteStreamSourceParserImpl { CanalJson(CanalJsonParser), } -pub type ParsedStreamImpl = impl SourceWithStateStream + Unpin; +pub type ParsedStreamImpl = impl ChunkedSourceStream + Unpin; impl ByteStreamSourceParserImpl { /// Converts this parser into a stream of [`StreamChunk`]. diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 2016fee6de60d..1e4a064c829f7 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -343,30 +343,11 @@ pub fn extract_source_struct(info: &PbStreamSourceInfo) -> Result pub type BoxSourceStream = BoxStream<'static, Result>>; -pub trait SourceWithStateStream = - Stream> + Send + 'static; -pub type BoxSourceWithStateStream = BoxStream<'static, Result>; +pub trait ChunkedSourceStream = + Stream> + Send + 'static; +pub type BoxChunkedSourceStream = BoxStream<'static, Result>; pub type BoxTryStream = BoxStream<'static, Result>; -/// [`StreamChunkWithState`] returns stream chunk together with offset for each split. In the -/// current design, one connector source can have multiple split reader. The keys are unique -/// `split_id` and values are the latest offset for each split. -#[derive(Clone, Debug, PartialEq)] -pub struct StreamChunkWithState { - pub chunk: StreamChunk, - pub split_offset_mapping: Option>, -} - -/// The `split_offset_mapping` field is unused for the table source, so we implement `From` for it. -impl From for StreamChunkWithState { - fn from(chunk: StreamChunk) -> Self { - Self { - chunk, - split_offset_mapping: None, - } - } -} - /// [`SplitReader`] is a new abstraction of the external connector read interface which is /// responsible for parsing, it is used to read messages from the outside and transform them into a /// stream of parsed [`StreamChunk`] @@ -383,7 +364,7 @@ pub trait SplitReader: Sized + Send { columns: Option>, ) -> Result; - fn into_stream(self) -> BoxSourceWithStateStream; + fn into_stream(self) -> BoxChunkedSourceStream; } for_all_sources!(impl_connector_properties); diff --git a/src/connector/src/source/cdc/source/reader.rs b/src/connector/src/source/cdc/source/reader.rs index cb9c7dae3d114..301c9b75c05d0 100644 --- a/src/connector/src/source/cdc/source/reader.rs +++ b/src/connector/src/source/cdc/source/reader.rs @@ -31,7 +31,7 @@ use crate::parser::ParserConfig; use crate::source::base::SourceMessage; use crate::source::cdc::{CdcProperties, CdcSourceType, CdcSourceTypeTrait, DebeziumCdcSplit}; use crate::source::{ - into_chunk_stream, BoxSourceWithStateStream, Column, CommonSplitReader, SourceContextRef, + into_chunk_stream, BoxChunkedSourceStream, Column, CommonSplitReader, SourceContextRef, SplitId, SplitMetaData, SplitReader, }; @@ -186,7 +186,7 @@ impl SplitReader for CdcSplitReader { } } - fn into_stream(self) -> BoxSourceWithStateStream { + fn into_stream(self) -> BoxChunkedSourceStream { 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/common.rs b/src/connector/src/source/common.rs index 11cbfce5d97f5..e5c6f98aff44c 100644 --- a/src/connector/src/source/common.rs +++ b/src/connector/src/source/common.rs @@ -14,10 +14,11 @@ use futures::{Stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; +use risingwave_common::array::StreamChunk; use risingwave_common::error::RwError; use crate::parser::ParserConfig; -use crate::source::{SourceContextRef, SourceMessage, SplitReader, StreamChunkWithState}; +use crate::source::{SourceContextRef, SourceMessage, SplitReader}; pub(crate) trait CommonSplitReader: SplitReader + 'static { fn into_data_stream( @@ -25,7 +26,7 @@ pub(crate) trait CommonSplitReader: SplitReader + 'static { ) -> impl Stream, anyhow::Error>> + Send; } -#[try_stream(boxed, ok = StreamChunkWithState, error = RwError)] +#[try_stream(boxed, ok = StreamChunk, error = RwError)] pub(crate) async fn into_chunk_stream( reader: impl CommonSplitReader, parser_config: ParserConfig, diff --git a/src/connector/src/source/datagen/source/generator.rs b/src/connector/src/source/datagen/source/generator.rs index 6bf3b782d3949..e13460710e78a 100644 --- a/src/connector/src/source/datagen/source/generator.rs +++ b/src/connector/src/source/datagen/source/generator.rs @@ -24,7 +24,7 @@ use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; use crate::parser::{EncodingProperties, ProtocolProperties, SpecificParserConfig}; -use crate::source::{SourceMessage, SourceMeta, SplitId, StreamChunkWithState}; +use crate::source::{SourceMessage, SourceMeta, SplitId}; pub enum FieldDesc { // field is invisible, generate None @@ -158,7 +158,7 @@ impl DatagenEventGenerator { } } - #[try_stream(ok = StreamChunkWithState, error = RwError)] + #[try_stream(ok = StreamChunk, error = RwError)] pub async fn into_native_stream(mut self) { let mut interval = tokio::time::interval(Duration::from_secs(1)); const MAX_ROWS_PER_YIELD: u64 = 1024; @@ -207,7 +207,7 @@ impl DatagenEventGenerator { let mapping = hashmap! { self.split_id.clone() => (self.offset - 1).to_string() }; - yield StreamChunkWithState { + yield StreamChunk { chunk, split_offset_mapping: Some(mapping), }; diff --git a/src/connector/src/source/datagen/source/reader.rs b/src/connector/src/source/datagen/source/reader.rs index d276e6b414210..d8982b4dfe9c2 100644 --- a/src/connector/src/source/datagen/source/reader.rs +++ b/src/connector/src/source/datagen/source/reader.rs @@ -25,7 +25,7 @@ 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::{ - into_chunk_stream, BoxSourceWithStateStream, Column, CommonSplitReader, DataType, + into_chunk_stream, BoxChunkedSourceStream, Column, CommonSplitReader, DataType, SourceContextRef, SourceMessage, SplitId, SplitMetaData, SplitReader, }; @@ -138,7 +138,7 @@ impl SplitReader for DatagenSplitReader { }) } - fn into_stream(self) -> BoxSourceWithStateStream { + fn into_stream(self) -> BoxChunkedSourceStream { // Will buffer at most 4 event chunks. const BUFFER_SIZE: usize = 4; // spawn_data_generation_stream(self.generator.into_native_stream(), BUFFER_SIZE).boxed() diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs index 57dfa7396128f..e5eade4787130 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -17,6 +17,7 @@ use async_trait::async_trait; use futures::TryStreamExt; use futures_async_stream::try_stream; use opendal::Operator; +use risingwave_common::array::StreamChunk; use risingwave_common::error::RwError; use tokio::io::BufReader; use tokio_util::io::{ReaderStream, StreamReader}; @@ -27,8 +28,8 @@ use crate::parser::{ByteStreamSourceParserImpl, ParserConfig}; use crate::source::filesystem::nd_streaming::need_nd_streaming; use crate::source::filesystem::{nd_streaming, OpendalFsSplit}; use crate::source::{ - BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SourceMeta, SplitMetaData, - SplitReader, StreamChunkWithState, + BoxChunkedSourceStream, Column, SourceContextRef, SourceMessage, SourceMeta, SplitMetaData, + SplitReader, }; const MAX_CHANNEL_BUFFER_SIZE: usize = 2048; @@ -62,13 +63,13 @@ impl SplitReader for OpendalReader { Ok(opendal_reader) } - fn into_stream(self) -> BoxSourceWithStateStream { + fn into_stream(self) -> BoxChunkedSourceStream { self.into_chunk_stream() } } impl OpendalReader { - #[try_stream(boxed, ok = StreamChunkWithState, error = RwError)] + #[try_stream(boxed, ok = StreamChunk, error = RwError)] async fn into_chunk_stream(self) { for split in self.splits { let actor_id = self.source_ctx.source_info.actor_id.to_string(); diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index 7d3e638811b63..d0192155d1000 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -25,6 +25,7 @@ use aws_smithy_types::body::SdkBody; use aws_smithy_types::byte_stream::ByteStream; use futures_async_stream::try_stream; use io::StreamReader; +use risingwave_common::array::StreamChunk; use risingwave_common::error::RwError; use tokio::io::BufReader; use tokio_util::io; @@ -39,8 +40,7 @@ use crate::source::filesystem::nd_streaming; use crate::source::filesystem::nd_streaming::need_nd_streaming; use crate::source::filesystem::s3::S3Properties; use crate::source::{ - BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SourceMeta, - StreamChunkWithState, + BoxChunkedSourceStream, Column, SourceContextRef, SourceMessage, SourceMeta, }; const MAX_CHANNEL_BUFFER_SIZE: usize = 2048; @@ -199,13 +199,13 @@ impl SplitReader for S3FileReader { Ok(s3_file_reader) } - fn into_stream(self) -> BoxSourceWithStateStream { + fn into_stream(self) -> BoxChunkedSourceStream { self.into_chunk_stream() } } impl S3FileReader { - #[try_stream(boxed, ok = StreamChunkWithState, error = RwError)] + #[try_stream(boxed, ok = StreamChunk, error = RwError)] async fn into_chunk_stream(self) { for split in self.splits { let actor_id = self.source_ctx.source_info.actor_id.to_string(); diff --git a/src/connector/src/source/google_pubsub/source/reader.rs b/src/connector/src/source/google_pubsub/source/reader.rs index dfaf21ad3164e..62c652f78d3a2 100644 --- a/src/connector/src/source/google_pubsub/source/reader.rs +++ b/src/connector/src/source/google_pubsub/source/reader.rs @@ -25,7 +25,7 @@ use super::TaggedReceivedMessage; use crate::parser::ParserConfig; use crate::source::google_pubsub::{PubsubProperties, PubsubSplit}; use crate::source::{ - into_chunk_stream, BoxSourceWithStateStream, Column, CommonSplitReader, SourceContextRef, + into_chunk_stream, BoxChunkedSourceStream, Column, CommonSplitReader, SourceContextRef, SourceMessage, SplitId, SplitMetaData, SplitReader, }; @@ -164,7 +164,7 @@ impl SplitReader for PubsubSplitReader { }) } - fn into_stream(self) -> BoxSourceWithStateStream { + fn into_stream(self) -> BoxChunkedSourceStream { 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 7f0cab97122ae..b26f3a57a55d2 100644 --- a/src/connector/src/source/kafka/source/reader.rs +++ b/src/connector/src/source/kafka/source/reader.rs @@ -33,7 +33,7 @@ use crate::source::kafka::{ KafkaProperties, KafkaSplit, PrivateLinkConsumerContext, KAFKA_ISOLATION_LEVEL, }; use crate::source::{ - into_chunk_stream, BoxSourceWithStateStream, Column, CommonSplitReader, SourceContextRef, + into_chunk_stream, BoxChunkedSourceStream, Column, CommonSplitReader, SourceContextRef, SplitId, SplitMetaData, SplitReader, }; @@ -145,7 +145,7 @@ impl SplitReader for KafkaSplitReader { }) } - fn into_stream(self) -> BoxSourceWithStateStream { + fn into_stream(self) -> BoxChunkedSourceStream { 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/kinesis/source/reader.rs b/src/connector/src/source/kinesis/source/reader.rs index 0e5b07d97736a..67e79f8144be6 100644 --- a/src/connector/src/source/kinesis/source/reader.rs +++ b/src/connector/src/source/kinesis/source/reader.rs @@ -29,7 +29,7 @@ use crate::source::kinesis::source::message::from_kinesis_record; use crate::source::kinesis::split::{KinesisOffset, KinesisSplit}; use crate::source::kinesis::KinesisProperties; use crate::source::{ - into_chunk_stream, BoxSourceWithStateStream, Column, CommonSplitReader, SourceContextRef, + into_chunk_stream, BoxChunkedSourceStream, Column, CommonSplitReader, SourceContextRef, SourceMessage, SplitId, SplitMetaData, SplitReader, }; @@ -113,7 +113,7 @@ impl SplitReader for KinesisSplitReader { }) } - fn into_stream(self) -> BoxSourceWithStateStream { + fn into_stream(self) -> BoxChunkedSourceStream { 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/nats/source/reader.rs b/src/connector/src/source/nats/source/reader.rs index 88ba9d862a925..a9ddd9ea79c83 100644 --- a/src/connector/src/source/nats/source/reader.rs +++ b/src/connector/src/source/nats/source/reader.rs @@ -24,7 +24,7 @@ use crate::parser::ParserConfig; use crate::source::common::{into_chunk_stream, CommonSplitReader}; use crate::source::nats::NatsProperties; use crate::source::{ - BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SplitId, SplitReader, + BoxChunkedSourceStream, Column, SourceContextRef, SourceMessage, SplitId, SplitReader, }; pub struct NatsSplitReader { @@ -93,7 +93,7 @@ impl SplitReader for NatsSplitReader { }) } - fn into_stream(self) -> BoxSourceWithStateStream { + fn into_stream(self) -> BoxChunkedSourceStream { 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/nexmark/source/reader.rs b/src/connector/src/source/nexmark/source/reader.rs index 10c052c6d1b98..ee58b2f371b49 100644 --- a/src/connector/src/source/nexmark/source/reader.rs +++ b/src/connector/src/source/nexmark/source/reader.rs @@ -34,8 +34,7 @@ use crate::source::nexmark::source::combined_event::{ }; use crate::source::nexmark::{NexmarkProperties, NexmarkSplit}; use crate::source::{ - BoxSourceWithStateStream, Column, SourceContextRef, SplitId, SplitMetaData, SplitReader, - StreamChunkWithState, + BoxChunkedSourceStream, Column, SourceContextRef, SplitId, SplitMetaData, SplitReader, }; #[derive(Debug)] @@ -106,7 +105,7 @@ impl SplitReader for NexmarkSplitReader { }) } - fn into_stream(self) -> BoxSourceWithStateStream { + fn into_stream(self) -> BoxChunkedSourceStream { 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 split_id = self.split_id.clone(); @@ -116,7 +115,7 @@ impl SplitReader for NexmarkSplitReader { const BUFFER_SIZE: usize = 4; spawn_data_generation_stream( self.into_native_stream().inspect_ok( - move |chunk_with_states: &StreamChunkWithState| { + move |chunk_with_states: &StreamChunk| { metrics .partition_input_count .with_label_values(&[&actor_id, &source_id, &split_id]) @@ -134,7 +133,7 @@ impl SplitReader for NexmarkSplitReader { } impl NexmarkSplitReader { - #[try_stream(boxed, ok = StreamChunkWithState, error = RwError)] + #[try_stream(boxed, ok = StreamChunk, error = RwError)] async fn into_native_stream(mut self) { let start_time = Instant::now(); let start_offset = self.generator.global_offset(); @@ -173,7 +172,7 @@ impl NexmarkSplitReader { } let mapping = hashmap! {self.split_id.clone() => self.generator.offset().to_string()}; let stream_chunk = StreamChunk::from_rows(&rows, &event_dtypes); - yield StreamChunkWithState { + yield StreamChunk { chunk: stream_chunk, split_offset_mapping: Some(mapping), }; diff --git a/src/connector/src/source/pulsar/source/reader.rs b/src/connector/src/source/pulsar/source/reader.rs index 8d80487a7da8b..7c240b5610b5d 100644 --- a/src/connector/src/source/pulsar/source/reader.rs +++ b/src/connector/src/source/pulsar/source/reader.rs @@ -37,8 +37,8 @@ use crate::parser::ParserConfig; use crate::source::pulsar::split::PulsarSplit; use crate::source::pulsar::{PulsarEnumeratorOffset, PulsarProperties}; use crate::source::{ - into_chunk_stream, BoxSourceWithStateStream, Column, CommonSplitReader, SourceContextRef, - SourceMessage, SplitId, SplitMetaData, SplitReader, StreamChunkWithState, + into_chunk_stream, BoxChunkedSourceStream, Column, CommonSplitReader, SourceContextRef, + SourceMessage, SplitId, SplitMetaData, SplitReader }; pub enum PulsarSplitReader { @@ -83,7 +83,7 @@ impl SplitReader for PulsarSplitReader { } } - fn into_stream(self) -> BoxSourceWithStateStream { + fn into_stream(self) -> BoxChunkedSourceStream { match self { Self::Broker(reader) => { let (parser_config, source_context) = @@ -234,7 +234,7 @@ impl SplitReader for PulsarBrokerReader { }) } - fn into_stream(self) -> BoxSourceWithStateStream { + fn into_stream(self) -> BoxChunkedSourceStream { let parser_config = self.parser_config.clone(); let source_context = self.source_ctx.clone(); into_chunk_stream(self, parser_config, source_context) @@ -356,7 +356,7 @@ impl PulsarIcebergReader { Ok(table) } - #[try_stream(ok = StreamChunkWithState, error = anyhow::Error)] + #[try_stream(ok = StreamChunk, error = anyhow::Error)] async fn as_stream_chunk_stream(&self) { #[for_await] for file_scan in self.scan().await? { @@ -371,7 +371,7 @@ impl PulsarIcebergReader { } } - #[try_stream(ok = StreamChunkWithState, error = RwError)] + #[try_stream(ok = StreamChunk, error = RwError)] async fn into_stream(self) { let (props, mut split, parser_config, source_ctx) = ( self.props.clone(), @@ -470,7 +470,7 @@ impl PulsarIcebergReader { fn convert_record_batch_to_source_with_state( &self, record_batch: &RecordBatch, - ) -> Result { + ) -> Result { let mut offsets = Vec::with_capacity(record_batch.num_rows()); let ledger_id_array = record_batch @@ -544,7 +544,7 @@ impl PulsarIcebergReader { offsets.last().unwrap().clone(), )])); - Ok(StreamChunkWithState { + Ok(StreamChunk { chunk: stream_chunk, split_offset_mapping: state, }) diff --git a/src/connector/src/source/test_source.rs b/src/connector/src/source/test_source.rs index 6c10ff9934eef..5020df37465f5 100644 --- a/src/connector/src/source/test_source.rs +++ b/src/connector/src/source/test_source.rs @@ -24,7 +24,7 @@ use with_options::WithOptions; use crate::parser::ParserConfig; use crate::source::{ - BoxSourceWithStateStream, Column, SourceContextRef, SourceEnumeratorContextRef, + BoxChunkedSourceStream, Column, SourceContextRef, SourceEnumeratorContextRef, SourceProperties, SplitEnumerator, SplitId, SplitMetaData, SplitReader, TryFromHashmap, }; @@ -44,7 +44,7 @@ pub type BoxIntoSourceStream = Box< ParserConfig, SourceContextRef, Option>, - ) -> BoxSourceWithStateStream + ) -> BoxChunkedSourceStream + Send + 'static, >; @@ -68,7 +68,7 @@ impl BoxSource { ParserConfig, SourceContextRef, Option>, - ) -> BoxSourceWithStateStream + ) -> BoxChunkedSourceStream + Send + 'static, ) -> BoxSource { @@ -218,7 +218,7 @@ impl SplitReader for TestSourceSplitReader { }) } - fn into_stream(self) -> BoxSourceWithStateStream { + fn into_stream(self) -> BoxChunkedSourceStream { (get_registry() .box_source .lock() diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 441a91836bb0a..23d25e6cde705 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -34,7 +34,7 @@ use risingwave_connector::source::filesystem::opendal_source::{ }; use risingwave_connector::source::filesystem::FsPageItem; use risingwave_connector::source::{ - create_split_reader, BoxSourceWithStateStream, BoxTryStream, Column, ConnectorProperties, + create_split_reader, BoxChunkedSourceStream, BoxTryStream, Column, ConnectorProperties, ConnectorState, FsFilterCtrlCtx, SourceColumnDesc, SourceContext, SplitReader, }; use tokio::time; @@ -117,7 +117,7 @@ impl ConnectorSource { state: ConnectorState, column_ids: Vec, source_ctx: Arc, - ) -> Result { + ) -> Result { let Some(splits) = state else { return Ok(pending().boxed()); }; diff --git a/src/source/src/fs_connector_source.rs b/src/source/src/fs_connector_source.rs index a44fc0a9792a2..f213e140b49f2 100644 --- a/src/source/src/fs_connector_source.rs +++ b/src/source/src/fs_connector_source.rs @@ -26,7 +26,7 @@ use risingwave_common::error::Result; use risingwave_connector::dispatch_source_prop; use risingwave_connector::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; use risingwave_connector::source::{ - create_split_reader, BoxSourceWithStateStream, ConnectorProperties, ConnectorState, + create_split_reader, BoxChunkedSourceStream, ConnectorProperties, ConnectorState, SourceColumnDesc, SourceContext, SplitReader, }; @@ -81,7 +81,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)?; diff --git a/src/source/src/table.rs b/src/source/src/table.rs index ba0292e4f6caf..9c540fdba8157 100644 --- a/src/source/src/table.rs +++ b/src/source/src/table.rs @@ -22,7 +22,7 @@ use risingwave_common::catalog::ColumnDesc; use risingwave_common::error::{Result, RwError}; use risingwave_common::transaction::transaction_id::TxnId; use risingwave_common::transaction::transaction_message::TxnMsg; -use risingwave_connector::source::StreamChunkWithState; +use risingwave_connector::source::StreamChunk; use tokio::sync::oneshot; use crate::txn_channel::{txn_channel, Receiver, Sender}; @@ -251,7 +251,7 @@ pub struct TableStreamReader { } impl TableStreamReader { - #[try_stream(boxed, ok = StreamChunkWithState, error = RwError)] + #[try_stream(boxed, ok = StreamChunk, error = RwError)] pub async fn into_data_stream_for_test(mut self) { while let Some((txn_msg, notifier)) = self.rx.recv().await { // Notify about that we've taken the chunk. diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 3aa885cfffe1b..52aa9285e666b 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -30,8 +30,8 @@ use risingwave_connector::source::filesystem::opendal_source::{ }; use risingwave_connector::source::filesystem::OpendalFsSplit; use risingwave_connector::source::{ - BoxSourceWithStateStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData, - StreamChunkWithState, + BoxChunkedSourceStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData, + StreamChunk, }; use risingwave_connector::ConnectorParams; use risingwave_source::source_desc::SourceDesc; @@ -96,7 +96,7 @@ impl FsFetchExecutor { column_ids: Vec, source_ctx: SourceContext, source_desc: &SourceDesc, - stream: &mut StreamReaderWithPause, + stream: &mut StreamReaderWithPause, ) -> StreamExecutorResult<()> { let mut batch = Vec::with_capacity(SPLIT_BATCH_SIZE); 'vnodes: for vnode in state_store_handler.state_store.vnodes().iter_vnodes() { @@ -159,7 +159,7 @@ impl FsFetchExecutor { source_ctx: SourceContext, source_desc: &SourceDesc, batch: SplitBatch, - ) -> StreamExecutorResult { + ) -> StreamExecutorResult { source_desc .source .stream_reader(batch, column_ids, Arc::new(source_ctx)) @@ -199,7 +199,7 @@ impl FsFetchExecutor { state_store_handler.init_epoch(barrier.epoch); let mut splits_on_fetch: usize = 0; - let mut stream = StreamReaderWithPause::::new( + let mut stream = StreamReaderWithPause::::new( upstream, stream::pending().boxed(), ); @@ -301,7 +301,7 @@ impl FsFetchExecutor { } // StreamChunk from FsSourceReader, and the reader reads only one file. // If the file read out, replace with a new file reader. - Either::Right(StreamChunkWithState { + Either::Right(StreamChunk { chunk, split_offset_mapping, }) => { diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index 6275ef5d116f6..a76c5326b20da 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -24,8 +24,8 @@ use futures_async_stream::try_stream; use risingwave_common::catalog::Schema; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_connector::source::{ - BoxSourceWithStateStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitId, SplitImpl, - SplitMetaData, StreamChunkWithState, + BoxChunkedSourceStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitId, SplitImpl, + SplitMetaData, StreamChunk, }; use risingwave_source::source_desc::{FsSourceDesc, SourceDescBuilder}; use risingwave_storage::StateStore; @@ -90,7 +90,7 @@ impl FsSourceExecutor { &mut self, source_desc: &FsSourceDesc, state: ConnectorState, - ) -> StreamExecutorResult { + ) -> StreamExecutorResult { let column_ids = source_desc .columns .iter() @@ -116,7 +116,7 @@ impl FsSourceExecutor { async fn apply_split_change( &mut self, source_desc: &FsSourceDesc, - stream: &mut StreamReaderWithPause, + stream: &mut StreamReaderWithPause, mapping: &HashMap>, ) -> StreamExecutorResult<()> { if let Some(target_splits) = mapping.get(&self.actor_ctx.id).cloned() { @@ -182,7 +182,7 @@ impl FsSourceExecutor { async fn replace_stream_reader_with_target_state( &mut self, source_desc: &FsSourceDesc, - stream: &mut StreamReaderWithPause, + stream: &mut StreamReaderWithPause, target_state: Vec, ) -> StreamExecutorResult<()> { tracing::info!( @@ -340,7 +340,7 @@ impl FsSourceExecutor { // Merge the chunks from source and the barriers into a single stream. We prioritize // barriers over source data chunks here. let barrier_stream = barrier_to_message_stream(barrier_receiver).boxed(); - let mut stream = StreamReaderWithPause::::new( + let mut stream = StreamReaderWithPause::::new( barrier_stream, source_chunk_reader, ); @@ -408,7 +408,7 @@ impl FsSourceExecutor { } }, - Either::Right(StreamChunkWithState { + Either::Right(StreamChunk { chunk, split_offset_mapping, }) => { diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 1bb61789f1359..be6a442f2fbe9 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -22,8 +22,8 @@ use futures_async_stream::try_stream; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_connector::source::{ - BoxSourceWithStateStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitMetaData, - StreamChunkWithState, + BoxChunkedSourceStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitMetaData, + StreamChunk, }; use risingwave_connector::ConnectorParams; use risingwave_source::source_desc::{SourceDesc, SourceDescBuilder}; @@ -92,7 +92,7 @@ impl SourceExecutor { &self, source_desc: &SourceDesc, state: ConnectorState, - ) -> StreamExecutorResult { + ) -> StreamExecutorResult { let column_ids = source_desc .columns .iter() @@ -135,7 +135,7 @@ impl SourceExecutor { async fn apply_split_change( &mut self, source_desc: &SourceDesc, - stream: &mut StreamReaderWithPause, + stream: &mut StreamReaderWithPause, split_assignment: &HashMap>, ) -> StreamExecutorResult>> { self.metrics @@ -228,7 +228,7 @@ impl SourceExecutor { async fn rebuild_stream_reader_from_error( &mut self, source_desc: &SourceDesc, - stream: &mut StreamReaderWithPause, + stream: &mut StreamReaderWithPause, split_info: &mut [SplitImpl], e: StreamExecutorError, ) -> StreamExecutorResult<()> { @@ -272,7 +272,7 @@ impl SourceExecutor { async fn replace_stream_reader_with_target_state( &mut self, source_desc: &SourceDesc, - stream: &mut StreamReaderWithPause, + stream: &mut StreamReaderWithPause, target_state: Vec, ) -> StreamExecutorResult<()> { tracing::info!( @@ -424,7 +424,7 @@ impl SourceExecutor { // Merge the chunks from source and the barriers into a single stream. We prioritize // barriers over source data chunks here. let barrier_stream = barrier_to_message_stream(barrier_receiver).boxed(); - let mut stream = StreamReaderWithPause::::new( + let mut stream = StreamReaderWithPause::::new( barrier_stream, source_chunk_reader, ); @@ -543,7 +543,7 @@ impl SourceExecutor { } }, - Either::Right(StreamChunkWithState { + Either::Right(StreamChunk { chunk, split_offset_mapping, }) => { diff --git a/src/stream/src/executor/stream_reader.rs b/src/stream/src/executor/stream_reader.rs index de490f730dea8..f5901dc7cbeee 100644 --- a/src/stream/src/executor/stream_reader.rs +++ b/src/stream/src/executor/stream_reader.rs @@ -143,7 +143,7 @@ mod tests { use futures::{pin_mut, FutureExt}; use risingwave_common::array::StreamChunk; use risingwave_common::transaction::transaction_id::TxnId; - use risingwave_connector::source::StreamChunkWithState; + use risingwave_connector::source::StreamChunk; use risingwave_source::TableDmlHandle; use tokio::sync::mpsc; @@ -172,7 +172,7 @@ mod tests { let barrier_stream = barrier_to_message_stream(barrier_rx).boxed(); let stream = - StreamReaderWithPause::::new(barrier_stream, source_stream); + StreamReaderWithPause::::new(barrier_stream, source_stream); pin_mut!(stream); macro_rules! next { @@ -192,7 +192,7 @@ mod tests { .write_chunk(StreamChunk::default()) .await .unwrap(); - // We don't call end() here, since we test `StreamChunkWithState` instead of `TxnMsg`. + // We don't call end() here, since we test `StreamChunk` instead of `TxnMsg`. assert_matches!(next!().unwrap(), Either::Right(_)); // Write a barrier, and we should receive it. @@ -210,7 +210,7 @@ mod tests { .write_chunk(StreamChunk::default()) .await .unwrap(); - // We don't call end() here, since we test `StreamChunkWithState` instead of `TxnMsg`. + // We don't call end() here, since we test `StreamChunk` instead of `TxnMsg`. // We should receive the barrier. assert_matches!(next!().unwrap(), Either::Left(_)); diff --git a/src/tests/simulation/tests/integration_tests/sink/utils.rs b/src/tests/simulation/tests/integration_tests/sink/utils.rs index 7d397b8066e00..cc704bccafbe7 100644 --- a/src/tests/simulation/tests/integration_tests/sink/utils.rs +++ b/src/tests/simulation/tests/integration_tests/sink/utils.rs @@ -38,7 +38,7 @@ use risingwave_connector::sink::SinkError; use risingwave_connector::source::test_source::{ registry_test_source, BoxSource, TestSourceRegistryGuard, TestSourceSplit, }; -use risingwave_connector::source::StreamChunkWithState; +use risingwave_connector::source::StreamChunk; use risingwave_simulation::cluster::{Cluster, ConfigPath, Configuration}; use tokio::time::sleep; @@ -297,7 +297,7 @@ impl SimulationTestSource { split.offset.parse::().unwrap() + 1 }; - let mut stream: BoxStream<'static, StreamChunkWithState> = empty().boxed(); + let mut stream: BoxStream<'static, StreamChunk> = empty().boxed(); while offset < id_list.len() { let mut chunks = Vec::new(); @@ -306,7 +306,7 @@ impl SimulationTestSource { let chunk = build_stream_chunk(once((id, simple_name_of_id(id)))); let mut split_offset = HashMap::new(); split_offset.insert(split.id.clone(), offset.to_string()); - let chunk_with_state = StreamChunkWithState { + let chunk_with_state = StreamChunk { chunk, split_offset_mapping: Some(split_offset), }; @@ -332,7 +332,7 @@ impl SimulationTestSource { } stream - .chain(async { pending::().await }.into_stream()) + .chain(async { pending::().await }.into_stream()) .map(|chunk| Ok(chunk)) .boxed() })) From 095f3a1beeaffd9f137cfe8eb69b99747acbe5e4 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Fri, 12 Jan 2024 14:03:30 +0800 Subject: [PATCH 03/42] refactor nexmark reader --- src/connector/benches/nexmark_integration.rs | 5 ++-- src/connector/src/parser/mod.rs | 5 ++-- src/connector/src/source/base.rs | 3 +- .../src/source/filesystem/s3/source/reader.rs | 6 ++-- .../source/nexmark/source/combined_event.rs | 14 +++++---- .../src/source/nexmark/source/reader.rs | 30 ++++++++++--------- .../src/source/pulsar/source/reader.rs | 2 +- src/connector/src/source/test_source.rs | 4 +-- .../src/executor/source/fetch_executor.rs | 9 ++---- .../src/executor/source/fs_source_executor.rs | 6 ++-- .../src/executor/source/source_executor.rs | 6 ++-- src/stream/src/executor/stream_reader.rs | 3 +- 12 files changed, 44 insertions(+), 49 deletions(-) diff --git a/src/connector/benches/nexmark_integration.rs b/src/connector/benches/nexmark_integration.rs index 85eb4756d3430..b26e0f85a8218 100644 --- a/src/connector/benches/nexmark_integration.rs +++ b/src/connector/benches/nexmark_integration.rs @@ -25,7 +25,7 @@ use risingwave_connector::parser::{ ByteStreamSourceParser, JsonParser, SourceParserIntoStreamExt, SpecificParserConfig, }; use risingwave_connector::source::{ - BoxSourceStream, BoxChunkedSourceStream, SourceColumnDesc, SourceMessage, SourceMeta, + BoxChunkedSourceStream, BoxSourceStream, SourceColumnDesc, SourceMessage, SourceMeta, StreamChunk, }; use tracing::Level; @@ -91,8 +91,7 @@ fn make_parser() -> impl ByteStreamSourceParser { } fn make_stream_iter() -> impl Iterator { - let mut stream: BoxChunkedSourceStream = - make_parser().into_stream(make_data_stream()).boxed(); + let mut stream: BoxChunkedSourceStream = make_parser().into_stream(make_data_stream()).boxed(); std::iter::from_fn(move || { stream diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 5c62edc1b4ee9..f010cf8dd4b2d 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -53,8 +53,9 @@ use crate::parser::util::{extract_headers_from_meta, extreact_timestamp_from_met use crate::schema::schema_registry::SchemaRegistryAuth; use crate::source::monitor::GLOBAL_SOURCE_METRICS; use crate::source::{ - extract_source_struct, BoxSourceStream, SourceColumnDesc, SourceColumnType, SourceContext, - SourceContextRef, SourceEncode, SourceFormat, SourceMeta, ChunkedSourceStream, SplitId, + extract_source_struct, BoxSourceStream, ChunkedSourceStream, SourceColumnDesc, + SourceColumnType, SourceContext, SourceContextRef, SourceEncode, SourceFormat, SourceMeta, + SplitId, }; pub mod additional_columns; diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 1e4a064c829f7..5a8da625d31c7 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -343,8 +343,7 @@ pub fn extract_source_struct(info: &PbStreamSourceInfo) -> Result pub type BoxSourceStream = BoxStream<'static, Result>>; -pub trait ChunkedSourceStream = - Stream> + Send + 'static; +pub trait ChunkedSourceStream = Stream> + Send + 'static; pub type BoxChunkedSourceStream = BoxStream<'static, Result>; pub type BoxTryStream = BoxStream<'static, Result>; diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index d0192155d1000..d706042cf4780 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -39,9 +39,7 @@ use crate::source::filesystem::file_common::FsSplit; use crate::source::filesystem::nd_streaming; use crate::source::filesystem::nd_streaming::need_nd_streaming; use crate::source::filesystem::s3::S3Properties; -use crate::source::{ - BoxChunkedSourceStream, Column, SourceContextRef, SourceMessage, SourceMeta, -}; +use crate::source::{BoxChunkedSourceStream, Column, SourceContextRef, SourceMessage, SourceMeta}; const MAX_CHANNEL_BUFFER_SIZE: usize = 2048; const STREAM_READER_CAPACITY: usize = 4096; @@ -235,7 +233,7 @@ impl S3FileReader { .metrics .partition_input_count .with_label_values(&[&actor_id, &source_id, &split_id]) - .inc_by(msg.chunk.cardinality() as u64); + .inc_by(msg.cardinality() as u64); yield msg; } } diff --git a/src/connector/src/source/nexmark/source/combined_event.rs b/src/connector/src/source/nexmark/source/combined_event.rs index 7242c838f77f4..f72ccc4ded540 100644 --- a/src/connector/src/source/nexmark/source/combined_event.rs +++ b/src/connector/src/source/nexmark/source/combined_event.rs @@ -16,7 +16,6 @@ pub use nexmark::event::EventType; use nexmark::event::{Auction, Bid, Event, Person}; use risingwave_common::array::StructValue; use risingwave_common::catalog::ROWID_PREFIX; -use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, Datum, ScalarImpl, StructType, Timestamp}; use serde::{Deserialize, Serialize}; @@ -137,6 +136,8 @@ pub(crate) fn get_event_data_types( fields.insert(row_id_index, DataType::Serial); } + fields.extend([DataType::Varchar, DataType::Varchar]); + fields } @@ -180,7 +181,10 @@ pub(crate) fn get_bid_struct_type() -> StructType { ]) } -pub(crate) fn combined_event_to_row(e: CombinedEvent, row_id_index: Option) -> OwnedRow { +pub(crate) fn combined_event_to_row( + e: CombinedEvent, + row_id_index: Option, +) -> Vec> { let mut fields = vec![ Some(ScalarImpl::Int64(e.event_type as i64)), e.person @@ -199,10 +203,10 @@ pub(crate) fn combined_event_to_row(e: CombinedEvent, row_id_index: Option) -> OwnedRow { +pub(crate) fn event_to_row(e: Event, row_id_index: Option) -> Vec> { let mut fields = match e { Event::Person(p) => person_to_datum(p), Event::Auction(a) => auction_to_datum(a), @@ -212,7 +216,7 @@ pub(crate) fn event_to_row(e: Event, row_id_index: Option) -> OwnedRow { // _row_id fields.insert(row_id_index, None); } - OwnedRow::new(fields) + fields } fn person_to_datum(p: Person) -> Vec { diff --git a/src/connector/src/source/nexmark/source/reader.rs b/src/connector/src/source/nexmark/source/reader.rs index ee58b2f371b49..f56638c4b6068 100644 --- a/src/connector/src/source/nexmark/source/reader.rs +++ b/src/connector/src/source/nexmark/source/reader.rs @@ -18,13 +18,14 @@ use anyhow::Result; use async_trait::async_trait; use futures::{StreamExt, TryStreamExt}; use futures_async_stream::try_stream; -use maplit::hashmap; use nexmark::config::NexmarkConfig; use nexmark::event::EventType; use nexmark::EventGenerator; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::error::RwError; use risingwave_common::estimate_size::EstimateSize; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::ScalarImpl; use tokio::time::Instant; use crate::parser::ParserConfig; @@ -114,18 +115,17 @@ impl SplitReader for NexmarkSplitReader { // Will buffer at most 4 event chunks. const BUFFER_SIZE: usize = 4; spawn_data_generation_stream( - self.into_native_stream().inspect_ok( - move |chunk_with_states: &StreamChunk| { + self.into_native_stream() + .inspect_ok(move |chunk_with_states: &StreamChunk| { metrics .partition_input_count .with_label_values(&[&actor_id, &source_id, &split_id]) - .inc_by(chunk_with_states.chunk.cardinality() as u64); + .inc_by(chunk_with_states.cardinality() as u64); metrics .partition_input_bytes .with_label_values(&[&actor_id, &source_id, &split_id]) - .inc_by(chunk_with_states.chunk.estimated_size() as u64); - }, - ), + .inc_by(chunk_with_states.estimated_size() as u64); + }), BUFFER_SIZE, ) .boxed() @@ -146,11 +146,17 @@ impl NexmarkSplitReader { break; } let event = self.generator.next().unwrap(); - let row = match self.event_type { + let fields = match self.event_type { Some(_) => event_to_row(event, self.row_id_index), None => combined_event_to_row(new_combined_event(event), self.row_id_index), }; - rows.push((Op::Insert, row)); + fields.extend([ + Some(ScalarImpl::Utf8(self.split_id.as_ref().into())), + Some(ScalarImpl::Utf8( + self.generator.offset().to_string().into_boxed_str(), + )), + ]); + rows.push((Op::Insert, OwnedRow::new(fields))); } if rows.is_empty() { break; @@ -170,12 +176,8 @@ impl NexmarkSplitReader { ) .await; } - let mapping = hashmap! {self.split_id.clone() => self.generator.offset().to_string()}; let stream_chunk = StreamChunk::from_rows(&rows, &event_dtypes); - yield StreamChunk { - chunk: stream_chunk, - split_offset_mapping: Some(mapping), - }; + yield stream_chunk; } tracing::debug!(?self.event_type, "nexmark generator finished"); diff --git a/src/connector/src/source/pulsar/source/reader.rs b/src/connector/src/source/pulsar/source/reader.rs index 7c240b5610b5d..5a7bd7c359675 100644 --- a/src/connector/src/source/pulsar/source/reader.rs +++ b/src/connector/src/source/pulsar/source/reader.rs @@ -38,7 +38,7 @@ use crate::source::pulsar::split::PulsarSplit; use crate::source::pulsar::{PulsarEnumeratorOffset, PulsarProperties}; use crate::source::{ into_chunk_stream, BoxChunkedSourceStream, Column, CommonSplitReader, SourceContextRef, - SourceMessage, SplitId, SplitMetaData, SplitReader + SourceMessage, SplitId, SplitMetaData, SplitReader, }; pub enum PulsarSplitReader { diff --git a/src/connector/src/source/test_source.rs b/src/connector/src/source/test_source.rs index 5020df37465f5..066143eb928d2 100644 --- a/src/connector/src/source/test_source.rs +++ b/src/connector/src/source/test_source.rs @@ -24,8 +24,8 @@ use with_options::WithOptions; use crate::parser::ParserConfig; use crate::source::{ - BoxChunkedSourceStream, Column, SourceContextRef, SourceEnumeratorContextRef, - SourceProperties, SplitEnumerator, SplitId, SplitMetaData, SplitReader, TryFromHashmap, + BoxChunkedSourceStream, Column, SourceContextRef, SourceEnumeratorContextRef, SourceProperties, + SplitEnumerator, SplitId, SplitMetaData, SplitReader, TryFromHashmap, }; pub type BoxListSplits = Box< diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 52aa9285e666b..2e1f8b3eb8403 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -30,8 +30,7 @@ use risingwave_connector::source::filesystem::opendal_source::{ }; use risingwave_connector::source::filesystem::OpendalFsSplit; use risingwave_connector::source::{ - BoxChunkedSourceStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData, - StreamChunk, + BoxChunkedSourceStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData, StreamChunk, }; use risingwave_connector::ConnectorParams; use risingwave_source::source_desc::SourceDesc; @@ -199,10 +198,8 @@ impl FsFetchExecutor { state_store_handler.init_epoch(barrier.epoch); let mut splits_on_fetch: usize = 0; - let mut stream = StreamReaderWithPause::::new( - upstream, - stream::pending().boxed(), - ); + let mut stream = + StreamReaderWithPause::::new(upstream, stream::pending().boxed()); if barrier.is_pause_on_startup() { stream.pause_stream(); diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index a76c5326b20da..c2ba380076748 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -340,10 +340,8 @@ impl FsSourceExecutor { // Merge the chunks from source and the barriers into a single stream. We prioritize // barriers over source data chunks here. let barrier_stream = barrier_to_message_stream(barrier_receiver).boxed(); - let mut stream = StreamReaderWithPause::::new( - barrier_stream, - source_chunk_reader, - ); + let mut stream = + StreamReaderWithPause::::new(barrier_stream, source_chunk_reader); if start_with_paused { stream.pause_stream(); } diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index be6a442f2fbe9..cbd5c07cd45e5 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -424,10 +424,8 @@ impl SourceExecutor { // Merge the chunks from source and the barriers into a single stream. We prioritize // barriers over source data chunks here. let barrier_stream = barrier_to_message_stream(barrier_receiver).boxed(); - let mut stream = StreamReaderWithPause::::new( - barrier_stream, - source_chunk_reader, - ); + let mut stream = + StreamReaderWithPause::::new(barrier_stream, source_chunk_reader); // If the first barrier requires us to pause on startup, pause the stream. if barrier.is_pause_on_startup() { diff --git a/src/stream/src/executor/stream_reader.rs b/src/stream/src/executor/stream_reader.rs index f5901dc7cbeee..72211b86fb0d9 100644 --- a/src/stream/src/executor/stream_reader.rs +++ b/src/stream/src/executor/stream_reader.rs @@ -171,8 +171,7 @@ mod tests { .unwrap(); let barrier_stream = barrier_to_message_stream(barrier_rx).boxed(); - let stream = - StreamReaderWithPause::::new(barrier_stream, source_stream); + let stream = StreamReaderWithPause::::new(barrier_stream, source_stream); pin_mut!(stream); macro_rules! next { From 2217dc6624a271c1f02e5d7db260508ae4e36283 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Fri, 12 Jan 2024 14:33:11 +0800 Subject: [PATCH 04/42] fix nexmark --- .../src/source/nexmark/source/combined_event.rs | 2 -- src/connector/src/source/nexmark/source/reader.rs | 10 ++++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/connector/src/source/nexmark/source/combined_event.rs b/src/connector/src/source/nexmark/source/combined_event.rs index f72ccc4ded540..47a4d7e6f421d 100644 --- a/src/connector/src/source/nexmark/source/combined_event.rs +++ b/src/connector/src/source/nexmark/source/combined_event.rs @@ -136,8 +136,6 @@ pub(crate) fn get_event_data_types( fields.insert(row_id_index, DataType::Serial); } - fields.extend([DataType::Varchar, DataType::Varchar]); - fields } diff --git a/src/connector/src/source/nexmark/source/reader.rs b/src/connector/src/source/nexmark/source/reader.rs index f56638c4b6068..bff143a39ff85 100644 --- a/src/connector/src/source/nexmark/source/reader.rs +++ b/src/connector/src/source/nexmark/source/reader.rs @@ -25,7 +25,7 @@ use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::error::RwError; use risingwave_common::estimate_size::EstimateSize; use risingwave_common::row::OwnedRow; -use risingwave_common::types::ScalarImpl; +use risingwave_common::types::{ScalarImpl, DataType}; use tokio::time::Instant; use crate::parser::ParserConfig; @@ -138,7 +138,9 @@ impl NexmarkSplitReader { let start_time = Instant::now(); let start_offset = self.generator.global_offset(); let start_ts = self.generator.timestamp(); - let event_dtypes = get_event_data_types(self.event_type, self.row_id_index); + let mut event_dtypes_with_offset = get_event_data_types(self.event_type, self.row_id_index); + event_dtypes_with_offset.extend([DataType::Varchar, DataType::Varchar]); + loop { let mut rows = vec![]; while (rows.len() as u64) < self.max_chunk_size { @@ -176,8 +178,8 @@ impl NexmarkSplitReader { ) .await; } - let stream_chunk = StreamChunk::from_rows(&rows, &event_dtypes); - yield stream_chunk; + + yield StreamChunk::from_rows(&rows, &event_dtypes_with_offset); } tracing::debug!(?self.event_type, "nexmark generator finished"); From 9fd4670ab9facabaef0fc2d76b289c278aee38bf Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Fri, 12 Jan 2024 14:33:17 +0800 Subject: [PATCH 05/42] refactor datagen --- .../src/source/datagen/source/generator.rs | 22 +++++++++---------- .../src/source/datagen/source/reader.rs | 6 ++--- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/connector/src/source/datagen/source/generator.rs b/src/connector/src/source/datagen/source/generator.rs index e13460710e78a..036b65c8c7f8b 100644 --- a/src/connector/src/source/datagen/source/generator.rs +++ b/src/connector/src/source/datagen/source/generator.rs @@ -15,12 +15,11 @@ use std::time::{Duration, SystemTime, UNIX_EPOCH}; use anyhow::Result; use futures_async_stream::try_stream; -use maplit::hashmap; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::error::RwError; use risingwave_common::field_generator::FieldGeneratorImpl; use risingwave_common::row::OwnedRow; -use risingwave_common::types::DataType; +use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::iter_util::ZipEqFast; use crate::parser::{EncodingProperties, ProtocolProperties, SpecificParserConfig}; @@ -163,6 +162,7 @@ impl DatagenEventGenerator { let mut interval = tokio::time::interval(Duration::from_secs(1)); const MAX_ROWS_PER_YIELD: u64 = 1024; let mut reach_end = false; + let dtypes_with_offset: Vec<_> = self.data_types.into_iter().chain([DataType::Varchar, DataType::Varchar]).collect(); loop { // generate `partition_rows_per_second` rows per second interval.tick().await; @@ -197,20 +197,20 @@ impl DatagenEventGenerator { row.push(datum); } - rows.push((Op::Insert, OwnedRow::new(row))); self.offset += 1; + row.extend([ + Some(ScalarImpl::Utf8(self.split_id.as_ref().into())), + Some(ScalarImpl::Utf8( + self.offset.to_string().into_boxed_str(), + )), + ]); + + rows.push((Op::Insert, OwnedRow::new(row))); rows_generated_this_second += 1; } if !rows.is_empty() { - let chunk = StreamChunk::from_rows(&rows, &self.data_types); - let mapping = hashmap! { - self.split_id.clone() => (self.offset - 1).to_string() - }; - yield StreamChunk { - chunk, - split_offset_mapping: Some(mapping), - }; + yield StreamChunk::from_rows(&rows, &dtypes_with_offset); } if reach_end { diff --git a/src/connector/src/source/datagen/source/reader.rs b/src/connector/src/source/datagen/source/reader.rs index d8982b4dfe9c2..2c823d8bce729 100644 --- a/src/connector/src/source/datagen/source/reader.rs +++ b/src/connector/src/source/datagen/source/reader.rs @@ -154,11 +154,11 @@ impl SplitReader for DatagenSplitReader { spawn_data_generation_stream( self.generator .into_native_stream() - .inspect_ok(move |chunk_with_states| { + .inspect_ok(move |stream_chunk| { metrics .partition_input_count .with_label_values(&[&actor_id, &source_id, &split_id]) - .inc_by(chunk_with_states.chunk.cardinality() as u64); + .inc_by(stream_chunk.cardinality() as u64); }), BUFFER_SIZE, ) @@ -397,7 +397,7 @@ mod tests { .into_stream(); let stream_chunk = reader.next().await.unwrap().unwrap(); - let (op, row) = stream_chunk.chunk.rows().next().unwrap(); + let (op, row) = stream_chunk.rows().next().unwrap(); assert_eq!(op, Op::Insert); assert_eq!(row.datum_at(0), Some(ScalarImpl::Int32(533)).to_datum_ref(),); assert_eq!( From e3ac10184cd54e2a61a8641b70473fae9b096d0a Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Fri, 12 Jan 2024 14:34:42 +0800 Subject: [PATCH 06/42] fmt --- src/connector/src/source/datagen/source/generator.rs | 10 ++++++---- src/connector/src/source/nexmark/source/reader.rs | 2 +- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/connector/src/source/datagen/source/generator.rs b/src/connector/src/source/datagen/source/generator.rs index 036b65c8c7f8b..a5ea9fc53d265 100644 --- a/src/connector/src/source/datagen/source/generator.rs +++ b/src/connector/src/source/datagen/source/generator.rs @@ -162,7 +162,11 @@ impl DatagenEventGenerator { let mut interval = tokio::time::interval(Duration::from_secs(1)); const MAX_ROWS_PER_YIELD: u64 = 1024; let mut reach_end = false; - let dtypes_with_offset: Vec<_> = self.data_types.into_iter().chain([DataType::Varchar, DataType::Varchar]).collect(); + let dtypes_with_offset: Vec<_> = self + .data_types + .into_iter() + .chain([DataType::Varchar, DataType::Varchar]) + .collect(); loop { // generate `partition_rows_per_second` rows per second interval.tick().await; @@ -200,9 +204,7 @@ impl DatagenEventGenerator { self.offset += 1; row.extend([ Some(ScalarImpl::Utf8(self.split_id.as_ref().into())), - Some(ScalarImpl::Utf8( - self.offset.to_string().into_boxed_str(), - )), + Some(ScalarImpl::Utf8(self.offset.to_string().into_boxed_str())), ]); rows.push((Op::Insert, OwnedRow::new(row))); diff --git a/src/connector/src/source/nexmark/source/reader.rs b/src/connector/src/source/nexmark/source/reader.rs index bff143a39ff85..0185e307332f7 100644 --- a/src/connector/src/source/nexmark/source/reader.rs +++ b/src/connector/src/source/nexmark/source/reader.rs @@ -25,7 +25,7 @@ use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::error::RwError; use risingwave_common::estimate_size::EstimateSize; use risingwave_common::row::OwnedRow; -use risingwave_common::types::{ScalarImpl, DataType}; +use risingwave_common::types::{DataType, ScalarImpl}; use tokio::time::Instant; use crate::parser::ParserConfig; From f696d25092d9081ace744081011059cd2d032200 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Fri, 12 Jan 2024 14:35:21 +0800 Subject: [PATCH 07/42] refactor opendal reader --- .../src/source/filesystem/opendal_source/opendal_reader.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs index e5eade4787130..5efd0995a9869 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -95,7 +95,7 @@ impl OpendalReader { .metrics .partition_input_count .with_label_values(&[&actor_id, &source_id, &split_id]) - .inc_by(msg.chunk.cardinality() as u64); + .inc_by(msg.cardinality() as u64); yield msg; } } From 3fe8df98965ef636c31271e6a864dfda38de29f2 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Fri, 12 Jan 2024 14:37:08 +0800 Subject: [PATCH 08/42] fix test of plain_parser --- src/connector/src/parser/plain_parser.rs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 1af396db3990a..a4360778ff616 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -218,18 +218,18 @@ mod tests { let output = output .unwrap() .into_iter() - .filter(|c| c.chunk.cardinality() > 0) + .filter(|c| c.cardinality() > 0) .enumerate() .map(|(i, c)| { if i == 0 { // begin + 3 data messages - assert_eq!(4, c.chunk.cardinality()); + assert_eq!(4, c.cardinality()); } if i == 1 { // 2 data messages + 1 end - assert_eq!(3, c.chunk.cardinality()); + assert_eq!(3, c.cardinality()); } - c.chunk + c }) .collect_vec(); @@ -255,11 +255,11 @@ mod tests { let output = output .unwrap() .into_iter() - .filter(|c| c.chunk.cardinality() > 0) + .filter(|c| c.cardinality() > 0) .map(|c| { // 5 data messages in a single chunk - assert_eq!(5, c.chunk.cardinality()); - c.chunk + assert_eq!(5, c.cardinality()); + c }) .collect_vec(); From df51578bf9822dae50024c060e6a173b33fa5afa Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Mon, 15 Jan 2024 11:33:35 +0800 Subject: [PATCH 09/42] refactor pulsar-iceberg source --- .../src/source/pulsar/source/reader.rs | 21 +++++++------------ 1 file changed, 7 insertions(+), 14 deletions(-) diff --git a/src/connector/src/source/pulsar/source/reader.rs b/src/connector/src/source/pulsar/source/reader.rs index 5a7bd7c359675..90968b51d0bc4 100644 --- a/src/connector/src/source/pulsar/source/reader.rs +++ b/src/connector/src/source/pulsar/source/reader.rs @@ -356,7 +356,7 @@ impl PulsarIcebergReader { Ok(table) } - #[try_stream(ok = StreamChunk, error = anyhow::Error)] + #[try_stream(ok = (StreamChunk, HashMap), error = anyhow::Error)] async fn as_stream_chunk_stream(&self) { #[for_await] for file_scan in self.scan().await? { @@ -384,13 +384,9 @@ impl PulsarIcebergReader { #[for_await] for msg in self.as_stream_chunk_stream() { - let msg = + let (_chunk, mapping) = msg.inspect_err(|e| tracing::error!("Failed to read message from iceberg: {}", e))?; - last_msg_id = msg - .split_offset_mapping - .as_ref() - .and_then(|m| m.get(self.split.topic.to_string().as_str())) - .cloned(); + last_msg_id = mapping.get(self.split.topic.to_string().as_str()).cloned(); } tracing::info!("Finished reading pulsar message from iceberg"); @@ -470,7 +466,7 @@ impl PulsarIcebergReader { fn convert_record_batch_to_source_with_state( &self, record_batch: &RecordBatch, - ) -> Result { + ) -> Result<(StreamChunk, HashMap)> { let mut offsets = Vec::with_capacity(record_batch.num_rows()); let ledger_id_array = record_batch @@ -539,14 +535,11 @@ impl PulsarIcebergReader { let stream_chunk = StreamChunk::from(data_chunk); - let state = Some(HashMap::from([( + let state = HashMap::from([( self.split.topic.to_string().into(), offsets.last().unwrap().clone(), - )])); + )]); - Ok(StreamChunk { - chunk: stream_chunk, - split_offset_mapping: state, - }) + Ok((stream_chunk, state)) } } From c40634e5470638cb700f7934f3aef8dadbdd91d2 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Mon, 15 Jan 2024 12:18:35 +0800 Subject: [PATCH 10/42] refactor fetch_executor roughly --- .../src/executor/source/fetch_executor.rs | 83 ++++++++++++------- 1 file changed, 54 insertions(+), 29 deletions(-) diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 2e1f8b3eb8403..745708eebbd0d 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -21,6 +21,7 @@ use either::Either; use futures::pin_mut; use futures::stream::{self, StreamExt}; use futures_async_stream::try_stream; +use itertools::Itertools; use risingwave_common::catalog::{ColumnId, Schema, TableId}; use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{OwnedRow, Row}; @@ -33,6 +34,7 @@ use risingwave_connector::source::{ BoxChunkedSourceStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData, StreamChunk, }; use risingwave_connector::ConnectorParams; +use risingwave_pb::plan_common::AdditionalColumnType; use risingwave_source::source_desc::SourceDesc; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; @@ -194,6 +196,25 @@ impl FsFetchExecutor { .build() .map_err(StreamExecutorError::connector_error)?; + let (Some(partition_idx), Some(col_idx)) = ({ + let mut partition_idx = None; + let mut col_idx = None; + for (idx, column) in source_desc.columns.iter().enumerate() { + match additional_column_type { + AdditionalColumnType::Partition => { + partition_idx = Some(idx); + } + AdditionalColumnType::Offset => { + col_idx = Some(idx); + } + _ => (), + } + } + (partition_idx, col_idx) + }) else { + unreachable!("Partition and offset columns must be set."); + }; + // Initialize state table. state_store_handler.init_epoch(barrier.epoch); @@ -275,7 +296,7 @@ impl FsFetchExecutor { yield msg; } // Receiving file assignments from upstream list executor, - // store into state table and try building a new reader. + // store into state table. Message::Chunk(chunk) => { let file_assignment = chunk .data_chunk() @@ -297,37 +318,41 @@ impl FsFetchExecutor { } } // StreamChunk from FsSourceReader, and the reader reads only one file. - // If the file read out, replace with a new file reader. - Either::Right(StreamChunk { - chunk, - split_offset_mapping, - }) => { - let mapping = split_offset_mapping.unwrap(); - debug_assert_eq!(mapping.len(), 1); - if let Some((split_id, offset)) = mapping.into_iter().next() { - let row = state_store_handler - .get(split_id.clone()) - .await? - .expect("The fs_split should be in the state table."); - let fs_split = match row.datum_at(1) { - Some(ScalarRefImpl::Jsonb(jsonb_ref)) => { - OpendalFsSplit::::restore_from_json( - jsonb_ref.to_owned_scalar(), - )? - } - _ => unreachable!(), - }; - - if offset.parse::().unwrap() >= fs_split.size { - splits_on_fetch -= 1; - state_store_handler.delete(split_id).await?; - } else { - state_store_handler - .set(split_id, fs_split.encode_to_json()) - .await?; + Either::Right(chunk) => { + let chunk_last_row = chunk + .rows() + .last() + .expect("The chunk should have at least one row."); + let split_id = + chunk_last_row.datum_at(partition_idx).unwrap().into_utf8(); + let offset = chunk_last_row.datum_at(col_idx).unwrap().into_utf8(); + + let state = state_store_handler + .get(split_id.clone()) + .await? + .expect("The fs_split should be in the state table."); + let fs_split = match state.datum_at(1) { + Some(ScalarRefImpl::Jsonb(jsonb_ref)) => { + OpendalFsSplit::::restore_from_json( + jsonb_ref.to_owned_scalar(), + )? } + _ => unreachable!(), + }; + if offset.parse::().unwrap() >= fs_split.size { + splits_on_fetch -= 1; + state_store_handler.delete(split_id).await?; + } else { + state_store_handler + .set(split_id, fs_split.encode_to_json()) + .await?; } + let chunk = chunk.project( + (0..chunk.dimension()) + .filter(|&idx| idx != partition_idx && idx != col_idx), + ); + yield Message::Chunk(chunk); } } From 5d570fc5101db5e726fd9bc71495938cbcabe7ec Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Mon, 15 Jan 2024 12:29:15 +0800 Subject: [PATCH 11/42] refactor parser --- src/connector/src/parser/mod.rs | 26 +++----------------------- 1 file changed, 3 insertions(+), 23 deletions(-) diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index f010cf8dd4b2d..1f94e21e21a2c 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -55,7 +55,6 @@ use crate::source::monitor::GLOBAL_SOURCE_METRICS; use crate::source::{ extract_source_struct, BoxSourceStream, ChunkedSourceStream, SourceColumnDesc, SourceColumnType, SourceContext, SourceContextRef, SourceEncode, SourceFormat, SourceMeta, - SplitId, }; pub mod additional_columns; @@ -579,7 +578,6 @@ async fn into_chunk_stream(mut parser: P, data_stream let columns = parser.columns().to_vec(); let mut builder = SourceStreamChunkBuilder::with_capacity(columns, 0); - let mut split_offset_mapping = HashMap::::new(); struct Transaction { id: Box, @@ -604,10 +602,7 @@ async fn into_chunk_stream(mut parser: P, data_stream ); *len = 0; // reset `len` while keeping `id` yield_asap = false; - yield StreamChunk { - chunk: builder.take(batch_len), - split_offset_mapping: Some(std::mem::take(&mut split_offset_mapping)), - }; + yield builder.take(batch_len); } else { // Normal transaction. After the transaction is committed, we should yield the last // batch immediately, so set `yield_asap` to true. @@ -617,7 +612,6 @@ async fn into_chunk_stream(mut parser: P, data_stream // Clean state. Reserve capacity for the builder. assert!(builder.is_empty()); assert!(!yield_asap); - assert!(split_offset_mapping.is_empty()); let _ = builder.take(batch_len); } @@ -625,12 +619,6 @@ async fn into_chunk_stream(mut parser: P, data_stream for (i, msg) in batch.into_iter().enumerate() { if msg.key.is_none() && msg.payload.is_none() { tracing::debug!(offset = msg.offset, "skip parsing of heartbeat message"); - // assumes an empty message as a heartbeat - // heartbeat message offset should not overwrite data messages offset - split_offset_mapping - .entry(msg.split_id) - .or_insert(msg.offset.clone()); - continue; } @@ -644,8 +632,6 @@ async fn into_chunk_stream(mut parser: P, data_stream .observe(lag_ms as f64); } - split_offset_mapping.insert(msg.split_id.clone(), msg.offset.clone()); - let old_op_num = builder.op_num(); match parser .parse_one_with_txn( @@ -711,10 +697,7 @@ async fn into_chunk_stream(mut parser: P, data_stream // chunk now. if current_transaction.is_none() && yield_asap { yield_asap = false; - yield StreamChunk { - chunk: builder.take(batch_len - (i + 1)), - split_offset_mapping: Some(std::mem::take(&mut split_offset_mapping)), - }; + yield builder.take(batch_len - (i + 1)); } } } @@ -724,10 +707,7 @@ async fn into_chunk_stream(mut parser: P, data_stream if current_transaction.is_none() { yield_asap = false; - yield StreamChunk { - chunk: builder.take(0), - split_offset_mapping: Some(std::mem::take(&mut split_offset_mapping)), - }; + yield builder.take(0); } } } From 703569456a3249aeccf704428246eef9f9dda68b Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Mon, 15 Jan 2024 12:32:40 +0800 Subject: [PATCH 12/42] fix misc --- src/connector/benches/nexmark_integration.rs | 2 +- src/connector/src/source/nexmark/source/reader.rs | 2 +- src/source/src/table.rs | 3 +-- src/stream/src/executor/source/fetch_executor.rs | 5 +++-- src/stream/src/executor/source/fs_source_executor.rs | 7 ++----- 5 files changed, 8 insertions(+), 11 deletions(-) diff --git a/src/connector/benches/nexmark_integration.rs b/src/connector/benches/nexmark_integration.rs index b26e0f85a8218..a841eae1db475 100644 --- a/src/connector/benches/nexmark_integration.rs +++ b/src/connector/benches/nexmark_integration.rs @@ -19,6 +19,7 @@ use std::sync::LazyLock; use criterion::{criterion_group, criterion_main, BatchSize, Criterion}; use futures::{FutureExt, StreamExt, TryStreamExt}; use itertools::Itertools; +use risingwave_common::array::StreamChunk; use risingwave_common::catalog::ColumnId; use risingwave_common::types::DataType; use risingwave_connector::parser::{ @@ -26,7 +27,6 @@ use risingwave_connector::parser::{ }; use risingwave_connector::source::{ BoxChunkedSourceStream, BoxSourceStream, SourceColumnDesc, SourceMessage, SourceMeta, - StreamChunk, }; use tracing::Level; use tracing_subscriber::prelude::*; diff --git a/src/connector/src/source/nexmark/source/reader.rs b/src/connector/src/source/nexmark/source/reader.rs index 0185e307332f7..793878ce2a4c4 100644 --- a/src/connector/src/source/nexmark/source/reader.rs +++ b/src/connector/src/source/nexmark/source/reader.rs @@ -148,7 +148,7 @@ impl NexmarkSplitReader { break; } let event = self.generator.next().unwrap(); - let fields = match self.event_type { + let mut fields = match self.event_type { Some(_) => event_to_row(event, self.row_id_index), None => combined_event_to_row(new_combined_event(event), self.row_id_index), }; diff --git a/src/source/src/table.rs b/src/source/src/table.rs index 9c540fdba8157..ac4ea39f8f345 100644 --- a/src/source/src/table.rs +++ b/src/source/src/table.rs @@ -22,7 +22,6 @@ use risingwave_common::catalog::ColumnDesc; use risingwave_common::error::{Result, RwError}; use risingwave_common::transaction::transaction_id::TxnId; use risingwave_common::transaction::transaction_message::TxnMsg; -use risingwave_connector::source::StreamChunk; use tokio::sync::oneshot; use crate::txn_channel::{txn_channel, Receiver, Sender}; @@ -261,7 +260,7 @@ impl TableStreamReader { } TxnMsg::Data(_, chunk) => { _ = notifier.send(chunk.cardinality()); - yield chunk.into(); + yield chunk; } } } diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 745708eebbd0d..b08be03c80f50 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -21,7 +21,7 @@ use either::Either; use futures::pin_mut; use futures::stream::{self, StreamExt}; use futures_async_stream::try_stream; -use itertools::Itertools; +use risingwave_common::array::StreamChunk; use risingwave_common::catalog::{ColumnId, Schema, TableId}; use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{OwnedRow, Row}; @@ -31,7 +31,7 @@ use risingwave_connector::source::filesystem::opendal_source::{ }; use risingwave_connector::source::filesystem::OpendalFsSplit; use risingwave_connector::source::{ - BoxChunkedSourceStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData, StreamChunk, + BoxChunkedSourceStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData, }; use risingwave_connector::ConnectorParams; use risingwave_pb::plan_common::AdditionalColumnType; @@ -348,6 +348,7 @@ impl FsFetchExecutor { .await?; } + // TODO: ignore if it is user defined let chunk = chunk.project( (0..chunk.dimension()) .filter(|&idx| idx != partition_idx && idx != col_idx), diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index c2ba380076748..e7af40dbbf8e4 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -25,7 +25,7 @@ use risingwave_common::catalog::Schema; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_connector::source::{ BoxChunkedSourceStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitId, SplitImpl, - SplitMetaData, StreamChunk, + SplitMetaData }; use risingwave_source::source_desc::{FsSourceDesc, SourceDescBuilder}; use risingwave_storage::StateStore; @@ -406,10 +406,7 @@ impl FsSourceExecutor { } }, - Either::Right(StreamChunk { - chunk, - split_offset_mapping, - }) => { + Either::Right(chunk) => { if last_barrier_time.elapsed().as_millis() > max_wait_barrier_time_ms { // Exceeds the max wait barrier time, the source will be paused. Currently // we can guarantee the source is not paused since it received stream From e8ea63ffe40362d27fa85b12afed53d65a7742c5 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Mon, 15 Jan 2024 14:04:42 +0800 Subject: [PATCH 13/42] fix executors --- .../src/executor/source/fetch_executor.rs | 41 +++++--------- .../src/executor/source/fs_source_executor.rs | 14 ++++- src/stream/src/executor/source/mod.rs | 53 +++++++++++++++++++ .../src/executor/source/source_executor.rs | 19 +++++-- 4 files changed, 93 insertions(+), 34 deletions(-) diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index b08be03c80f50..859eac4226b6e 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -31,15 +31,15 @@ use risingwave_connector::source::filesystem::opendal_source::{ }; use risingwave_connector::source::filesystem::OpendalFsSplit; use risingwave_connector::source::{ - BoxChunkedSourceStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData, + BoxChunkedSourceStream, SourceContext, SourceCtrlOpts, SplitId, SplitImpl, SplitMetaData, }; use risingwave_connector::ConnectorParams; -use risingwave_pb::plan_common::AdditionalColumnType; use risingwave_source::source_desc::SourceDesc; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; use thiserror_ext::AsReport; +use super::{get_partition_offset_col_idx, prune_additional_cols}; use crate::executor::stream_reader::StreamReaderWithPause; use crate::executor::{ expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, @@ -196,22 +196,9 @@ impl FsFetchExecutor { .build() .map_err(StreamExecutorError::connector_error)?; - let (Some(partition_idx), Some(col_idx)) = ({ - let mut partition_idx = None; - let mut col_idx = None; - for (idx, column) in source_desc.columns.iter().enumerate() { - match additional_column_type { - AdditionalColumnType::Partition => { - partition_idx = Some(idx); - } - AdditionalColumnType::Offset => { - col_idx = Some(idx); - } - _ => (), - } - } - (partition_idx, col_idx) - }) else { + let (Some(partition_idx), Some(offset_idx)) = + get_partition_offset_col_idx(&source_desc.columns) + else { unreachable!("Partition and offset columns must be set."); }; @@ -319,13 +306,16 @@ impl FsFetchExecutor { } // StreamChunk from FsSourceReader, and the reader reads only one file. Either::Right(chunk) => { - let chunk_last_row = chunk + let (_, chunk_last_row) = chunk .rows() .last() .expect("The chunk should have at least one row."); - let split_id = - chunk_last_row.datum_at(partition_idx).unwrap().into_utf8(); - let offset = chunk_last_row.datum_at(col_idx).unwrap().into_utf8(); + let split_id: SplitId = chunk_last_row + .datum_at(partition_idx) + .unwrap() + .into_utf8() + .into(); + let offset = chunk_last_row.datum_at(offset_idx).unwrap().into_utf8(); let state = state_store_handler .get(split_id.clone()) @@ -348,12 +338,7 @@ impl FsFetchExecutor { .await?; } - // TODO: ignore if it is user defined - let chunk = chunk.project( - (0..chunk.dimension()) - .filter(|&idx| idx != partition_idx && idx != col_idx), - ); - + let chunk = prune_additional_cols(&chunk, partition_idx, offset_idx); yield Message::Chunk(chunk); } } diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index e7af40dbbf8e4..d51c4f7d069db 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -25,7 +25,7 @@ use risingwave_common::catalog::Schema; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_connector::source::{ BoxChunkedSourceStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitId, SplitImpl, - SplitMetaData + SplitMetaData, }; use risingwave_source::source_desc::{FsSourceDesc, SourceDescBuilder}; use risingwave_storage::StateStore; @@ -280,6 +280,12 @@ impl FsSourceExecutor { .build_fs_source_desc() .map_err(StreamExecutorError::connector_error)?; + let (Some(partition_idx), Some(offset_idx)) = + get_partition_offset_col_idx(&source_desc.columns) + else { + unreachable!("Partition and offset columns must be set."); + }; + // If the first barrier requires us to pause on startup, pause the stream. let start_with_paused = barrier.is_pause_on_startup(); @@ -407,6 +413,9 @@ impl FsSourceExecutor { }, Either::Right(chunk) => { + // TODO: confirm when split_offset_mapping is None + let split_offset_mapping = + get_split_offset_mapping_from_chunk(&chunk, partition_idx, offset_idx); if last_barrier_time.elapsed().as_millis() > max_wait_barrier_time_ms { // Exceeds the max wait barrier time, the source will be paused. Currently // we can guarantee the source is not paused since it received stream @@ -446,7 +455,10 @@ impl FsSourceExecutor { self.actor_ctx.id.to_string().as_str(), ]) .inc_by(chunk.cardinality() as u64); + + let chunk = prune_additional_cols(&chunk, partition_idx, offset_idx); yield Message::Chunk(chunk); + self.try_flush_data().await?; } } diff --git a/src/stream/src/executor/source/mod.rs b/src/stream/src/executor/source/mod.rs index 318581bb996f6..79b88f1abefaa 100644 --- a/src/stream/src/executor/source/mod.rs +++ b/src/stream/src/executor/source/mod.rs @@ -13,11 +13,18 @@ // limitations under the License. pub mod executor_core; +use std::collections::HashMap; + use await_tree::InstrumentAwait; pub use executor_core::StreamSourceCore; mod fs_source_executor; pub use fs_source_executor::*; +use itertools::Itertools; +use risingwave_common::array::StreamChunk; use risingwave_common::bail; +use risingwave_common::row::Row; +use risingwave_connector::source::{SourceColumnDesc, SplitId}; +use risingwave_pb::plan_common::AdditionalColumnType; pub use state_table_handler::*; pub mod fetch_executor; pub use fetch_executor::*; @@ -41,3 +48,49 @@ pub async fn barrier_to_message_stream(mut rx: UnboundedReceiver) { } bail!("barrier reader closed unexpectedly"); } + +pub fn get_split_offset_mapping_from_chunk( + chunk: &StreamChunk, + partition_idx: usize, + offset_idx: usize, +) -> Option> { + let mut split_offset_mapping = HashMap::new(); + for (_, row) in chunk.rows() { + let split_id: SplitId = row.datum_at(partition_idx).unwrap().into_utf8().into(); + let offset = row.datum_at(offset_idx).unwrap().into_utf8(); + split_offset_mapping.insert(split_id, offset.to_string()); + } + Some(split_offset_mapping) +} + +pub fn get_partition_offset_col_idx( + column_descs: &[SourceColumnDesc], +) -> (Option, Option) { + let mut partition_idx = None; + let mut offset_idx = None; + for (idx, column) in column_descs.iter().enumerate() { + match column.additional_column_type { + AdditionalColumnType::Partition => { + partition_idx = Some(idx); + } + AdditionalColumnType::Offset => { + offset_idx = Some(idx); + } + _ => (), + } + } + (partition_idx, offset_idx) +} + +pub fn prune_additional_cols( + chunk: &StreamChunk, + partition_idx: usize, + offset_idx: usize, +) -> StreamChunk { + // TODO: ignore if it is user defined + chunk.project( + &(0..chunk.dimension()) + .filter(|&idx| idx != partition_idx && idx != offset_idx) + .collect_vec(), + ) +} diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index cbd5c07cd45e5..60a746eb31dfd 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -23,7 +23,6 @@ use risingwave_common::metrics::GLOBAL_ERROR_METRICS; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_connector::source::{ BoxChunkedSourceStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitMetaData, - StreamChunk, }; use risingwave_connector::ConnectorParams; use risingwave_source::source_desc::{SourceDesc, SourceDescBuilder}; @@ -374,6 +373,12 @@ impl SourceExecutor { .build() .map_err(StreamExecutorError::connector_error)?; + let (Some(partition_idx), Some(offset_idx)) = + get_partition_offset_col_idx(&source_desc.columns) + else { + unreachable!("Partition and offset columns must be set."); + }; + let mut boot_state = Vec::default(); if let Some(mutation) = barrier.mutation.as_ref() { match mutation.as_ref() { @@ -541,10 +546,13 @@ impl SourceExecutor { } }, - Either::Right(StreamChunk { - chunk, - split_offset_mapping, - }) => { + Either::Right(chunk) => { + // TODO: confirm when split_offset_mapping is None + let split_offset_mapping = get_split_offset_mapping_from_chunk( + &chunk, + partition_idx, + offset_idx, + ); if last_barrier_time.elapsed().as_millis() > max_wait_barrier_time_ms { // Exceeds the max wait barrier time, the source will be paused. // Currently we can guarantee the @@ -605,6 +613,7 @@ impl SourceExecutor { .collect::>(), ) .inc_by(chunk.cardinality() as u64); + let chunk = prune_additional_cols(&chunk, partition_idx, offset_idx); yield Message::Chunk(chunk); self.try_flush_data().await?; } From 0ffdf3b09a73563215e41a14eabcc2c5ea0327d8 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Mon, 15 Jan 2024 14:04:51 +0800 Subject: [PATCH 14/42] fix misc --- src/batch/src/executor/source.rs | 2 +- src/stream/src/executor/stream_reader.rs | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/batch/src/executor/source.rs b/src/batch/src/executor/source.rs index 2714d5335b906..2471d8f5d5ab9 100644 --- a/src/batch/src/executor/source.rs +++ b/src/batch/src/executor/source.rs @@ -156,7 +156,7 @@ impl SourceExecutor { #[for_await] for chunk in stream { let chunk = chunk.map_err(BatchError::connector)?; - let data_chunk = covert_stream_chunk_to_batch_chunk(chunk.chunk)?; + let data_chunk = covert_stream_chunk_to_batch_chunk(chunk)?; if data_chunk.capacity() > 0 { yield data_chunk; } diff --git a/src/stream/src/executor/stream_reader.rs b/src/stream/src/executor/stream_reader.rs index 72211b86fb0d9..b59c2172734e0 100644 --- a/src/stream/src/executor/stream_reader.rs +++ b/src/stream/src/executor/stream_reader.rs @@ -143,7 +143,6 @@ mod tests { use futures::{pin_mut, FutureExt}; use risingwave_common::array::StreamChunk; use risingwave_common::transaction::transaction_id::TxnId; - use risingwave_connector::source::StreamChunk; use risingwave_source::TableDmlHandle; use tokio::sync::mpsc; From a544eb0142a3c275687b5bace00c2e4997f59604 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Mon, 15 Jan 2024 14:11:23 +0800 Subject: [PATCH 15/42] const additional column name --- src/common/src/catalog/mod.rs | 4 ++++ src/source/src/source_desc.rs | 30 ++++++++++++++++-------------- 2 files changed, 20 insertions(+), 14 deletions(-) diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index 14d44c9c0bd2d..b356b0bbd4864 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -114,6 +114,10 @@ pub fn row_id_column_desc() -> ColumnDesc { ColumnDesc::named(ROWID_PREFIX, ROW_ID_COLUMN_ID, DataType::Serial) } +/// The additional columns for partition/offset. +pub const ADDITION_PARTITION_COLUMN_NAME: &str = "_rw_addition_partition"; +pub const ADDITION_OFFSET_COLUMN_NAME: &str = "_rw_addition_offset"; + pub const OFFSET_COLUMN_NAME: &str = "_rw_offset"; // The number of columns output by the cdc source job diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index 369ac9930a96c..0635514c1dad4 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -15,7 +15,7 @@ use std::collections::HashMap; use std::sync::Arc; -use risingwave_common::catalog::{ColumnDesc, ColumnId}; +use risingwave_common::catalog::{ColumnDesc, ColumnId, ADDITION_PARTITION_COLUMN_NAME}; use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; use risingwave_common::types::DataType; @@ -95,7 +95,7 @@ impl SourceDescBuilder { ColumnCatalog { column_desc: Some( ColumnDesc::named_with_additional_column( - "_rw_partition", + ADDITION_PARTITION_COLUMN_NAME, last_column_id.next(), DataType::Varchar, AdditionalColumnType::Partition, @@ -107,7 +107,7 @@ impl SourceDescBuilder { ColumnCatalog { column_desc: Some( ColumnDesc::named_with_additional_column( - "_rw_offset", + ADDITION_PARTITION_COLUMN_NAME, last_column_id.next().next(), DataType::Varchar, AdditionalColumnType::Offset, @@ -123,20 +123,22 @@ impl SourceDescBuilder { .iter() .chain(additional_columns.iter()) .filter_map(|c| { - if match c.column_desc.as_ref().unwrap().get_additional_column_type() { - Ok(AdditionalColumnType::Partition) => { - std::mem::replace(&mut columns_exist[0], true) - } - Ok(AdditionalColumnType::Offset) => { - std::mem::replace(&mut columns_exist[1], true) - } - _ => false, - } { - None - } else { + let addition_col_existed = + match c.column_desc.as_ref().unwrap().get_additional_column_type() { + Ok(AdditionalColumnType::Partition) => { + std::mem::replace(&mut columns_exist[0], true) + } + Ok(AdditionalColumnType::Offset) => { + std::mem::replace(&mut columns_exist[1], true) + } + _ => false, + }; + if !addition_col_existed { Some(SourceColumnDesc::from(&ColumnDesc::from( c.column_desc.as_ref().unwrap(), ))) + } else { + None } }) .collect(); From 8c1f930bbf2af91aefde25a5bfa774ab38b99e49 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Mon, 15 Jan 2024 14:19:21 +0800 Subject: [PATCH 16/42] refactor fetch_executor --- .../src/executor/source/fetch_executor.rs | 63 ++++++++++--------- src/stream/src/executor/source/mod.rs | 2 +- 2 files changed, 33 insertions(+), 32 deletions(-) diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 859eac4226b6e..236ec51ca16b3 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -31,7 +31,7 @@ use risingwave_connector::source::filesystem::opendal_source::{ }; use risingwave_connector::source::filesystem::OpendalFsSplit; use risingwave_connector::source::{ - BoxChunkedSourceStream, SourceContext, SourceCtrlOpts, SplitId, SplitImpl, SplitMetaData, + BoxChunkedSourceStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData, }; use risingwave_connector::ConnectorParams; use risingwave_source::source_desc::SourceDesc; @@ -39,7 +39,9 @@ use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; use thiserror_ext::AsReport; -use super::{get_partition_offset_col_idx, prune_additional_cols}; +use super::{ + get_partition_offset_col_idx, get_split_offset_mapping_from_chunk, prune_additional_cols, +}; use crate::executor::stream_reader::StreamReaderWithPause; use crate::executor::{ expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, @@ -306,36 +308,35 @@ impl FsFetchExecutor { } // StreamChunk from FsSourceReader, and the reader reads only one file. Either::Right(chunk) => { - let (_, chunk_last_row) = chunk - .rows() - .last() - .expect("The chunk should have at least one row."); - let split_id: SplitId = chunk_last_row - .datum_at(partition_idx) - .unwrap() - .into_utf8() - .into(); - let offset = chunk_last_row.datum_at(offset_idx).unwrap().into_utf8(); - - let state = state_store_handler - .get(split_id.clone()) - .await? - .expect("The fs_split should be in the state table."); - let fs_split = match state.datum_at(1) { - Some(ScalarRefImpl::Jsonb(jsonb_ref)) => { - OpendalFsSplit::::restore_from_json( - jsonb_ref.to_owned_scalar(), - )? + let mapping = get_split_offset_mapping_from_chunk( + &chunk, + partition_idx, + offset_idx, + ) + .unwrap(); + debug_assert_eq!(mapping.len(), 1); + if let Some((split_id, offset)) = mapping.into_iter().next() { + let row = state_store_handler + .get(split_id.clone()) + .await? + .expect("The fs_split should be in the state table."); + let fs_split = match row.datum_at(1) { + Some(ScalarRefImpl::Jsonb(jsonb_ref)) => { + OpendalFsSplit::::restore_from_json( + jsonb_ref.to_owned_scalar(), + )? + } + _ => unreachable!(), + }; + + if offset.parse::().unwrap() >= fs_split.size { + splits_on_fetch -= 1; + state_store_handler.delete(split_id).await?; + } else { + state_store_handler + .set(split_id, fs_split.encode_to_json()) + .await?; } - _ => unreachable!(), - }; - if offset.parse::().unwrap() >= fs_split.size { - splits_on_fetch -= 1; - state_store_handler.delete(split_id).await?; - } else { - state_store_handler - .set(split_id, fs_split.encode_to_json()) - .await?; } let chunk = prune_additional_cols(&chunk, partition_idx, offset_idx); diff --git a/src/stream/src/executor/source/mod.rs b/src/stream/src/executor/source/mod.rs index 79b88f1abefaa..972d099bb1652 100644 --- a/src/stream/src/executor/source/mod.rs +++ b/src/stream/src/executor/source/mod.rs @@ -56,7 +56,7 @@ pub fn get_split_offset_mapping_from_chunk( ) -> Option> { let mut split_offset_mapping = HashMap::new(); for (_, row) in chunk.rows() { - let split_id: SplitId = row.datum_at(partition_idx).unwrap().into_utf8().into(); + let split_id = row.datum_at(partition_idx).unwrap().into_utf8().into(); let offset = row.datum_at(offset_idx).unwrap().into_utf8(); split_offset_mapping.insert(split_id, offset.to_string()); } From 633c71057373af97d941541e47b71220fbb386c6 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Mon, 15 Jan 2024 14:21:42 +0800 Subject: [PATCH 17/42] simplify import --- src/stream/src/executor/source/fetch_executor.rs | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 236ec51ca16b3..d1d01a30ca8fc 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -39,15 +39,8 @@ use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; use thiserror_ext::AsReport; -use super::{ - get_partition_offset_col_idx, get_split_offset_mapping_from_chunk, prune_additional_cols, -}; use crate::executor::stream_reader::StreamReaderWithPause; -use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, Mutation, PkIndicesRef, SourceStateTableHandler, StreamExecutorError, - StreamExecutorResult, StreamSourceCore, -}; +use crate::executor::*; const SPLIT_BATCH_SIZE: usize = 1000; From d5203d289a0e9aca593730cfd2ffff5b0ceef59b Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Tue, 16 Jan 2024 19:57:02 +0800 Subject: [PATCH 18/42] fix comments --- src/common/src/catalog/mod.rs | 4 -- src/connector/benches/nexmark_integration.rs | 4 +- src/connector/src/parser/mod.rs | 10 +-- src/connector/src/source/base.rs | 6 +- src/connector/src/source/cdc/source/reader.rs | 6 +- .../src/source/datagen/source/reader.rs | 6 +- .../opendal_source/opendal_reader.rs | 4 +- .../src/source/filesystem/s3/source/reader.rs | 4 +- .../src/source/google_pubsub/source/reader.rs | 4 +- .../src/source/kafka/source/reader.rs | 6 +- .../src/source/kinesis/source/reader.rs | 4 +- .../src/source/nats/source/reader.rs | 4 +- .../src/source/nexmark/source/reader.rs | 4 +- .../src/source/pulsar/source/reader.rs | 6 +- src/connector/src/source/test_source.rs | 8 +-- src/source/src/connector_source.rs | 4 +- src/source/src/fs_connector_source.rs | 4 +- src/source/src/source_desc.rs | 61 +++++++++---------- .../src/executor/source/fetch_executor.rs | 18 +++--- .../src/executor/source/fs_source_executor.rs | 11 ++-- src/stream/src/executor/source/mod.rs | 18 +++--- .../src/executor/source/source_executor.rs | 16 ++--- 22 files changed, 98 insertions(+), 114 deletions(-) diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index b356b0bbd4864..14d44c9c0bd2d 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -114,10 +114,6 @@ pub fn row_id_column_desc() -> ColumnDesc { ColumnDesc::named(ROWID_PREFIX, ROW_ID_COLUMN_ID, DataType::Serial) } -/// The additional columns for partition/offset. -pub const ADDITION_PARTITION_COLUMN_NAME: &str = "_rw_addition_partition"; -pub const ADDITION_OFFSET_COLUMN_NAME: &str = "_rw_addition_offset"; - pub const OFFSET_COLUMN_NAME: &str = "_rw_offset"; // The number of columns output by the cdc source job diff --git a/src/connector/benches/nexmark_integration.rs b/src/connector/benches/nexmark_integration.rs index a841eae1db475..e6388ed4b0d25 100644 --- a/src/connector/benches/nexmark_integration.rs +++ b/src/connector/benches/nexmark_integration.rs @@ -26,7 +26,7 @@ use risingwave_connector::parser::{ ByteStreamSourceParser, JsonParser, SourceParserIntoStreamExt, SpecificParserConfig, }; use risingwave_connector::source::{ - BoxChunkedSourceStream, BoxSourceStream, SourceColumnDesc, SourceMessage, SourceMeta, + BoxChunkSourceStream, BoxSourceStream, SourceColumnDesc, SourceMessage, SourceMeta, }; use tracing::Level; use tracing_subscriber::prelude::*; @@ -91,7 +91,7 @@ fn make_parser() -> impl ByteStreamSourceParser { } fn make_stream_iter() -> impl Iterator { - let mut stream: BoxChunkedSourceStream = make_parser().into_stream(make_data_stream()).boxed(); + let mut stream: BoxChunkSourceStream = make_parser().into_stream(make_data_stream()).boxed(); std::iter::from_fn(move || { stream diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 1f94e21e21a2c..d05a63f1b4eed 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -53,8 +53,8 @@ use crate::parser::util::{extract_headers_from_meta, extreact_timestamp_from_met use crate::schema::schema_registry::SchemaRegistryAuth; use crate::source::monitor::GLOBAL_SOURCE_METRICS; use crate::source::{ - extract_source_struct, BoxSourceStream, ChunkedSourceStream, SourceColumnDesc, - SourceColumnType, SourceContext, SourceContextRef, SourceEncode, SourceFormat, SourceMeta, + extract_source_struct, BoxSourceStream, ChunkSourceStream, SourceColumnDesc, SourceColumnType, + SourceContext, SourceContextRef, SourceEncode, SourceFormat, SourceMeta, }; pub mod additional_columns; @@ -550,8 +550,8 @@ impl P { /// /// # Returns /// - /// A [`ChunkedSourceStream`] which is a stream of parsed messages. - pub fn into_stream(self, data_stream: BoxSourceStream) -> impl ChunkedSourceStream { + /// A [`ChunkSourceStream`] which is a stream of parsed messages. + pub fn into_stream(self, data_stream: BoxSourceStream) -> impl ChunkSourceStream { // Enable tracing to provide more information for parsing failures. let source_info = self.source_ctx().source_info; @@ -779,7 +779,7 @@ pub enum ByteStreamSourceParserImpl { CanalJson(CanalJsonParser), } -pub type ParsedStreamImpl = impl ChunkedSourceStream + Unpin; +pub type ParsedStreamImpl = impl ChunkSourceStream + Unpin; impl ByteStreamSourceParserImpl { /// Converts this parser into a stream of [`StreamChunk`]. diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 5a8da625d31c7..3f027b2dddfe9 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -343,8 +343,8 @@ pub fn extract_source_struct(info: &PbStreamSourceInfo) -> Result pub type BoxSourceStream = BoxStream<'static, Result>>; -pub trait ChunkedSourceStream = Stream> + Send + 'static; -pub type BoxChunkedSourceStream = BoxStream<'static, Result>; +pub trait ChunkSourceStream = Stream> + Send + 'static; +pub type BoxChunkSourceStream = BoxStream<'static, Result>; pub type BoxTryStream = BoxStream<'static, Result>; /// [`SplitReader`] is a new abstraction of the external connector read interface which is @@ -363,7 +363,7 @@ pub trait SplitReader: Sized + Send { columns: Option>, ) -> Result; - fn into_stream(self) -> BoxChunkedSourceStream; + fn into_stream(self) -> BoxChunkSourceStream; } for_all_sources!(impl_connector_properties); diff --git a/src/connector/src/source/cdc/source/reader.rs b/src/connector/src/source/cdc/source/reader.rs index 301c9b75c05d0..1e1bf98afede6 100644 --- a/src/connector/src/source/cdc/source/reader.rs +++ b/src/connector/src/source/cdc/source/reader.rs @@ -31,8 +31,8 @@ use crate::parser::ParserConfig; use crate::source::base::SourceMessage; use crate::source::cdc::{CdcProperties, CdcSourceType, CdcSourceTypeTrait, DebeziumCdcSplit}; use crate::source::{ - into_chunk_stream, BoxChunkedSourceStream, Column, CommonSplitReader, SourceContextRef, - SplitId, SplitMetaData, SplitReader, + into_chunk_stream, BoxChunkSourceStream, Column, CommonSplitReader, SourceContextRef, SplitId, + SplitMetaData, SplitReader, }; pub struct CdcSplitReader { @@ -186,7 +186,7 @@ impl SplitReader for CdcSplitReader { } } - fn into_stream(self) -> BoxChunkedSourceStream { + fn into_stream(self) -> BoxChunkSourceStream { 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/datagen/source/reader.rs b/src/connector/src/source/datagen/source/reader.rs index 2c823d8bce729..46112395b298a 100644 --- a/src/connector/src/source/datagen/source/reader.rs +++ b/src/connector/src/source/datagen/source/reader.rs @@ -25,8 +25,8 @@ 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::{ - into_chunk_stream, BoxChunkedSourceStream, Column, CommonSplitReader, DataType, - SourceContextRef, SourceMessage, SplitId, SplitMetaData, SplitReader, + into_chunk_stream, BoxChunkSourceStream, Column, CommonSplitReader, DataType, SourceContextRef, + SourceMessage, SplitId, SplitMetaData, SplitReader, }; pub struct DatagenSplitReader { @@ -138,7 +138,7 @@ impl SplitReader for DatagenSplitReader { }) } - fn into_stream(self) -> BoxChunkedSourceStream { + fn into_stream(self) -> BoxChunkSourceStream { // Will buffer at most 4 event chunks. const BUFFER_SIZE: usize = 4; // spawn_data_generation_stream(self.generator.into_native_stream(), BUFFER_SIZE).boxed() diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs index 5efd0995a9869..7096df9d684c4 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -28,7 +28,7 @@ use crate::parser::{ByteStreamSourceParserImpl, ParserConfig}; use crate::source::filesystem::nd_streaming::need_nd_streaming; use crate::source::filesystem::{nd_streaming, OpendalFsSplit}; use crate::source::{ - BoxChunkedSourceStream, Column, SourceContextRef, SourceMessage, SourceMeta, SplitMetaData, + BoxChunkSourceStream, Column, SourceContextRef, SourceMessage, SourceMeta, SplitMetaData, SplitReader, }; @@ -63,7 +63,7 @@ impl SplitReader for OpendalReader { Ok(opendal_reader) } - fn into_stream(self) -> BoxChunkedSourceStream { + fn into_stream(self) -> BoxChunkSourceStream { self.into_chunk_stream() } } diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index d706042cf4780..c223febd1cf56 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -39,7 +39,7 @@ use crate::source::filesystem::file_common::FsSplit; use crate::source::filesystem::nd_streaming; use crate::source::filesystem::nd_streaming::need_nd_streaming; use crate::source::filesystem::s3::S3Properties; -use crate::source::{BoxChunkedSourceStream, Column, SourceContextRef, SourceMessage, SourceMeta}; +use crate::source::{BoxChunkSourceStream, Column, SourceContextRef, SourceMessage, SourceMeta}; const MAX_CHANNEL_BUFFER_SIZE: usize = 2048; const STREAM_READER_CAPACITY: usize = 4096; @@ -197,7 +197,7 @@ impl SplitReader for S3FileReader { Ok(s3_file_reader) } - fn into_stream(self) -> BoxChunkedSourceStream { + fn into_stream(self) -> BoxChunkSourceStream { self.into_chunk_stream() } } diff --git a/src/connector/src/source/google_pubsub/source/reader.rs b/src/connector/src/source/google_pubsub/source/reader.rs index 62c652f78d3a2..d18fcb0be258b 100644 --- a/src/connector/src/source/google_pubsub/source/reader.rs +++ b/src/connector/src/source/google_pubsub/source/reader.rs @@ -25,7 +25,7 @@ use super::TaggedReceivedMessage; use crate::parser::ParserConfig; use crate::source::google_pubsub::{PubsubProperties, PubsubSplit}; use crate::source::{ - into_chunk_stream, BoxChunkedSourceStream, Column, CommonSplitReader, SourceContextRef, + into_chunk_stream, BoxChunkSourceStream, Column, CommonSplitReader, SourceContextRef, SourceMessage, SplitId, SplitMetaData, SplitReader, }; @@ -164,7 +164,7 @@ impl SplitReader for PubsubSplitReader { }) } - fn into_stream(self) -> BoxChunkedSourceStream { + fn into_stream(self) -> BoxChunkSourceStream { 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 b26f3a57a55d2..28c9eb3df0ae1 100644 --- a/src/connector/src/source/kafka/source/reader.rs +++ b/src/connector/src/source/kafka/source/reader.rs @@ -33,8 +33,8 @@ use crate::source::kafka::{ KafkaProperties, KafkaSplit, PrivateLinkConsumerContext, KAFKA_ISOLATION_LEVEL, }; use crate::source::{ - into_chunk_stream, BoxChunkedSourceStream, Column, CommonSplitReader, SourceContextRef, - SplitId, SplitMetaData, SplitReader, + into_chunk_stream, BoxChunkSourceStream, Column, CommonSplitReader, SourceContextRef, SplitId, + SplitMetaData, SplitReader, }; pub struct KafkaSplitReader { @@ -145,7 +145,7 @@ impl SplitReader for KafkaSplitReader { }) } - fn into_stream(self) -> BoxChunkedSourceStream { + fn into_stream(self) -> BoxChunkSourceStream { 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/kinesis/source/reader.rs b/src/connector/src/source/kinesis/source/reader.rs index 67e79f8144be6..51b3c77710410 100644 --- a/src/connector/src/source/kinesis/source/reader.rs +++ b/src/connector/src/source/kinesis/source/reader.rs @@ -29,7 +29,7 @@ use crate::source::kinesis::source::message::from_kinesis_record; use crate::source::kinesis::split::{KinesisOffset, KinesisSplit}; use crate::source::kinesis::KinesisProperties; use crate::source::{ - into_chunk_stream, BoxChunkedSourceStream, Column, CommonSplitReader, SourceContextRef, + into_chunk_stream, BoxChunkSourceStream, Column, CommonSplitReader, SourceContextRef, SourceMessage, SplitId, SplitMetaData, SplitReader, }; @@ -113,7 +113,7 @@ impl SplitReader for KinesisSplitReader { }) } - fn into_stream(self) -> BoxChunkedSourceStream { + fn into_stream(self) -> BoxChunkSourceStream { 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/nats/source/reader.rs b/src/connector/src/source/nats/source/reader.rs index a9ddd9ea79c83..20a8f9c0dbc0b 100644 --- a/src/connector/src/source/nats/source/reader.rs +++ b/src/connector/src/source/nats/source/reader.rs @@ -24,7 +24,7 @@ use crate::parser::ParserConfig; use crate::source::common::{into_chunk_stream, CommonSplitReader}; use crate::source::nats::NatsProperties; use crate::source::{ - BoxChunkedSourceStream, Column, SourceContextRef, SourceMessage, SplitId, SplitReader, + BoxChunkSourceStream, Column, SourceContextRef, SourceMessage, SplitId, SplitReader, }; pub struct NatsSplitReader { @@ -93,7 +93,7 @@ impl SplitReader for NatsSplitReader { }) } - fn into_stream(self) -> BoxChunkedSourceStream { + fn into_stream(self) -> BoxChunkSourceStream { 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/nexmark/source/reader.rs b/src/connector/src/source/nexmark/source/reader.rs index 793878ce2a4c4..ed6a6b40eaad3 100644 --- a/src/connector/src/source/nexmark/source/reader.rs +++ b/src/connector/src/source/nexmark/source/reader.rs @@ -35,7 +35,7 @@ use crate::source::nexmark::source::combined_event::{ }; use crate::source::nexmark::{NexmarkProperties, NexmarkSplit}; use crate::source::{ - BoxChunkedSourceStream, Column, SourceContextRef, SplitId, SplitMetaData, SplitReader, + BoxChunkSourceStream, Column, SourceContextRef, SplitId, SplitMetaData, SplitReader, }; #[derive(Debug)] @@ -106,7 +106,7 @@ impl SplitReader for NexmarkSplitReader { }) } - fn into_stream(self) -> BoxChunkedSourceStream { + fn into_stream(self) -> BoxChunkSourceStream { 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 split_id = self.split_id.clone(); diff --git a/src/connector/src/source/pulsar/source/reader.rs b/src/connector/src/source/pulsar/source/reader.rs index 90968b51d0bc4..07265f1e4acfc 100644 --- a/src/connector/src/source/pulsar/source/reader.rs +++ b/src/connector/src/source/pulsar/source/reader.rs @@ -37,7 +37,7 @@ use crate::parser::ParserConfig; use crate::source::pulsar::split::PulsarSplit; use crate::source::pulsar::{PulsarEnumeratorOffset, PulsarProperties}; use crate::source::{ - into_chunk_stream, BoxChunkedSourceStream, Column, CommonSplitReader, SourceContextRef, + into_chunk_stream, BoxChunkSourceStream, Column, CommonSplitReader, SourceContextRef, SourceMessage, SplitId, SplitMetaData, SplitReader, }; @@ -83,7 +83,7 @@ impl SplitReader for PulsarSplitReader { } } - fn into_stream(self) -> BoxChunkedSourceStream { + fn into_stream(self) -> BoxChunkSourceStream { match self { Self::Broker(reader) => { let (parser_config, source_context) = @@ -234,7 +234,7 @@ impl SplitReader for PulsarBrokerReader { }) } - fn into_stream(self) -> BoxChunkedSourceStream { + fn into_stream(self) -> BoxChunkSourceStream { 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/test_source.rs b/src/connector/src/source/test_source.rs index 066143eb928d2..e0b901ddbf253 100644 --- a/src/connector/src/source/test_source.rs +++ b/src/connector/src/source/test_source.rs @@ -24,7 +24,7 @@ use with_options::WithOptions; use crate::parser::ParserConfig; use crate::source::{ - BoxChunkedSourceStream, Column, SourceContextRef, SourceEnumeratorContextRef, SourceProperties, + BoxChunkSourceStream, Column, SourceContextRef, SourceEnumeratorContextRef, SourceProperties, SplitEnumerator, SplitId, SplitMetaData, SplitReader, TryFromHashmap, }; @@ -44,7 +44,7 @@ pub type BoxIntoSourceStream = Box< ParserConfig, SourceContextRef, Option>, - ) -> BoxChunkedSourceStream + ) -> BoxChunkSourceStream + Send + 'static, >; @@ -68,7 +68,7 @@ impl BoxSource { ParserConfig, SourceContextRef, Option>, - ) -> BoxChunkedSourceStream + ) -> BoxChunkSourceStream + Send + 'static, ) -> BoxSource { @@ -218,7 +218,7 @@ impl SplitReader for TestSourceSplitReader { }) } - fn into_stream(self) -> BoxChunkedSourceStream { + fn into_stream(self) -> BoxChunkSourceStream { (get_registry() .box_source .lock() diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 23d25e6cde705..60d7e6f55ad33 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -34,7 +34,7 @@ use risingwave_connector::source::filesystem::opendal_source::{ }; use risingwave_connector::source::filesystem::FsPageItem; use risingwave_connector::source::{ - create_split_reader, BoxChunkedSourceStream, BoxTryStream, Column, ConnectorProperties, + create_split_reader, BoxChunkSourceStream, BoxTryStream, Column, ConnectorProperties, ConnectorState, FsFilterCtrlCtx, SourceColumnDesc, SourceContext, SplitReader, }; use tokio::time; @@ -117,7 +117,7 @@ impl ConnectorSource { state: ConnectorState, column_ids: Vec, source_ctx: Arc, - ) -> Result { + ) -> Result { let Some(splits) = state else { return Ok(pending().boxed()); }; diff --git a/src/source/src/fs_connector_source.rs b/src/source/src/fs_connector_source.rs index f213e140b49f2..665a646a34e6d 100644 --- a/src/source/src/fs_connector_source.rs +++ b/src/source/src/fs_connector_source.rs @@ -26,7 +26,7 @@ use risingwave_common::error::Result; use risingwave_connector::dispatch_source_prop; use risingwave_connector::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; use risingwave_connector::source::{ - create_split_reader, BoxChunkedSourceStream, ConnectorProperties, ConnectorState, + create_split_reader, BoxChunkSourceStream, ConnectorProperties, ConnectorState, SourceColumnDesc, SourceContext, SplitReader, }; @@ -81,7 +81,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)?; diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index 0635514c1dad4..d80fb5a4146b5 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -15,16 +15,18 @@ use std::collections::HashMap; use std::sync::Arc; -use risingwave_common::catalog::{ColumnDesc, ColumnId, ADDITION_PARTITION_COLUMN_NAME}; +use risingwave_common::catalog::{ColumnDesc, ColumnId}; use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; -use risingwave_common::types::DataType; use risingwave_connector::parser::{EncodingProperties, ProtocolProperties, SpecificParserConfig}; use risingwave_connector::source::monitor::SourceMetrics; -use risingwave_connector::source::{SourceColumnDesc, SourceColumnType}; +use risingwave_connector::source::{ + get_connector_compatible_additional_columns, SourceColumnDesc, SourceColumnType, + UPSTREAM_SOURCE_KEY, +}; use risingwave_connector::ConnectorParams; use risingwave_pb::catalog::PbStreamSourceInfo; -use risingwave_pb::plan_common::{AdditionalColumnType, ColumnCatalog, PbColumnCatalog}; +use risingwave_pb::plan_common::{AdditionalColumnType, PbColumnCatalog}; use crate::connector_source::ConnectorSource; use crate::fs_connector_source::FsConnectorSource; @@ -85,38 +87,33 @@ impl SourceDescBuilder { fn column_catalogs_to_source_column_descs(&self) -> Vec { let mut columns_exist = [false; 2]; - let last_column_id = self + let mut last_column_id = self .columns .last() .map(|c| c.column_desc.as_ref().unwrap().column_id.into()) .unwrap_or(ColumnId::new(0)); + let connector_name = self + .with_properties + .get(UPSTREAM_SOURCE_KEY) + .map(|s| s.to_lowercase()) + .unwrap(); - let additional_columns = [ - ColumnCatalog { - column_desc: Some( - ColumnDesc::named_with_additional_column( - ADDITION_PARTITION_COLUMN_NAME, - last_column_id.next(), - DataType::Varchar, - AdditionalColumnType::Partition, - ) - .to_protobuf(), - ), - is_hidden: true, - }, - ColumnCatalog { - column_desc: Some( - ColumnDesc::named_with_additional_column( - ADDITION_PARTITION_COLUMN_NAME, - last_column_id.next().next(), - DataType::Varchar, - AdditionalColumnType::Offset, - ) - .to_protobuf(), - ), - is_hidden: true, - }, - ]; + let additional_columns: Vec<_> = + match get_connector_compatible_additional_columns(&connector_name) { + Some(col_list) => ["partition", "file", "offset"] + .into_iter() + .filter_map(|key_name| { + let col_name = format!("_rw_{}_{}", connector_name, key_name); + last_column_id = last_column_id.next(); + col_list.iter().find_map(|(n, f)| { + (key_name == *n).then_some(f(last_column_id, &col_name).to_protobuf()) + }) + }) + .collect(), + _ => Vec::new(), + }; + + debug_assert_eq!(additional_columns.len(), 2); let mut columns: Vec<_> = self .columns @@ -125,7 +122,7 @@ impl SourceDescBuilder { .filter_map(|c| { let addition_col_existed = match c.column_desc.as_ref().unwrap().get_additional_column_type() { - Ok(AdditionalColumnType::Partition) => { + Ok(AdditionalColumnType::Partition | AdditionalColumnType::Filename) => { std::mem::replace(&mut columns_exist[0], true) } Ok(AdditionalColumnType::Offset) => { diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index d1d01a30ca8fc..1c7419cf23dfa 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -31,7 +31,7 @@ use risingwave_connector::source::filesystem::opendal_source::{ }; use risingwave_connector::source::filesystem::OpendalFsSplit; use risingwave_connector::source::{ - BoxChunkedSourceStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData, + BoxChunkSourceStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData, }; use risingwave_connector::ConnectorParams; use risingwave_source::source_desc::SourceDesc; @@ -155,7 +155,7 @@ impl FsFetchExecutor { source_ctx: SourceContext, source_desc: &SourceDesc, batch: SplitBatch, - ) -> StreamExecutorResult { + ) -> StreamExecutorResult { source_desc .source .stream_reader(batch, column_ids, Arc::new(source_ctx)) @@ -191,8 +191,7 @@ impl FsFetchExecutor { .build() .map_err(StreamExecutorError::connector_error)?; - let (Some(partition_idx), Some(offset_idx)) = - get_partition_offset_col_idx(&source_desc.columns) + let (Some(split_idx), Some(offset_idx)) = get_split_offset_col_idx(&source_desc.columns) else { unreachable!("Partition and offset columns must be set."); }; @@ -301,12 +300,9 @@ impl FsFetchExecutor { } // StreamChunk from FsSourceReader, and the reader reads only one file. Either::Right(chunk) => { - let mapping = get_split_offset_mapping_from_chunk( - &chunk, - partition_idx, - offset_idx, - ) - .unwrap(); + let mapping = + get_split_offset_mapping_from_chunk(&chunk, split_idx, offset_idx) + .unwrap(); debug_assert_eq!(mapping.len(), 1); if let Some((split_id, offset)) = mapping.into_iter().next() { let row = state_store_handler @@ -332,7 +328,7 @@ impl FsFetchExecutor { } } - let chunk = prune_additional_cols(&chunk, partition_idx, offset_idx); + let chunk = prune_additional_cols(&chunk, split_idx, offset_idx); yield Message::Chunk(chunk); } } diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index d51c4f7d069db..8686e69bf4378 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -24,7 +24,7 @@ use futures_async_stream::try_stream; use risingwave_common::catalog::Schema; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_connector::source::{ - BoxChunkedSourceStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitId, SplitImpl, + BoxChunkSourceStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitId, SplitImpl, SplitMetaData, }; use risingwave_source::source_desc::{FsSourceDesc, SourceDescBuilder}; @@ -90,7 +90,7 @@ impl FsSourceExecutor { &mut self, source_desc: &FsSourceDesc, state: ConnectorState, - ) -> StreamExecutorResult { + ) -> StreamExecutorResult { let column_ids = source_desc .columns .iter() @@ -280,8 +280,7 @@ impl FsSourceExecutor { .build_fs_source_desc() .map_err(StreamExecutorError::connector_error)?; - let (Some(partition_idx), Some(offset_idx)) = - get_partition_offset_col_idx(&source_desc.columns) + let (Some(split_idx), Some(offset_idx)) = get_split_offset_col_idx(&source_desc.columns) else { unreachable!("Partition and offset columns must be set."); }; @@ -415,7 +414,7 @@ impl FsSourceExecutor { Either::Right(chunk) => { // TODO: confirm when split_offset_mapping is None let split_offset_mapping = - get_split_offset_mapping_from_chunk(&chunk, partition_idx, offset_idx); + get_split_offset_mapping_from_chunk(&chunk, split_idx, offset_idx); if last_barrier_time.elapsed().as_millis() > max_wait_barrier_time_ms { // Exceeds the max wait barrier time, the source will be paused. Currently // we can guarantee the source is not paused since it received stream @@ -456,7 +455,7 @@ impl FsSourceExecutor { ]) .inc_by(chunk.cardinality() as u64); - let chunk = prune_additional_cols(&chunk, partition_idx, offset_idx); + let chunk = prune_additional_cols(&chunk, split_idx, offset_idx); yield Message::Chunk(chunk); self.try_flush_data().await?; diff --git a/src/stream/src/executor/source/mod.rs b/src/stream/src/executor/source/mod.rs index 972d099bb1652..c4eb83eab6820 100644 --- a/src/stream/src/executor/source/mod.rs +++ b/src/stream/src/executor/source/mod.rs @@ -51,27 +51,27 @@ pub async fn barrier_to_message_stream(mut rx: UnboundedReceiver) { pub fn get_split_offset_mapping_from_chunk( chunk: &StreamChunk, - partition_idx: usize, + split_idx: usize, offset_idx: usize, ) -> Option> { let mut split_offset_mapping = HashMap::new(); for (_, row) in chunk.rows() { - let split_id = row.datum_at(partition_idx).unwrap().into_utf8().into(); + let split_id = row.datum_at(split_idx).unwrap().into_utf8().into(); let offset = row.datum_at(offset_idx).unwrap().into_utf8(); split_offset_mapping.insert(split_id, offset.to_string()); } Some(split_offset_mapping) } -pub fn get_partition_offset_col_idx( +pub fn get_split_offset_col_idx( column_descs: &[SourceColumnDesc], ) -> (Option, Option) { - let mut partition_idx = None; + let mut split_idx = None; let mut offset_idx = None; for (idx, column) in column_descs.iter().enumerate() { match column.additional_column_type { - AdditionalColumnType::Partition => { - partition_idx = Some(idx); + AdditionalColumnType::Partition | AdditionalColumnType::Filename => { + split_idx = Some(idx); } AdditionalColumnType::Offset => { offset_idx = Some(idx); @@ -79,18 +79,18 @@ pub fn get_partition_offset_col_idx( _ => (), } } - (partition_idx, offset_idx) + (split_idx, offset_idx) } pub fn prune_additional_cols( chunk: &StreamChunk, - partition_idx: usize, + split_idx: usize, offset_idx: usize, ) -> StreamChunk { // TODO: ignore if it is user defined chunk.project( &(0..chunk.dimension()) - .filter(|&idx| idx != partition_idx && idx != offset_idx) + .filter(|&idx| idx != split_idx && idx != offset_idx) .collect_vec(), ) } diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 60a746eb31dfd..72ff01d9a27af 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -22,7 +22,7 @@ use futures_async_stream::try_stream; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_connector::source::{ - BoxChunkedSourceStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitMetaData, + BoxChunkSourceStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitMetaData, }; use risingwave_connector::ConnectorParams; use risingwave_source::source_desc::{SourceDesc, SourceDescBuilder}; @@ -91,7 +91,7 @@ impl SourceExecutor { &self, source_desc: &SourceDesc, state: ConnectorState, - ) -> StreamExecutorResult { + ) -> StreamExecutorResult { let column_ids = source_desc .columns .iter() @@ -373,8 +373,7 @@ impl SourceExecutor { .build() .map_err(StreamExecutorError::connector_error)?; - let (Some(partition_idx), Some(offset_idx)) = - get_partition_offset_col_idx(&source_desc.columns) + let (Some(split_idx), Some(offset_idx)) = get_split_offset_col_idx(&source_desc.columns) else { unreachable!("Partition and offset columns must be set."); }; @@ -548,11 +547,8 @@ impl SourceExecutor { Either::Right(chunk) => { // TODO: confirm when split_offset_mapping is None - let split_offset_mapping = get_split_offset_mapping_from_chunk( - &chunk, - partition_idx, - offset_idx, - ); + let split_offset_mapping = + get_split_offset_mapping_from_chunk(&chunk, split_idx, offset_idx); if last_barrier_time.elapsed().as_millis() > max_wait_barrier_time_ms { // Exceeds the max wait barrier time, the source will be paused. // Currently we can guarantee the @@ -613,7 +609,7 @@ impl SourceExecutor { .collect::>(), ) .inc_by(chunk.cardinality() as u64); - let chunk = prune_additional_cols(&chunk, partition_idx, offset_idx); + let chunk = prune_additional_cols(&chunk, split_idx, offset_idx); yield Message::Chunk(chunk); self.try_flush_data().await?; } From 4b70038e1b0290e63116d6da889724a43e43a99e Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Wed, 17 Jan 2024 10:10:49 +0800 Subject: [PATCH 19/42] refactor column_catalogs_to_source_column_descs --- src/source/src/source_desc.rs | 48 ++++++++++++++++++++--------------- 1 file changed, 28 insertions(+), 20 deletions(-) diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index d80fb5a4146b5..8a459bca4d79b 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -18,6 +18,7 @@ use std::sync::Arc; use risingwave_common::catalog::{ColumnDesc, ColumnId}; use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; +use risingwave_common::util::iter_util::ZipEqFast; use risingwave_connector::parser::{EncodingProperties, ProtocolProperties, SpecificParserConfig}; use risingwave_connector::source::monitor::SourceMetrics; use risingwave_connector::source::{ @@ -115,30 +116,37 @@ impl SourceDescBuilder { debug_assert_eq!(additional_columns.len(), 2); + for col in &self.columns { + match col + .column_desc + .as_ref() + .unwrap() + .get_additional_column_type() + { + Ok(AdditionalColumnType::Partition | AdditionalColumnType::Filename) => { + columns_exist[0] = true; + } + Ok(AdditionalColumnType::Offset) => { + columns_exist[1] = true; + } + _ => (), + } + } + let mut columns: Vec<_> = self .columns .iter() - .chain(additional_columns.iter()) - .filter_map(|c| { - let addition_col_existed = - match c.column_desc.as_ref().unwrap().get_additional_column_type() { - Ok(AdditionalColumnType::Partition | AdditionalColumnType::Filename) => { - std::mem::replace(&mut columns_exist[0], true) - } - Ok(AdditionalColumnType::Offset) => { - std::mem::replace(&mut columns_exist[1], true) - } - _ => false, - }; - if !addition_col_existed { - Some(SourceColumnDesc::from(&ColumnDesc::from( - c.column_desc.as_ref().unwrap(), - ))) - } else { - None - } - }) + .map(|c| SourceColumnDesc::from(&ColumnDesc::from(c.column_desc.as_ref().unwrap()))) .collect(); + + for (existed, c) in columns_exist.iter().zip_eq_fast(&additional_columns) { + if !existed { + columns.push(SourceColumnDesc::from(&ColumnDesc::from( + c.column_desc.as_ref().unwrap(), + ))); + } + } + if let Some(row_id_index) = self.row_id_index { columns[row_id_index].column_type = SourceColumnType::RowId; } From 1df6e825fe45886348bd22d8871a967e0e298c68 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Wed, 17 Jan 2024 16:23:53 +0800 Subject: [PATCH 20/42] fix test_source_executor --- src/common/src/catalog/mod.rs | 8 ++++ .../src/parser/additional_columns.rs | 37 ++++++++++++++++++- .../src/source/datagen/source/generator.rs | 29 ++++++++------- src/connector/src/source/manager.rs | 8 ++-- src/source/src/source_desc.rs | 29 ++++++++------- 5 files changed, 78 insertions(+), 33 deletions(-) diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index 14d44c9c0bd2d..f641b1abe126b 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -95,6 +95,14 @@ pub fn is_system_schema(schema_name: &str) -> bool { SYSTEM_SCHEMAS.iter().any(|s| *s == schema_name) } +/// Hidden additional columns for partition/file and offset. +/// Refer to for more details. +pub const ADDITION_SPLIT_OFFSET_COLUMN_PREFIX: &str = "_rw_addition_"; + +pub fn get_addition_key_name(col_name: &str) -> Option<&str> { + col_name.strip_prefix(ADDITION_SPLIT_OFFSET_COLUMN_PREFIX) +} + pub const ROWID_PREFIX: &str = "_row_id"; pub fn is_row_id_column_name(name: &str) -> bool { diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index f604f1225227c..aacc3a2e7d08d 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -31,7 +31,7 @@ pub fn get_connector_compatible_additional_columns( KAFKA_CONNECTOR => kafka_compatible_column_vec(), PULSAR_CONNECTOR => pulsar_compatible_column_vec(), KINESIS_CONNECTOR => kinesis_compatible_column_vec(), - OPENDAL_S3_CONNECTOR | S3_CONNECTOR | GCS_CONNECTOR => s3_compatible_column_column_vec(), + OPENDAL_S3_CONNECTOR | S3_CONNECTOR | GCS_CONNECTOR => s3_compatible_column_vec(), _ => return None, }; Some(compatible_columns) @@ -225,7 +225,7 @@ fn kinesis_compatible_column_vec() -> Vec<(&'static str, CompatibleAdditionalCol ] } -fn s3_compatible_column_column_vec() -> Vec<(&'static str, CompatibleAdditionalColumnsFn)> { +fn s3_compatible_column_vec() -> Vec<(&'static str, CompatibleAdditionalColumnsFn)> { vec![ ( "file", @@ -257,3 +257,36 @@ fn s3_compatible_column_column_vec() -> Vec<(&'static str, CompatibleAdditionalC ), ] } + +pub fn common_compatible_column_vec() -> Vec<(&'static str, CompatibleAdditionalColumnsFn)> { + vec![ + ( + "partition", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named_with_additional_column( + name, + id, + DataType::Varchar, + AdditionalColumnType::Partition, + ), + is_hidden: false, + } + }), + ), + ( + "offset", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named_with_additional_column( + name, + id, + DataType::Varchar, + AdditionalColumnType::Offset, + ), + is_hidden: false, + } + }), + ), + ] +} diff --git a/src/connector/src/source/datagen/source/generator.rs b/src/connector/src/source/datagen/source/generator.rs index a5ea9fc53d265..56dd8ba894ec5 100644 --- a/src/connector/src/source/datagen/source/generator.rs +++ b/src/connector/src/source/datagen/source/generator.rs @@ -16,6 +16,7 @@ use std::time::{Duration, SystemTime, UNIX_EPOCH}; use anyhow::Result; use futures_async_stream::try_stream; use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::catalog::get_addition_key_name; use risingwave_common::error::RwError; use risingwave_common::field_generator::FieldGeneratorImpl; use risingwave_common::row::OwnedRow; @@ -162,11 +163,6 @@ impl DatagenEventGenerator { let mut interval = tokio::time::interval(Duration::from_secs(1)); const MAX_ROWS_PER_YIELD: u64 = 1024; let mut reach_end = false; - let dtypes_with_offset: Vec<_> = self - .data_types - .into_iter() - .chain([DataType::Varchar, DataType::Varchar]) - .collect(); loop { // generate `partition_rows_per_second` rows per second interval.tick().await; @@ -178,10 +174,20 @@ impl DatagenEventGenerator { self.partition_rows_per_second - rows_generated_this_second, ); 'outer: for _ in 0..num_rows_to_generate { - let mut row = Vec::with_capacity(self.fields_vec.len()); - for field_generator in &mut self.fields_vec { + let mut row = Vec::with_capacity(self.data_types.len()); + for (field_generator, field_name) in + self.fields_vec.iter_mut().zip_eq_fast(&self.field_names) + { let datum = match field_generator { - FieldDesc::Invisible => None, + FieldDesc::Invisible => match get_addition_key_name(field_name) { + Some("partition") => { + Some(ScalarImpl::Utf8(self.split_id.as_ref().into())) + } + Some("offset") => { + Some(ScalarImpl::Utf8(self.offset.to_string().into_boxed_str())) + } + _ => None, + }, FieldDesc::Visible(field_generator) => { let datum = field_generator.generate_datum(self.offset); if datum.is_none() { @@ -202,17 +208,12 @@ impl DatagenEventGenerator { } self.offset += 1; - row.extend([ - Some(ScalarImpl::Utf8(self.split_id.as_ref().into())), - Some(ScalarImpl::Utf8(self.offset.to_string().into_boxed_str())), - ]); - rows.push((Op::Insert, OwnedRow::new(row))); rows_generated_this_second += 1; } if !rows.is_empty() { - yield StreamChunk::from_rows(&rows, &dtypes_with_offset); + yield StreamChunk::from_rows(&rows, &self.data_types); } if reach_end { diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index 07ce5a3d4898e..752934153e001 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -15,8 +15,8 @@ use std::fmt::Debug; use risingwave_common::catalog::{ - ColumnDesc, ColumnId, KAFKA_TIMESTAMP_COLUMN_NAME, OFFSET_COLUMN_NAME, ROWID_PREFIX, - TABLE_NAME_COLUMN_NAME, + ColumnDesc, ColumnId, ADDITION_SPLIT_OFFSET_COLUMN_PREFIX, KAFKA_TIMESTAMP_COLUMN_NAME, + OFFSET_COLUMN_NAME, ROWID_PREFIX, TABLE_NAME_COLUMN_NAME, }; use risingwave_common::types::DataType; use risingwave_pb::plan_common::{AdditionalColumnType, ColumnDescVersion}; @@ -58,7 +58,9 @@ pub enum SourceColumnType { impl SourceColumnType { pub fn from_name(name: &str) -> Self { - if name.starts_with(KAFKA_TIMESTAMP_COLUMN_NAME) || name.starts_with(TABLE_NAME_COLUMN_NAME) + if name.starts_with(KAFKA_TIMESTAMP_COLUMN_NAME) + || name.starts_with(TABLE_NAME_COLUMN_NAME) + || name.starts_with(ADDITION_SPLIT_OFFSET_COLUMN_PREFIX) { Self::Meta } else if name == (ROWID_PREFIX) { diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index 8a459bca4d79b..1b023ea0831ae 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -15,10 +15,11 @@ use std::collections::HashMap; use std::sync::Arc; -use risingwave_common::catalog::{ColumnDesc, ColumnId}; +use risingwave_common::catalog::{ColumnDesc, ColumnId, ADDITION_SPLIT_OFFSET_COLUMN_PREFIX}; use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_connector::parser::additional_columns::common_compatible_column_vec; use risingwave_connector::parser::{EncodingProperties, ProtocolProperties, SpecificParserConfig}; use risingwave_connector::source::monitor::SourceMetrics; use risingwave_connector::source::{ @@ -99,20 +100,20 @@ impl SourceDescBuilder { .map(|s| s.to_lowercase()) .unwrap(); - let additional_columns: Vec<_> = - match get_connector_compatible_additional_columns(&connector_name) { - Some(col_list) => ["partition", "file", "offset"] - .into_iter() - .filter_map(|key_name| { - let col_name = format!("_rw_{}_{}", connector_name, key_name); - last_column_id = last_column_id.next(); - col_list.iter().find_map(|(n, f)| { - (key_name == *n).then_some(f(last_column_id, &col_name).to_protobuf()) - }) + let additional_columns: Vec<_> = { + let col_list = get_connector_compatible_additional_columns(&connector_name) + .unwrap_or(common_compatible_column_vec()); + ["partition", "file", "offset"] + .into_iter() + .filter_map(|key_name| { + let col_name = format!("{}{}", ADDITION_SPLIT_OFFSET_COLUMN_PREFIX, key_name); + last_column_id = last_column_id.next(); + col_list.iter().find_map(|(n, f)| { + (key_name == *n).then_some(f(last_column_id, &col_name).to_protobuf()) }) - .collect(), - _ => Vec::new(), - }; + }) + .collect() + }; debug_assert_eq!(additional_columns.len(), 2); From f8f3a45b0b02d41fbba0c3dbd5c172eb92a84b86 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Wed, 17 Jan 2024 16:51:52 +0800 Subject: [PATCH 21/42] prune chunk conditionally --- src/stream/src/executor/source/fetch_executor.rs | 7 ++++++- src/stream/src/executor/source/fs_source_executor.rs | 3 ++- src/stream/src/executor/source/mod.rs | 12 ++++++++++-- src/stream/src/executor/source/source_executor.rs | 7 ++++++- 4 files changed, 24 insertions(+), 5 deletions(-) diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 1c7419cf23dfa..bb27f29426202 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -328,7 +328,12 @@ impl FsFetchExecutor { } } - let chunk = prune_additional_cols(&chunk, split_idx, offset_idx); + let chunk = prune_additional_cols( + &chunk, + split_idx, + offset_idx, + &source_desc.columns, + ); yield Message::Chunk(chunk); } } diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index 8686e69bf4378..328bf944acd2c 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -455,7 +455,8 @@ impl FsSourceExecutor { ]) .inc_by(chunk.cardinality() as u64); - let chunk = prune_additional_cols(&chunk, split_idx, offset_idx); + let chunk = + prune_additional_cols(&chunk, split_idx, offset_idx, &source_desc.columns); yield Message::Chunk(chunk); self.try_flush_data().await?; diff --git a/src/stream/src/executor/source/mod.rs b/src/stream/src/executor/source/mod.rs index c4eb83eab6820..88f5c43643a74 100644 --- a/src/stream/src/executor/source/mod.rs +++ b/src/stream/src/executor/source/mod.rs @@ -22,6 +22,7 @@ pub use fs_source_executor::*; use itertools::Itertools; use risingwave_common::array::StreamChunk; use risingwave_common::bail; +use risingwave_common::catalog::get_addition_key_name; use risingwave_common::row::Row; use risingwave_connector::source::{SourceColumnDesc, SplitId}; use risingwave_pb::plan_common::AdditionalColumnType; @@ -82,15 +83,22 @@ pub fn get_split_offset_col_idx( (split_idx, offset_idx) } +fn is_hidden_addition_col(column_desc: &SourceColumnDesc) -> bool { + get_addition_key_name(&column_desc.name).is_some() +} + pub fn prune_additional_cols( chunk: &StreamChunk, split_idx: usize, offset_idx: usize, + column_descs: &[SourceColumnDesc], ) -> StreamChunk { - // TODO: ignore if it is user defined chunk.project( &(0..chunk.dimension()) - .filter(|&idx| idx != split_idx && idx != offset_idx) + .filter(|&idx| { + (idx != split_idx && idx != offset_idx) + || !is_hidden_addition_col(&column_descs[idx]) + }) .collect_vec(), ) } diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 72ff01d9a27af..92279ef6e8194 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -609,7 +609,12 @@ impl SourceExecutor { .collect::>(), ) .inc_by(chunk.cardinality() as u64); - let chunk = prune_additional_cols(&chunk, split_idx, offset_idx); + let chunk = prune_additional_cols( + &chunk, + split_idx, + offset_idx, + &source_desc.columns, + ); yield Message::Chunk(chunk); self.try_flush_data().await?; } From 7098e96120ce0df5a750bf21aa7a4629532d6b28 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Wed, 17 Jan 2024 21:22:12 +0800 Subject: [PATCH 22/42] fix e2e-source-test --- src/connector/src/source/manager.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index 752934153e001..778c51a42b753 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -15,8 +15,8 @@ use std::fmt::Debug; use risingwave_common::catalog::{ - ColumnDesc, ColumnId, ADDITION_SPLIT_OFFSET_COLUMN_PREFIX, KAFKA_TIMESTAMP_COLUMN_NAME, - OFFSET_COLUMN_NAME, ROWID_PREFIX, TABLE_NAME_COLUMN_NAME, + ColumnDesc, ColumnId, KAFKA_TIMESTAMP_COLUMN_NAME, OFFSET_COLUMN_NAME, ROWID_PREFIX, + TABLE_NAME_COLUMN_NAME, }; use risingwave_common::types::DataType; use risingwave_pb::plan_common::{AdditionalColumnType, ColumnDescVersion}; @@ -58,9 +58,7 @@ pub enum SourceColumnType { impl SourceColumnType { pub fn from_name(name: &str) -> Self { - if name.starts_with(KAFKA_TIMESTAMP_COLUMN_NAME) - || name.starts_with(TABLE_NAME_COLUMN_NAME) - || name.starts_with(ADDITION_SPLIT_OFFSET_COLUMN_PREFIX) + if name.starts_with(KAFKA_TIMESTAMP_COLUMN_NAME) || name.starts_with(TABLE_NAME_COLUMN_NAME) { Self::Meta } else if name == (ROWID_PREFIX) { @@ -108,6 +106,7 @@ impl SourceColumnDesc { #[inline] pub fn is_visible(&self) -> bool { self.column_type == SourceColumnType::Normal + && self.additional_column_type == AdditionalColumnType::Normal } } From 9fdc2a1154eb33054d78441299fc1e4c96abd213 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Wed, 17 Jan 2024 21:36:59 +0800 Subject: [PATCH 23/42] debug --- src/stream/src/executor/source/source_executor.rs | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 92279ef6e8194..3575c021e69d5 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -547,6 +547,13 @@ impl SourceExecutor { Either::Right(chunk) => { // TODO: confirm when split_offset_mapping is None + tracing::debug!( + "source executor({}-{}): colums: {:?} chunk: {:?}", + split_idx, + offset_idx, + source_desc.columns, + chunk + ); let split_offset_mapping = get_split_offset_mapping_from_chunk(&chunk, split_idx, offset_idx); if last_barrier_time.elapsed().as_millis() > max_wait_barrier_time_ms { From 9fc26d2083a02c90550649dc4d1543260dc0e35e Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Wed, 17 Jan 2024 22:19:09 +0800 Subject: [PATCH 24/42] debug debug fix column_id debug debug debug debug debug debug debug fix duplicate column_id debug fix missing columns debug --- src/source/src/source_desc.rs | 13 +++++++++---- src/stream/src/executor/source/source_executor.rs | 7 ------- src/stream/src/from_proto/source/fs_fetch.rs | 13 +++++++++---- src/stream/src/from_proto/source/trad_source.rs | 14 ++++++++++---- 4 files changed, 28 insertions(+), 19 deletions(-) diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index 1b023ea0831ae..3d8513bd4dc56 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -91,9 +91,10 @@ impl SourceDescBuilder { let mut columns_exist = [false; 2]; let mut last_column_id = self .columns - .last() + .iter() .map(|c| c.column_desc.as_ref().unwrap().column_id.into()) - .unwrap_or(ColumnId::new(0)); + .max() + .unwrap_or(ColumnId::placeholder()); let connector_name = self .with_properties .get(UPSTREAM_SOURCE_KEY) @@ -107,9 +108,13 @@ impl SourceDescBuilder { .into_iter() .filter_map(|key_name| { let col_name = format!("{}{}", ADDITION_SPLIT_OFFSET_COLUMN_PREFIX, key_name); - last_column_id = last_column_id.next(); col_list.iter().find_map(|(n, f)| { - (key_name == *n).then_some(f(last_column_id, &col_name).to_protobuf()) + if key_name == *n { + last_column_id = last_column_id.next(); + Some(f(last_column_id, &col_name).to_protobuf()) + } else { + None + } }) }) .collect() diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index 3575c021e69d5..92279ef6e8194 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -547,13 +547,6 @@ impl SourceExecutor { Either::Right(chunk) => { // TODO: confirm when split_offset_mapping is None - tracing::debug!( - "source executor({}-{}): colums: {:?} chunk: {:?}", - split_idx, - offset_idx, - source_desc.columns, - chunk - ); let split_offset_mapping = get_split_offset_mapping_from_chunk(&chunk, split_idx, offset_idx); if last_barrier_time.elapsed().as_millis() > max_wait_barrier_time_ms { diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs index 8d8cb78a80b19..ea1557503a5fe 100644 --- a/src/stream/src/from_proto/source/fs_fetch.rs +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use risingwave_common::catalog::{ColumnId, TableId}; +use risingwave_common::catalog::TableId; use risingwave_connector::source::filesystem::opendal_source::{ OpendalGcs, OpendalPosixFs, OpendalS3, }; @@ -26,7 +26,7 @@ use risingwave_storage::StateStore; use crate::error::StreamResult; use crate::executor::{ BoxedExecutor, Executor, FlowControlExecutor, FsFetchExecutor, SourceStateTableHandler, - StreamSourceCore, + StreamExecutorError, StreamSourceCore, }; use crate::from_proto::ExecutorBuilder; use crate::task::{ExecutorParams, LocalStreamManagerCore}; @@ -64,10 +64,15 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { chunk_size: params.env.config().developer.chunk_size, }; - let source_column_ids: Vec<_> = source + let source_desc = source_desc_builder + .clone() + .build() + .map_err(StreamExecutorError::connector_error)?; + + let source_column_ids: Vec<_> = source_desc .columns .iter() - .map(|column| ColumnId::from(column.get_column_desc().unwrap().column_id)) + .map(|column| column.column_id) .collect(); let vnodes = Some(Arc::new( diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index 5b93585aea4b1..deed8d2526e1a 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -13,7 +13,7 @@ // limitations under the License. use risingwave_common::catalog::{ - default_key_column_name_version_mapping, ColumnId, TableId, KAFKA_TIMESTAMP_COLUMN_NAME, + default_key_column_name_version_mapping, TableId, KAFKA_TIMESTAMP_COLUMN_NAME, }; use risingwave_connector::source::{ConnectorProperties, SourceCtrlOpts}; use risingwave_pb::data::data_type::TypeName as PbTypeName; @@ -29,7 +29,7 @@ use super::*; use crate::executor::source::{FsListExecutor, StreamSourceCore}; use crate::executor::source_executor::SourceExecutor; use crate::executor::state_table_handler::SourceStateTableHandler; -use crate::executor::{FlowControlExecutor, FsSourceExecutor}; +use crate::executor::{FlowControlExecutor, FsSourceExecutor, StreamExecutorError}; const FS_CONNECTORS: &[&str] = &["s3"]; pub struct SourceExecutorBuilder; @@ -134,9 +134,15 @@ impl ExecutorBuilder for SourceExecutorBuilder { chunk_size: params.env.config().developer.chunk_size, }; - let source_column_ids: Vec<_> = source_columns + let source_desc = source_desc_builder + .clone() + .build() + .map_err(StreamExecutorError::connector_error)?; + + let source_column_ids: Vec<_> = source_desc + .columns .iter() - .map(|column| ColumnId::from(column.get_column_desc().unwrap().column_id)) + .map(|column| column.column_id) .collect(); let state_table_handler = SourceStateTableHandler::from_table_catalog( From b0a3b047c64addbb966cbf58f25c80e0b2762351 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Fri, 19 Jan 2024 12:28:12 +0800 Subject: [PATCH 25/42] fix hidden columns --- src/common/src/catalog/mod.rs | 4 ++++ src/connector/src/source/manager.rs | 7 +++---- src/stream/src/executor/source/mod.rs | 8 ++------ 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index f641b1abe126b..eb5ff443f0bed 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -103,6 +103,10 @@ pub fn get_addition_key_name(col_name: &str) -> Option<&str> { col_name.strip_prefix(ADDITION_SPLIT_OFFSET_COLUMN_PREFIX) } +pub fn is_hidden_addition_col(col_name: &str) -> bool { + col_name.starts_with(ADDITION_SPLIT_OFFSET_COLUMN_PREFIX) +} + pub const ROWID_PREFIX: &str = "_row_id"; pub fn is_row_id_column_name(name: &str) -> bool { diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index 778c51a42b753..9a2b98cb424b3 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -15,8 +15,8 @@ use std::fmt::Debug; use risingwave_common::catalog::{ - ColumnDesc, ColumnId, KAFKA_TIMESTAMP_COLUMN_NAME, OFFSET_COLUMN_NAME, ROWID_PREFIX, - TABLE_NAME_COLUMN_NAME, + is_hidden_addition_col, ColumnDesc, ColumnId, KAFKA_TIMESTAMP_COLUMN_NAME, OFFSET_COLUMN_NAME, + ROWID_PREFIX, TABLE_NAME_COLUMN_NAME, }; use risingwave_common::types::DataType; use risingwave_pb::plan_common::{AdditionalColumnType, ColumnDescVersion}; @@ -105,8 +105,7 @@ impl SourceColumnDesc { #[inline] pub fn is_visible(&self) -> bool { - self.column_type == SourceColumnType::Normal - && self.additional_column_type == AdditionalColumnType::Normal + self.column_type == SourceColumnType::Normal && !is_hidden_addition_col(&self.name) } } diff --git a/src/stream/src/executor/source/mod.rs b/src/stream/src/executor/source/mod.rs index 88f5c43643a74..6b41982e0e9d2 100644 --- a/src/stream/src/executor/source/mod.rs +++ b/src/stream/src/executor/source/mod.rs @@ -22,7 +22,7 @@ pub use fs_source_executor::*; use itertools::Itertools; use risingwave_common::array::StreamChunk; use risingwave_common::bail; -use risingwave_common::catalog::get_addition_key_name; +use risingwave_common::catalog::is_hidden_addition_col; use risingwave_common::row::Row; use risingwave_connector::source::{SourceColumnDesc, SplitId}; use risingwave_pb::plan_common::AdditionalColumnType; @@ -83,10 +83,6 @@ pub fn get_split_offset_col_idx( (split_idx, offset_idx) } -fn is_hidden_addition_col(column_desc: &SourceColumnDesc) -> bool { - get_addition_key_name(&column_desc.name).is_some() -} - pub fn prune_additional_cols( chunk: &StreamChunk, split_idx: usize, @@ -97,7 +93,7 @@ pub fn prune_additional_cols( &(0..chunk.dimension()) .filter(|&idx| { (idx != split_idx && idx != offset_idx) - || !is_hidden_addition_col(&column_descs[idx]) + || !is_hidden_addition_col(&column_descs[idx].name) }) .collect_vec(), ) From 830e389ca069b1e5b78433be744bd97689e78ce9 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Fri, 19 Jan 2024 13:21:44 +0800 Subject: [PATCH 26/42] fix debezuim mongo --- src/connector/src/parser/debezium/mongo_json_parser.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/connector/src/parser/debezium/mongo_json_parser.rs b/src/connector/src/parser/debezium/mongo_json_parser.rs index 9233bc987eb17..89d46d7eb20f8 100644 --- a/src/connector/src/parser/debezium/mongo_json_parser.rs +++ b/src/connector/src/parser/debezium/mongo_json_parser.rs @@ -63,7 +63,8 @@ impl DebeziumMongoJsonParser { })? .clone(); - if rw_columns.len() != 2 { + // _rw_addition_partition & _rw_addition_offset are created automatically. + if rw_columns.len() != 4 { return Err(RwError::from(ProtocolError( "Debezuim Mongo needs no more columns except `_id` and `payload` in table".into(), ))); From 40b50248b418c9d008fdc344caa959d593edf34c Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Fri, 19 Jan 2024 14:37:11 +0800 Subject: [PATCH 27/42] fix confict --- src/bench/sink_bench/main.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/bench/sink_bench/main.rs b/src/bench/sink_bench/main.rs index 943c2dc26e1b2..0d43dcb480d7f 100644 --- a/src/bench/sink_bench/main.rs +++ b/src/bench/sink_bench/main.rs @@ -240,7 +240,7 @@ impl MockDatagenSource { loop { for i in &mut readers { let item = i.next().await.unwrap().unwrap(); - yield Message::Chunk(item.chunk); + yield Message::Chunk(item); } } } From 3216a23a62b8797c85e3d2679779ebaee31c846a Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Fri, 19 Jan 2024 15:22:04 +0800 Subject: [PATCH 28/42] fix mongo test --- .../src/parser/debezium/mongo_json_parser.rs | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/src/connector/src/parser/debezium/mongo_json_parser.rs b/src/connector/src/parser/debezium/mongo_json_parser.rs index 89d46d7eb20f8..858f1164629bb 100644 --- a/src/connector/src/parser/debezium/mongo_json_parser.rs +++ b/src/connector/src/parser/debezium/mongo_json_parser.rs @@ -17,6 +17,7 @@ use std::fmt::Debug; use risingwave_common::error::ErrorCode::{self, ProtocolError}; use risingwave_common::error::{Result, RwError}; use risingwave_common::types::DataType; +use risingwave_pb::plan_common::AdditionalColumnType; use simd_json::prelude::MutableObject; use simd_json::BorrowedValue; @@ -63,8 +64,20 @@ impl DebeziumMongoJsonParser { })? .clone(); - // _rw_addition_partition & _rw_addition_offset are created automatically. - if rw_columns.len() != 4 { + // _rw_{connector}_file/partition & _rw_{connector}_offset are created automatically. + if rw_columns + .iter() + .filter(|desc| { + !matches!( + desc.additional_column_type, + AdditionalColumnType::Partition + | AdditionalColumnType::Filename + | AdditionalColumnType::Offset + ) + }) + .count() + != 2 + { return Err(RwError::from(ProtocolError( "Debezuim Mongo needs no more columns except `_id` and `payload` in table".into(), ))); From e569e5d814e8c688a21aeed40e6e100e949e23d0 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Mon, 22 Jan 2024 12:11:32 +0800 Subject: [PATCH 29/42] fix simulation test --- .../simulation/tests/integration_tests/sink/utils.rs | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/src/tests/simulation/tests/integration_tests/sink/utils.rs b/src/tests/simulation/tests/integration_tests/sink/utils.rs index d06d5a3625db4..9fb5abfd8c932 100644 --- a/src/tests/simulation/tests/integration_tests/sink/utils.rs +++ b/src/tests/simulation/tests/integration_tests/sink/utils.rs @@ -38,7 +38,6 @@ use risingwave_connector::sink::SinkError; use risingwave_connector::source::test_source::{ registry_test_source, BoxSource, TestSourceRegistryGuard, TestSourceSplit, }; -use risingwave_connector::source::StreamChunk; use risingwave_simulation::cluster::{Cluster, ConfigPath, Configuration}; use tokio::time::sleep; @@ -304,13 +303,7 @@ impl SimulationTestSource { while offset < id_list.len() && chunks.len() < pause_interval { let id = id_list[offset]; let chunk = build_stream_chunk(once((id, simple_name_of_id(id)))); - let mut split_offset = HashMap::new(); - split_offset.insert(split.id.clone(), offset.to_string()); - let chunk_with_state = StreamChunk { - chunk, - split_offset_mapping: Some(split_offset), - }; - chunks.push(chunk_with_state); + chunks.push(chunk); offset += 1; } From 6e3250073fb2e0c0294b2048d1f1b6389e626661 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Mon, 22 Jan 2024 14:24:13 +0800 Subject: [PATCH 30/42] fix simulation test --- .../tests/integration_tests/sink/utils.rs | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/tests/simulation/tests/integration_tests/sink/utils.rs b/src/tests/simulation/tests/integration_tests/sink/utils.rs index 9fb5abfd8c932..0932d5a772e99 100644 --- a/src/tests/simulation/tests/integration_tests/sink/utils.rs +++ b/src/tests/simulation/tests/integration_tests/sink/utils.rs @@ -229,20 +229,24 @@ impl SimulationTestSink { } } -pub fn build_stream_chunk(row_iter: impl Iterator) -> StreamChunk { +pub fn build_stream_chunk( + row_iter: impl Iterator, +) -> StreamChunk { static ROW_ID_GEN: LazyLock> = LazyLock::new(|| Arc::new(AtomicI64::new(0))); let mut builder = DataChunkBuilder::new( vec![DataType::Int32, DataType::Varchar, DataType::Serial], 100000, ); - for (id, name) in row_iter { + for (id, name, split_id, offset) in row_iter { let row_id = ROW_ID_GEN.fetch_add(1, Relaxed); std::assert!(builder .append_one_row([ Some(ScalarImpl::Int32(id)), Some(ScalarImpl::Utf8(name.into())), Some(ScalarImpl::Serial(Serial::from(row_id))), + Some(ScalarImpl::Utf8(split_id.into())), + Some(ScalarImpl::Utf8(offset.into())), ]) .is_none()); } @@ -302,7 +306,12 @@ impl SimulationTestSource { let mut chunks = Vec::new(); while offset < id_list.len() && chunks.len() < pause_interval { let id = id_list[offset]; - let chunk = build_stream_chunk(once((id, simple_name_of_id(id)))); + let chunk = build_stream_chunk(once(( + id, + simple_name_of_id(id), + split.id.to_string(), + offset.to_string(), + ))); chunks.push(chunk); offset += 1; From 91480d4c676630d49367791a79bfbeee6be49bc6 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Mon, 22 Jan 2024 14:54:36 +0800 Subject: [PATCH 31/42] fix simulation test --- .../simulation/tests/integration_tests/sink/utils.rs | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/tests/simulation/tests/integration_tests/sink/utils.rs b/src/tests/simulation/tests/integration_tests/sink/utils.rs index 0932d5a772e99..a134ab1a265ff 100644 --- a/src/tests/simulation/tests/integration_tests/sink/utils.rs +++ b/src/tests/simulation/tests/integration_tests/sink/utils.rs @@ -235,7 +235,13 @@ pub fn build_stream_chunk( static ROW_ID_GEN: LazyLock> = LazyLock::new(|| Arc::new(AtomicI64::new(0))); let mut builder = DataChunkBuilder::new( - vec![DataType::Int32, DataType::Varchar, DataType::Serial], + vec![ + DataType::Int32, + DataType::Varchar, + DataType::Serial, + DataType::Varchar, + DataType::Varchar, + ], 100000, ); for (id, name, split_id, offset) in row_iter { From 62df41ddbf107048472625cae0cbd3af01c048eb Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Mon, 22 Jan 2024 18:20:45 +0800 Subject: [PATCH 32/42] determine hidden without name --- src/common/src/catalog/mod.rs | 12 ------------ .../src/parser/debezium/simd_json_parser.rs | 1 + src/connector/src/parser/json_parser.rs | 1 + .../src/source/datagen/source/generator.rs | 5 ++++- src/connector/src/source/manager.rs | 19 ++++++++++++++++--- src/source/src/source_desc.rs | 11 ++++++----- src/stream/src/executor/source/mod.rs | 4 +--- 7 files changed, 29 insertions(+), 24 deletions(-) diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index 2ae403ef66618..9bf025061f9b9 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -97,18 +97,6 @@ pub fn is_system_schema(schema_name: &str) -> bool { SYSTEM_SCHEMAS.iter().any(|s| *s == schema_name) } -/// Hidden additional columns for partition/file and offset. -/// Refer to for more details. -pub const ADDITION_SPLIT_OFFSET_COLUMN_PREFIX: &str = "_rw_addition_"; - -pub fn get_addition_key_name(col_name: &str) -> Option<&str> { - col_name.strip_prefix(ADDITION_SPLIT_OFFSET_COLUMN_PREFIX) -} - -pub fn is_hidden_addition_col(col_name: &str) -> bool { - col_name.starts_with(ADDITION_SPLIT_OFFSET_COLUMN_PREFIX) -} - pub const ROWID_PREFIX: &str = "_row_id"; pub fn is_row_id_column_name(name: &str) -> bool { diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index 5efdd237e9e32..d6ad8f1c2488a 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -566,6 +566,7 @@ mod tests { fields: vec![], column_type: SourceColumnType::Normal, is_pk: false, + is_hidden: false, additional_column_type: AdditionalColumnType::Normal, }, SourceColumnDesc::simple("o_enum", DataType::Varchar, ColumnId::from(8)), diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 5ed5e2811df98..59940dfe1acc7 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -580,6 +580,7 @@ mod tests { fields: vec![], column_type: SourceColumnType::Normal, is_pk: true, + is_hidden: false, additional_column_type: AdditionalColumnType::Key, }; let descs = vec![ diff --git a/src/connector/src/source/datagen/source/generator.rs b/src/connector/src/source/datagen/source/generator.rs index 56dd8ba894ec5..fd6378d8253c5 100644 --- a/src/connector/src/source/datagen/source/generator.rs +++ b/src/connector/src/source/datagen/source/generator.rs @@ -16,7 +16,6 @@ use std::time::{Duration, SystemTime, UNIX_EPOCH}; use anyhow::Result; use futures_async_stream::try_stream; use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::catalog::get_addition_key_name; use risingwave_common::error::RwError; use risingwave_common::field_generator::FieldGeneratorImpl; use risingwave_common::row::OwnedRow; @@ -26,6 +25,10 @@ use risingwave_common::util::iter_util::ZipEqFast; use crate::parser::{EncodingProperties, ProtocolProperties, SpecificParserConfig}; use crate::source::{SourceMessage, SourceMeta, SplitId}; +fn get_addition_key_name(field_name: &str) -> Option<&str> { + field_name.strip_prefix("_rw_datagen_") +} + pub enum FieldDesc { // field is invisible, generate None Invisible, diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index 9a2b98cb424b3..801225b5178ea 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -15,8 +15,8 @@ use std::fmt::Debug; use risingwave_common::catalog::{ - is_hidden_addition_col, ColumnDesc, ColumnId, KAFKA_TIMESTAMP_COLUMN_NAME, OFFSET_COLUMN_NAME, - ROWID_PREFIX, TABLE_NAME_COLUMN_NAME, + ColumnDesc, ColumnId, KAFKA_TIMESTAMP_COLUMN_NAME, OFFSET_COLUMN_NAME, ROWID_PREFIX, + TABLE_NAME_COLUMN_NAME, }; use risingwave_common::types::DataType; use risingwave_pb::plan_common::{AdditionalColumnType, ColumnDescVersion}; @@ -34,6 +34,9 @@ pub struct SourceColumnDesc { // `is_pk` is used to indicate whether the column is part of the primary key columns. pub is_pk: bool, + // `is_hidden` is used to indicate whether the column is hidden from the user. + pub is_hidden: bool, + // `additional_column_type` and `column_type` are orthogonal // `additional_column_type` is used to indicate the column is from which part of the message // `column_type` is used to indicate the type of the column, only used in cdc scenario @@ -87,10 +90,18 @@ impl SourceColumnDesc { fields: vec![], column_type: SourceColumnType::Normal, is_pk: false, + is_hidden: false, additional_column_type: AdditionalColumnType::Normal, } } + pub fn from_column_desc_with_hidden(c: &ColumnDesc, is_hidden: bool) -> Self { + Self { + is_hidden, + ..c.into() + } + } + pub fn is_row_id(&self) -> bool { self.column_type == SourceColumnType::RowId } @@ -105,13 +116,14 @@ impl SourceColumnDesc { #[inline] pub fn is_visible(&self) -> bool { - self.column_type == SourceColumnType::Normal && !is_hidden_addition_col(&self.name) + !self.is_hidden } } impl From<&ColumnDesc> for SourceColumnDesc { fn from(c: &ColumnDesc) -> Self { let column_type = SourceColumnType::from_name(c.name.as_str()); + let is_hidden = column_type != SourceColumnType::Normal; Self { name: c.name.clone(), data_type: c.data_type.clone(), @@ -119,6 +131,7 @@ impl From<&ColumnDesc> for SourceColumnDesc { fields: c.field_descs.clone(), column_type, is_pk: false, + is_hidden, additional_column_type: c.additional_column_type, } } diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index 3d8513bd4dc56..c03b63f3f0f09 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -15,7 +15,7 @@ use std::collections::HashMap; use std::sync::Arc; -use risingwave_common::catalog::{ColumnDesc, ColumnId, ADDITION_SPLIT_OFFSET_COLUMN_PREFIX}; +use risingwave_common::catalog::{ColumnDesc, ColumnId}; use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; use risingwave_common::util::iter_util::ZipEqFast; @@ -107,7 +107,7 @@ impl SourceDescBuilder { ["partition", "file", "offset"] .into_iter() .filter_map(|key_name| { - let col_name = format!("{}{}", ADDITION_SPLIT_OFFSET_COLUMN_PREFIX, key_name); + let col_name = format!("_rw_{}_{}", connector_name, key_name); col_list.iter().find_map(|(n, f)| { if key_name == *n { last_column_id = last_column_id.next(); @@ -147,9 +147,10 @@ impl SourceDescBuilder { for (existed, c) in columns_exist.iter().zip_eq_fast(&additional_columns) { if !existed { - columns.push(SourceColumnDesc::from(&ColumnDesc::from( - c.column_desc.as_ref().unwrap(), - ))); + columns.push(SourceColumnDesc::from_column_desc_with_hidden( + &ColumnDesc::from(c.column_desc.as_ref().unwrap()), + true, + )); } } diff --git a/src/stream/src/executor/source/mod.rs b/src/stream/src/executor/source/mod.rs index 6b41982e0e9d2..f53c6dd6eb850 100644 --- a/src/stream/src/executor/source/mod.rs +++ b/src/stream/src/executor/source/mod.rs @@ -22,7 +22,6 @@ pub use fs_source_executor::*; use itertools::Itertools; use risingwave_common::array::StreamChunk; use risingwave_common::bail; -use risingwave_common::catalog::is_hidden_addition_col; use risingwave_common::row::Row; use risingwave_connector::source::{SourceColumnDesc, SplitId}; use risingwave_pb::plan_common::AdditionalColumnType; @@ -92,8 +91,7 @@ pub fn prune_additional_cols( chunk.project( &(0..chunk.dimension()) .filter(|&idx| { - (idx != split_idx && idx != offset_idx) - || !is_hidden_addition_col(&column_descs[idx].name) + (idx != split_idx && idx != offset_idx) || column_descs[idx].is_visible() }) .collect_vec(), ) From 052f852d28835d8ceace3c9a5e60d7b27b6242a4 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Mon, 22 Jan 2024 18:39:24 +0800 Subject: [PATCH 33/42] fix unit test --- .../src/parser/debezium/simd_json_parser.rs | 2 +- src/connector/src/parser/json_parser.rs | 2 +- src/connector/src/source/manager.rs | 15 +++++++-------- src/source/src/source_desc.rs | 3 +-- 4 files changed, 10 insertions(+), 12 deletions(-) diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index d6ad8f1c2488a..4fd2c8c5c3d4b 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -566,7 +566,7 @@ mod tests { fields: vec![], column_type: SourceColumnType::Normal, is_pk: false, - is_hidden: false, + is_hidden_addition_col: false, additional_column_type: AdditionalColumnType::Normal, }, SourceColumnDesc::simple("o_enum", DataType::Varchar, ColumnId::from(8)), diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 59940dfe1acc7..640af79b33077 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -580,7 +580,7 @@ mod tests { fields: vec![], column_type: SourceColumnType::Normal, is_pk: true, - is_hidden: false, + is_hidden_addition_col: false, additional_column_type: AdditionalColumnType::Key, }; let descs = vec![ diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index 801225b5178ea..7ed30e842f974 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -34,8 +34,8 @@ pub struct SourceColumnDesc { // `is_pk` is used to indicate whether the column is part of the primary key columns. pub is_pk: bool, - // `is_hidden` is used to indicate whether the column is hidden from the user. - pub is_hidden: bool, + // `is_hidden_addition_col` is used to indicate whether the column is a hidden addition column. + pub is_hidden_addition_col: bool, // `additional_column_type` and `column_type` are orthogonal // `additional_column_type` is used to indicate the column is from which part of the message @@ -90,14 +90,14 @@ impl SourceColumnDesc { fields: vec![], column_type: SourceColumnType::Normal, is_pk: false, - is_hidden: false, + is_hidden_addition_col: false, additional_column_type: AdditionalColumnType::Normal, } } - pub fn from_column_desc_with_hidden(c: &ColumnDesc, is_hidden: bool) -> Self { + pub fn hidden_addition_col_from_column_desc(c: &ColumnDesc) -> Self { Self { - is_hidden, + is_hidden_addition_col: true, ..c.into() } } @@ -116,14 +116,13 @@ impl SourceColumnDesc { #[inline] pub fn is_visible(&self) -> bool { - !self.is_hidden + !self.is_hidden_addition_col && self.column_type == SourceColumnType::Normal } } impl From<&ColumnDesc> for SourceColumnDesc { fn from(c: &ColumnDesc) -> Self { let column_type = SourceColumnType::from_name(c.name.as_str()); - let is_hidden = column_type != SourceColumnType::Normal; Self { name: c.name.clone(), data_type: c.data_type.clone(), @@ -131,7 +130,7 @@ impl From<&ColumnDesc> for SourceColumnDesc { fields: c.field_descs.clone(), column_type, is_pk: false, - is_hidden, + is_hidden_addition_col: false, additional_column_type: c.additional_column_type, } } diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index c03b63f3f0f09..4245ed474b440 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -147,9 +147,8 @@ impl SourceDescBuilder { for (existed, c) in columns_exist.iter().zip_eq_fast(&additional_columns) { if !existed { - columns.push(SourceColumnDesc::from_column_desc_with_hidden( + columns.push(SourceColumnDesc::hidden_addition_col_from_column_desc( &ColumnDesc::from(c.column_desc.as_ref().unwrap()), - true, )); } } From a18f3613b04faf1b19053cb94b527d97d2b4dfe7 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Mon, 22 Jan 2024 19:33:55 +0800 Subject: [PATCH 34/42] avoid building a source_desc in from_proto --- src/source/src/source_desc.rs | 2 +- src/stream/src/from_proto/source/fs_fetch.rs | 11 +++-------- src/stream/src/from_proto/source/trad_source.rs | 11 +++-------- 3 files changed, 7 insertions(+), 17 deletions(-) diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index 4245ed474b440..8931c743eda5d 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -87,7 +87,7 @@ impl SourceDescBuilder { } } - fn column_catalogs_to_source_column_descs(&self) -> Vec { + pub fn column_catalogs_to_source_column_descs(&self) -> Vec { let mut columns_exist = [false; 2]; let mut last_column_id = self .columns diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs index 8cd5dcfa7d918..e211cde0b76a6 100644 --- a/src/stream/src/from_proto/source/fs_fetch.rs +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -26,7 +26,7 @@ use risingwave_storage::StateStore; use crate::error::StreamResult; use crate::executor::{ BoxedExecutor, Executor, FlowControlExecutor, FsFetchExecutor, SourceStateTableHandler, - StreamExecutorError, StreamSourceCore, + StreamSourceCore, }; use crate::from_proto::ExecutorBuilder; use crate::task::ExecutorParams; @@ -63,13 +63,8 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { chunk_size: params.env.config().developer.chunk_size, }; - let source_desc = source_desc_builder - .clone() - .build() - .map_err(StreamExecutorError::connector_error)?; - - let source_column_ids: Vec<_> = source_desc - .columns + let source_column_ids: Vec<_> = source_desc_builder + .column_catalogs_to_source_column_descs() .iter() .map(|column| column.column_id) .collect(); diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index 3d03cacda85a7..2da5ce47573a3 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -29,7 +29,7 @@ use super::*; use crate::executor::source::{FsListExecutor, StreamSourceCore}; use crate::executor::source_executor::SourceExecutor; use crate::executor::state_table_handler::SourceStateTableHandler; -use crate::executor::{FlowControlExecutor, FsSourceExecutor, StreamExecutorError}; +use crate::executor::{FlowControlExecutor, FsSourceExecutor}; const FS_CONNECTORS: &[&str] = &["s3"]; pub struct SourceExecutorBuilder; @@ -132,13 +132,8 @@ impl ExecutorBuilder for SourceExecutorBuilder { chunk_size: params.env.config().developer.chunk_size, }; - let source_desc = source_desc_builder - .clone() - .build() - .map_err(StreamExecutorError::connector_error)?; - - let source_column_ids: Vec<_> = source_desc - .columns + let source_column_ids: Vec<_> = source_desc_builder + .column_catalogs_to_source_column_descs() .iter() .map(|column| column.column_id) .collect(); From 08f8f123c8679785033b524d3fc810ca6b64cabb Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Tue, 23 Jan 2024 10:59:34 +0800 Subject: [PATCH 35/42] chores --- .../src/parser/debezium/mongo_json_parser.rs | 15 +-------------- src/source/src/source_desc.rs | 7 +++---- 2 files changed, 4 insertions(+), 18 deletions(-) diff --git a/src/connector/src/parser/debezium/mongo_json_parser.rs b/src/connector/src/parser/debezium/mongo_json_parser.rs index 858f1164629bb..899068a4cccb5 100644 --- a/src/connector/src/parser/debezium/mongo_json_parser.rs +++ b/src/connector/src/parser/debezium/mongo_json_parser.rs @@ -17,7 +17,6 @@ use std::fmt::Debug; use risingwave_common::error::ErrorCode::{self, ProtocolError}; use risingwave_common::error::{Result, RwError}; use risingwave_common::types::DataType; -use risingwave_pb::plan_common::AdditionalColumnType; use simd_json::prelude::MutableObject; use simd_json::BorrowedValue; @@ -65,19 +64,7 @@ impl DebeziumMongoJsonParser { .clone(); // _rw_{connector}_file/partition & _rw_{connector}_offset are created automatically. - if rw_columns - .iter() - .filter(|desc| { - !matches!( - desc.additional_column_type, - AdditionalColumnType::Partition - | AdditionalColumnType::Filename - | AdditionalColumnType::Offset - ) - }) - .count() - != 2 - { + if rw_columns.iter().filter(|desc| desc.is_visible()).count() != 2 { return Err(RwError::from(ProtocolError( "Debezuim Mongo needs no more columns except `_id` and `payload` in table".into(), ))); diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index 8931c743eda5d..8a1f45d972fd9 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -105,11 +105,11 @@ impl SourceDescBuilder { let col_list = get_connector_compatible_additional_columns(&connector_name) .unwrap_or(common_compatible_column_vec()); ["partition", "file", "offset"] - .into_iter() + .iter() .filter_map(|key_name| { let col_name = format!("_rw_{}_{}", connector_name, key_name); col_list.iter().find_map(|(n, f)| { - if key_name == *n { + if key_name == n { last_column_id = last_column_id.next(); Some(f(last_column_id, &col_name).to_protobuf()) } else { @@ -119,8 +119,7 @@ impl SourceDescBuilder { }) .collect() }; - - debug_assert_eq!(additional_columns.len(), 2); + assert_eq!(additional_columns.len(), 2); for col in &self.columns { match col From e5ce20d712a0b0e5f361bc9fdeebab8b446d56f0 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Thu, 25 Jan 2024 10:11:32 +0800 Subject: [PATCH 36/42] fix comments --- src/connector/src/parser/additional_columns.rs | 1 + src/connector/src/source/manager.rs | 10 +++++----- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index aacc3a2e7d08d..a6503b98a95e8 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -258,6 +258,7 @@ fn s3_compatible_column_vec() -> Vec<(&'static str, CompatibleAdditionalColumnsF ] } +/// For sources that do not support include clause, we still need to add hidden columns `partition` and `offset`. pub fn common_compatible_column_vec() -> Vec<(&'static str, CompatibleAdditionalColumnsFn)> { vec![ ( diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index 7ed30e842f974..7bb26bb514f39 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -31,15 +31,15 @@ pub struct SourceColumnDesc { pub fields: Vec, pub column_type: SourceColumnType, - // `is_pk` is used to indicate whether the column is part of the primary key columns. + /// `is_pk` is used to indicate whether the column is part of the primary key columns. pub is_pk: bool, - // `is_hidden_addition_col` is used to indicate whether the column is a hidden addition column. + /// `is_hidden_addition_col` is used to indicate whether the column is a hidden addition column. pub is_hidden_addition_col: bool, - // `additional_column_type` and `column_type` are orthogonal - // `additional_column_type` is used to indicate the column is from which part of the message - // `column_type` is used to indicate the type of the column, only used in cdc scenario + /// `additional_column_type` and `column_type` are orthogonal + /// `additional_column_type` is used to indicate the column is from which part of the message + /// `column_type` is used to indicate the type of the column, only used in cdc scenario pub additional_column_type: AdditionalColumnType, } From 8ac8f68787bee8d54d8740398654a84cf148de47 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Thu, 25 Jan 2024 10:11:51 +0800 Subject: [PATCH 37/42] refactor datagen with StreamChunkBuilder --- .../src/source/datagen/source/generator.rs | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/src/connector/src/source/datagen/source/generator.rs b/src/connector/src/source/datagen/source/generator.rs index fd6378d8253c5..d2574f0d84a66 100644 --- a/src/connector/src/source/datagen/source/generator.rs +++ b/src/connector/src/source/datagen/source/generator.rs @@ -15,6 +15,7 @@ use std::time::{Duration, SystemTime, UNIX_EPOCH}; use anyhow::Result; use futures_async_stream::try_stream; +use risingwave_common::array::stream_chunk_builder::StreamChunkBuilder; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::error::RwError; use risingwave_common::field_generator::FieldGeneratorImpl; @@ -25,10 +26,6 @@ use risingwave_common::util::iter_util::ZipEqFast; use crate::parser::{EncodingProperties, ProtocolProperties, SpecificParserConfig}; use crate::source::{SourceMessage, SourceMeta, SplitId}; -fn get_addition_key_name(field_name: &str) -> Option<&str> { - field_name.strip_prefix("_rw_datagen_") -} - pub enum FieldDesc { // field is invisible, generate None Invisible, @@ -170,23 +167,25 @@ impl DatagenEventGenerator { // generate `partition_rows_per_second` rows per second interval.tick().await; let mut rows_generated_this_second = 0; + let mut chunk_builder = + StreamChunkBuilder::new(MAX_ROWS_PER_YIELD as usize, self.data_types.clone()); while rows_generated_this_second < self.partition_rows_per_second { - let mut rows = vec![]; + self.offset += 1; let num_rows_to_generate = std::cmp::min( MAX_ROWS_PER_YIELD, self.partition_rows_per_second - rows_generated_this_second, ); - 'outer: for _ in 0..num_rows_to_generate { + 'outer: for i in 0..num_rows_to_generate { let mut row = Vec::with_capacity(self.data_types.len()); for (field_generator, field_name) in self.fields_vec.iter_mut().zip_eq_fast(&self.field_names) { let datum = match field_generator { - FieldDesc::Invisible => match get_addition_key_name(field_name) { - Some("partition") => { + FieldDesc::Invisible => match field_name.as_str() { + "_rw_datagen_partition" => { Some(ScalarImpl::Utf8(self.split_id.as_ref().into())) } - Some("offset") => { + "_rw_datagen_offset" => { Some(ScalarImpl::Utf8(self.offset.to_string().into_boxed_str())) } _ => None, @@ -210,13 +209,14 @@ impl DatagenEventGenerator { row.push(datum); } - self.offset += 1; - rows.push((Op::Insert, OwnedRow::new(row))); rows_generated_this_second += 1; + if let Some(chunk) = chunk_builder.append_row(Op::Insert, OwnedRow::new(row)) { + yield chunk; + } } - if !rows.is_empty() { - yield StreamChunk::from_rows(&rows, &self.data_types); + if let Some(chunk) = chunk_builder.take() { + yield chunk; } if reach_end { From ab1e53a379cefb6a0c53296bec387b0d182ca2b7 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Thu, 25 Jan 2024 10:11:59 +0800 Subject: [PATCH 38/42] refactor nexmark with StreamChunkBuilder --- .../src/source/nexmark/source/reader.rs | 123 ++++++++++++------ 1 file changed, 86 insertions(+), 37 deletions(-) diff --git a/src/connector/src/source/nexmark/source/reader.rs b/src/connector/src/source/nexmark/source/reader.rs index ed6a6b40eaad3..e8ff3097d8217 100644 --- a/src/connector/src/source/nexmark/source/reader.rs +++ b/src/connector/src/source/nexmark/source/reader.rs @@ -21,6 +21,7 @@ use futures_async_stream::try_stream; use nexmark::config::NexmarkConfig; use nexmark::event::EventType; use nexmark::EventGenerator; +use risingwave_common::array::stream_chunk_builder::StreamChunkBuilder; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::error::RwError; use risingwave_common::estimate_size::EstimateSize; @@ -116,15 +117,15 @@ impl SplitReader for NexmarkSplitReader { const BUFFER_SIZE: usize = 4; spawn_data_generation_stream( self.into_native_stream() - .inspect_ok(move |chunk_with_states: &StreamChunk| { + .inspect_ok(move |chunk: &StreamChunk| { metrics .partition_input_count .with_label_values(&[&actor_id, &source_id, &split_id]) - .inc_by(chunk_with_states.cardinality() as u64); + .inc_by(chunk.cardinality() as u64); metrics .partition_input_bytes .with_label_values(&[&actor_id, &source_id, &split_id]) - .inc_by(chunk_with_states.estimated_size() as u64); + .inc_by(chunk.estimated_size() as u64); }), BUFFER_SIZE, ) @@ -133,6 +134,23 @@ impl SplitReader for NexmarkSplitReader { } impl NexmarkSplitReader { + async fn sleep_til_next_event(&self, start_time: Instant, start_offset: u64, start_ts: u64) { + if self.use_real_time { + tokio::time::sleep_until( + start_time + Duration::from_millis(self.generator.timestamp() - start_ts), + ) + .await; + } else if self.min_event_gap_in_ns > 0 { + tokio::time::sleep_until( + start_time + + Duration::from_nanos( + self.min_event_gap_in_ns * (self.generator.global_offset() - start_offset), + ), + ) + .await; + } + } + #[try_stream(boxed, ok = StreamChunk, error = RwError)] async fn into_native_stream(mut self) { let start_time = Instant::now(); @@ -141,45 +159,35 @@ impl NexmarkSplitReader { let mut event_dtypes_with_offset = get_event_data_types(self.event_type, self.row_id_index); event_dtypes_with_offset.extend([DataType::Varchar, DataType::Varchar]); + let mut chunk_builder = + StreamChunkBuilder::new(self.max_chunk_size as usize, event_dtypes_with_offset); loop { - let mut rows = vec![]; - while (rows.len() as u64) < self.max_chunk_size { - if self.generator.global_offset() >= self.event_num { - break; - } - let event = self.generator.next().unwrap(); - let mut fields = match self.event_type { - Some(_) => event_to_row(event, self.row_id_index), - None => combined_event_to_row(new_combined_event(event), self.row_id_index), - }; - fields.extend([ - Some(ScalarImpl::Utf8(self.split_id.as_ref().into())), - Some(ScalarImpl::Utf8( - self.generator.offset().to_string().into_boxed_str(), - )), - ]); - rows.push((Op::Insert, OwnedRow::new(fields))); - } - if rows.is_empty() { + if self.generator.global_offset() >= self.event_num { break; } - if self.use_real_time { - tokio::time::sleep_until( - start_time + Duration::from_millis(self.generator.timestamp() - start_ts), - ) - .await; - } else if self.min_event_gap_in_ns > 0 { - tokio::time::sleep_until( - start_time - + Duration::from_nanos( - self.min_event_gap_in_ns - * (self.generator.global_offset() - start_offset), - ), - ) - .await; + let event = self.generator.next().unwrap(); + let mut fields = match self.event_type { + Some(_) => event_to_row(event, self.row_id_index), + None => combined_event_to_row(new_combined_event(event), self.row_id_index), + }; + fields.extend([ + Some(ScalarImpl::Utf8(self.split_id.as_ref().into())), + Some(ScalarImpl::Utf8( + self.generator.offset().to_string().into_boxed_str(), + )), + ]); + + if let Some(chunk) = chunk_builder.append_row(Op::Insert, OwnedRow::new(fields)) { + self.sleep_til_next_event(start_time, start_offset, start_ts) + .await; + yield chunk; } + } - yield StreamChunk::from_rows(&rows, &event_dtypes_with_offset); + if let Some(chunk) = chunk_builder.take() { + self.sleep_til_next_event(start_time, start_offset, start_ts) + .await; + yield chunk; } tracing::debug!(?self.event_type, "nexmark generator finished"); @@ -227,4 +235,45 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn test_nexmark_event_num() -> Result<()> { + let max_chunk_size = 32; + let event_num = max_chunk_size * 128 + 1; + let props = NexmarkProperties { + split_num: 1, + min_event_gap_in_ns: 0, + table_type: None, + max_chunk_size, + event_num, + ..Default::default() + }; + + let mut enumerator = + NexmarkSplitEnumerator::new(props.clone(), SourceEnumeratorContext::default().into()) + .await?; + let list_splits_resp: Vec<_> = enumerator.list_splits().await?.into_iter().collect(); + + for split in list_splits_resp { + let state = vec![split]; + let reader = NexmarkSplitReader::new( + props.clone(), + state, + Default::default(), + Default::default(), + None, + ) + .await? + .into_stream(); + let (rows_count, chunk_count) = reader + .fold((0, 0), |acc, x| async move { + (acc.0 + x.unwrap().cardinality(), acc.1 + 1) + }) + .await; + assert_eq!(rows_count as u64, event_num); + assert_eq!(chunk_count as u64, event_num / max_chunk_size + 1); + } + + Ok(()) + } } From 487fc1c01a772a06d0b060fcb6475e0d1b93f51b Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Thu, 25 Jan 2024 10:24:15 +0800 Subject: [PATCH 39/42] fix warning --- src/connector/src/source/datagen/source/generator.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/connector/src/source/datagen/source/generator.rs b/src/connector/src/source/datagen/source/generator.rs index d2574f0d84a66..644359ab69afe 100644 --- a/src/connector/src/source/datagen/source/generator.rs +++ b/src/connector/src/source/datagen/source/generator.rs @@ -175,7 +175,7 @@ impl DatagenEventGenerator { MAX_ROWS_PER_YIELD, self.partition_rows_per_second - rows_generated_this_second, ); - 'outer: for i in 0..num_rows_to_generate { + 'outer: for _ in 0..num_rows_to_generate { let mut row = Vec::with_capacity(self.data_types.len()); for (field_generator, field_name) in self.fields_vec.iter_mut().zip_eq_fast(&self.field_names) From f78c2563ab3acfe7bd9a6e5300433a200757a677 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Thu, 25 Jan 2024 12:13:47 +0800 Subject: [PATCH 40/42] fix unit test --- src/connector/src/source/datagen/source/generator.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/connector/src/source/datagen/source/generator.rs b/src/connector/src/source/datagen/source/generator.rs index 644359ab69afe..e51d51e9fb889 100644 --- a/src/connector/src/source/datagen/source/generator.rs +++ b/src/connector/src/source/datagen/source/generator.rs @@ -170,7 +170,6 @@ impl DatagenEventGenerator { let mut chunk_builder = StreamChunkBuilder::new(MAX_ROWS_PER_YIELD as usize, self.data_types.clone()); while rows_generated_this_second < self.partition_rows_per_second { - self.offset += 1; let num_rows_to_generate = std::cmp::min( MAX_ROWS_PER_YIELD, self.partition_rows_per_second - rows_generated_this_second, @@ -209,6 +208,7 @@ impl DatagenEventGenerator { row.push(datum); } + self.offset += 1; rows_generated_this_second += 1; if let Some(chunk) = chunk_builder.append_row(Op::Insert, OwnedRow::new(row)) { yield chunk; From a2888cf2a07021d579bde4649009ce74af1bbf02 Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Thu, 25 Jan 2024 16:27:57 +0800 Subject: [PATCH 41/42] add comments --- src/connector/src/source/reader/desc.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/connector/src/source/reader/desc.rs b/src/connector/src/source/reader/desc.rs index f85aa3bc97087..d6c6a3e2138f3 100644 --- a/src/connector/src/source/reader/desc.rs +++ b/src/connector/src/source/reader/desc.rs @@ -90,6 +90,8 @@ impl SourceDescBuilder { } } + /// This function builds SourceColumnDesc from ColumnCatalog, and handle the creation + /// of hidden columns like partition/file, offset that are not specified by user. pub fn column_catalogs_to_source_column_descs(&self) -> Vec { let mut columns_exist = [false; 2]; let mut last_column_id = self From e39c486c1307482631a2bf8ed3e1877b9b86652d Mon Sep 17 00:00:00 2001 From: Rossil <40714231+Rossil2012@users.noreply.github.com> Date: Thu, 25 Jan 2024 17:55:53 +0800 Subject: [PATCH 42/42] fix conflicts --- .../src/source/datagen/source/generator.rs | 1 + src/connector/src/source/reader/desc.rs | 65 +++++++++++-------- src/frontend/src/handler/create_source.rs | 2 + src/stream/src/executor/source/mod.rs | 11 +++- 4 files changed, 49 insertions(+), 30 deletions(-) diff --git a/src/connector/src/source/datagen/source/generator.rs b/src/connector/src/source/datagen/source/generator.rs index e51d51e9fb889..b298d8eafcfa1 100644 --- a/src/connector/src/source/datagen/source/generator.rs +++ b/src/connector/src/source/datagen/source/generator.rs @@ -180,6 +180,7 @@ impl DatagenEventGenerator { self.fields_vec.iter_mut().zip_eq_fast(&self.field_names) { let datum = match field_generator { + // TODO: avoid distinguishing hidden partition/offset columns by name FieldDesc::Invisible => match field_name.as_str() { "_rw_datagen_partition" => { Some(ScalarImpl::Utf8(self.split_id.as_ref().into())) diff --git a/src/connector/src/source/reader/desc.rs b/src/connector/src/source/reader/desc.rs index d6c6a3e2138f3..e049be8bbe940 100644 --- a/src/connector/src/source/reader/desc.rs +++ b/src/connector/src/source/reader/desc.rs @@ -20,18 +20,19 @@ use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_pb::catalog::PbStreamSourceInfo; -use risingwave_pb::plan_common::{AdditionalColumnType, PbColumnCatalog}; +use risingwave_pb::plan_common::additional_column::ColumnType; +use risingwave_pb::plan_common::{AdditionalColumn, PbColumnCatalog}; #[expect(deprecated)] use super::fs_reader::FsSourceReader; use super::reader::SourceReader; -use crate::parser::additional_columns::common_compatible_column_vec; +use crate::parser::additional_columns::{ + build_additional_column_catalog, COMMON_COMPATIBLE_ADDITIONAL_COLUMNS, + COMPATIBLE_ADDITIONAL_COLUMNS, +}; use crate::parser::{EncodingProperties, ProtocolProperties, SpecificParserConfig}; use crate::source::monitor::SourceMetrics; -use crate::source::{ - get_connector_compatible_additional_columns, SourceColumnDesc, SourceColumnType, - UPSTREAM_SOURCE_KEY, -}; +use crate::source::{SourceColumnDesc, SourceColumnType, UPSTREAM_SOURCE_KEY}; use crate::ConnectorParams; pub const DEFAULT_CONNECTOR_MESSAGE_BUFFER_SIZE: usize = 16; @@ -90,7 +91,7 @@ impl SourceDescBuilder { } } - /// This function builds SourceColumnDesc from ColumnCatalog, and handle the creation + /// This function builds `SourceColumnDesc` from `ColumnCatalog`, and handle the creation /// of hidden columns like partition/file, offset that are not specified by user. pub fn column_catalogs_to_source_column_descs(&self) -> Vec { let mut columns_exist = [false; 2]; @@ -107,36 +108,46 @@ impl SourceDescBuilder { .unwrap(); let additional_columns: Vec<_> = { - let col_list = get_connector_compatible_additional_columns(&connector_name) - .unwrap_or(common_compatible_column_vec()); + let compat_col_types = COMPATIBLE_ADDITIONAL_COLUMNS + .get(&*connector_name) + .unwrap_or(&COMMON_COMPATIBLE_ADDITIONAL_COLUMNS); ["partition", "file", "offset"] .iter() - .filter_map(|key_name| { - let col_name = format!("_rw_{}_{}", connector_name, key_name); - col_list.iter().find_map(|(n, f)| { - if key_name == n { - last_column_id = last_column_id.next(); - Some(f(last_column_id, &col_name).to_protobuf()) - } else { - None - } - }) + .filter_map(|col_type| { + last_column_id = last_column_id.next(); + if compat_col_types.contains(col_type) { + Some( + build_additional_column_catalog( + last_column_id, + &connector_name, + col_type, + None, + None, + None, + false, + ) + .unwrap() + .to_protobuf(), + ) + } else { + None + } }) .collect() }; assert_eq!(additional_columns.len(), 2); + // Check if partition/file/offset columns are included explicitly. for col in &self.columns { - match col - .column_desc - .as_ref() - .unwrap() - .get_additional_column_type() - { - Ok(AdditionalColumnType::Partition | AdditionalColumnType::Filename) => { + match col.column_desc.as_ref().unwrap().get_additional_columns() { + Ok(AdditionalColumn { + column_type: Some(ColumnType::Partition(_) | ColumnType::Filename(_)), + }) => { columns_exist[0] = true; } - Ok(AdditionalColumnType::Offset) => { + Ok(AdditionalColumn { + column_type: Some(ColumnType::Offset(_)), + }) => { columns_exist[1] = true; } _ => (), diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index a03e074301cd1..5f25d12650f0c 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -544,6 +544,7 @@ pub fn handle_addition_columns( item.column_alias.map(|alias| alias.real_value()), item.inner_field.as_deref(), data_type_name.as_deref(), + true, )?); } @@ -863,6 +864,7 @@ fn check_and_add_timestamp_column( Some(KAFKA_TIMESTAMP_COLUMN_NAME.to_string()), None, None, + true, ) .unwrap(); catalog.is_hidden = true; diff --git a/src/stream/src/executor/source/mod.rs b/src/stream/src/executor/source/mod.rs index 2fd684dc28271..7df7cc2ea8373 100644 --- a/src/stream/src/executor/source/mod.rs +++ b/src/stream/src/executor/source/mod.rs @@ -25,7 +25,8 @@ use risingwave_common::array::StreamChunk; use risingwave_common::bail; use risingwave_common::row::Row; use risingwave_connector::source::{SourceColumnDesc, SplitId}; -use risingwave_pb::plan_common::AdditionalColumnType; +use risingwave_pb::plan_common::additional_column::ColumnType; +use risingwave_pb::plan_common::AdditionalColumn; pub use state_table_handler::*; pub mod fetch_executor; pub use fetch_executor::*; @@ -71,10 +72,14 @@ pub fn get_split_offset_col_idx( let mut offset_idx = None; for (idx, column) in column_descs.iter().enumerate() { match column.additional_column_type { - AdditionalColumnType::Partition | AdditionalColumnType::Filename => { + AdditionalColumn { + column_type: Some(ColumnType::Partition(_) | ColumnType::Filename(_)), + } => { split_idx = Some(idx); } - AdditionalColumnType::Offset => { + AdditionalColumn { + column_type: Some(ColumnType::Offset(_)), + } => { offset_idx = Some(idx); } _ => (),