From eb7b03507d2172b9a0359a8d65e4310bec548523 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 24 Oct 2023 16:36:05 +0800 Subject: [PATCH 01/47] save work --- Cargo.lock | 1 + src/connector/Cargo.toml | 1 + src/connector/src/source/filesystem/mod.rs | 1 + .../source/filesystem/opendal_source/gcs.rs | 44 ++++++ .../source/filesystem/opendal_source/mod.rs | 16 +++ .../opendal_source/opendal_enumerator.rs | 127 ++++++++++++++++++ 6 files changed, 190 insertions(+) create mode 100644 src/connector/src/source/filesystem/opendal_source/gcs.rs create mode 100644 src/connector/src/source/filesystem/opendal_source/mod.rs create mode 100644 src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs diff --git a/Cargo.lock b/Cargo.lock index bb58e6cac3fa1..80e3e110eb965 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7285,6 +7285,7 @@ dependencies = [ "mysql_common", "nexmark", "num-bigint", + "opendal 0.39.0", "parking_lot 0.12.1", "paste", "prometheus", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index abb7486de3091..fa0c7f5917fb0 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -73,6 +73,7 @@ mysql_async = { version = "0.32", default-features = false, features = [ mysql_common = { version = "0.30", default-features = false, features = [ "chrono", ] } +opendal = "0.39" nexmark = { version = "0.2", features = ["serde"] } num-bigint = "0.4" parking_lot = "0.12" diff --git a/src/connector/src/source/filesystem/mod.rs b/src/connector/src/source/filesystem/mod.rs index 8f2587384280b..8b2ca5b56c1d3 100644 --- a/src/connector/src/source/filesystem/mod.rs +++ b/src/connector/src/source/filesystem/mod.rs @@ -19,4 +19,5 @@ pub mod nd_streaming; pub use file_common::{FsPage, FsPageItem, FsSplit}; mod s3; pub mod s3_v2; +pub mod opendal_source; pub const S3_V2_CONNECTOR: &str = "s3_v2"; diff --git a/src/connector/src/source/filesystem/opendal_source/gcs.rs b/src/connector/src/source/filesystem/opendal_source/gcs.rs new file mode 100644 index 0000000000000..89104e9308cfb --- /dev/null +++ b/src/connector/src/source/filesystem/opendal_source/gcs.rs @@ -0,0 +1,44 @@ + // Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use opendal::{services::Gcs, layers::{LoggingLayer, RetryLayer}, Operator}; + +use super::opendal_enumerator::{OpendalSource, EngineType}; + +impl OpendalSource { + /// create opendal gcs engine. + pub fn new_gcs_source(bucket: String, root: String) -> anyhow::Result { + // Create gcs backend builder. + let mut builder = Gcs::default(); + + builder.bucket(&bucket); + + builder.root(&root); + + + // if credential env is set, use it. Otherwise, ADC will be used. + let cred = std::env::var("GOOGLE_APPLICATION_CREDENTIALS"); + if let Ok(cred) = cred { + builder.credential(&cred); + } + let op: Operator = Operator::new(builder)? + .layer(LoggingLayer::default()) + .layer(RetryLayer::default()) + .finish(); + Ok(Self { + op, + engine_type: EngineType::Gcs, + }) + } +} \ No newline at end of file diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs new file mode 100644 index 0000000000000..2ec531490ef03 --- /dev/null +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -0,0 +1,16 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +pub mod gcs; +pub mod opendal_enumerator; diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs new file mode 100644 index 0000000000000..ac724b7e06554 --- /dev/null +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -0,0 +1,127 @@ + // Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use async_nats::jetstream::object_store::ObjectMetadata; +use opendal::{Operator, Lister}; + +pub struct OpendalSource { + pub(crate) op: Operator, + pub(crate) engine_type: EngineType, +} + +impl OpendalSource{ + async fn list(&self, prefix: &str) -> ObjectResult { + let lister = self.op.scan(prefix).await?; + Ok(Box::pin(OpenDalObjectIter::new(lister, self.op.clone()))) + } +} + +#[derive(Clone)] +pub enum EngineType { + + Gcs, +} + + +struct OpenDalSourceLister { + lister: Option, + op: Option, + #[allow(clippy::type_complexity)] + next_future: Option>, Lister)>>, + #[allow(clippy::type_complexity)] + metadata_future: Option, Operator)>>, +} + +impl OpenDalSourceLister{ + fn new(lister: Lister, op: Operator) -> Self { + Self { + lister: Some(lister), + op: Some(op), + next_future: None, + metadata_future: None, + } + } +} + +impl Stream for OpenDalObjectIter { + type Item = ObjectResult; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + if let Some(metadata_future) = self.metadata_future.as_mut() { + let (result, op) = ready!(metadata_future.poll_unpin(cx)); + self.op = Some(op); + return match result { + Ok(m) => { + self.metadata_future = None; + Poll::Ready(Some(Ok(m))) + } + Err(e) => { + self.metadata_future = None; + Poll::Ready(Some(Err(e.into()))) + } + }; + } + if let Some(next_future) = self.next_future.as_mut() { + let (option, lister) = ready!(next_future.poll_unpin(cx)); + self.lister = Some(lister); + return match option { + None => { + self.next_future = None; + Poll::Ready(None) + } + Some(result) => { + self.next_future = None; + match result { + Ok(object) => { + let op = self.op.take().expect("op should not be None"); + let f = async move { + let key = object.path().to_string(); + // FIXME: How does opendal metadata cache work? + // Will below line result in one IO per object? + let om = match op + .metadata( + &object, + Metakey::LastModified | Metakey::ContentLength, + ) + .await + { + Ok(om) => om, + Err(e) => return (Err(e), op), + }; + let last_modified = match om.last_modified() { + Some(t) => t.timestamp() as f64, + None => 0_f64, + }; + let total_size = om.content_length() as usize; + let metadata = ObjectMetadata { + key, + last_modified, + total_size, + }; + (Ok(metadata), op) + }; + self.metadata_future = Some(Box::pin(f)); + self.poll_next(cx) + } + Err(e) => Poll::Ready(Some(Err(e.into()))), + } + } + }; + } + let mut lister = self.lister.take().expect("list should not be None"); + let f = async move { (lister.next().await, lister) }; + self.next_future = Some(Box::pin(f)); + self.poll_next(cx) + } +} \ No newline at end of file From 4fa7226a366ce01e439e84413974c0ba14bb8633 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 31 Oct 2023 11:30:01 +0800 Subject: [PATCH 02/47] save work --- Cargo.lock | 51 +----- src/connector/Cargo.toml | 2 +- .../opendal_source/opendal_enumerator.rs | 171 +++++++++--------- 3 files changed, 87 insertions(+), 137 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c374fcb7ad3cb..b2457aa244fcd 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3898,7 +3898,7 @@ dependencies = [ "log", "murmur3", "once_cell", - "opendal 0.40.0", + "opendal", "ordered-float 3.9.1", "parquet", "regex", @@ -5268,40 +5268,6 @@ version = "11.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" -[[package]] -name = "opendal" -version = "0.40.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ddba7299bab261d3ae2f37617fb7f45b19ed872752bb4e22cf93a69d979366c5" -dependencies = [ - "anyhow", - "async-compat", - "async-trait", - "backon", - "base64 0.21.4", - "bytes", - "chrono", - "flagset", - "futures", - "http", - "hyper", - "log", - "md-5", - "once_cell", - "parking_lot 0.12.1", - "percent-encoding", - "pin-project", - "prometheus", - "quick-xml 0.29.0", - "reqsign", - "reqwest", - "serde", - "serde_json", - "sha2", - "tokio", - "uuid", -] - [[package]] name = "opendal" version = "0.41.0" @@ -5325,6 +5291,7 @@ dependencies = [ "parking_lot 0.12.1", "percent-encoding", "pin-project", + "prometheus", "quick-xml 0.30.0", "reqsign", "reqwest", @@ -6556,16 +6523,6 @@ dependencies = [ "memchr", ] -[[package]] -name = "quick-xml" -version = "0.29.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81b9228215d82c7b61490fec1de287136b5de6f5700f6e58ea9ad61a7964ca51" -dependencies = [ - "memchr", - "serde", -] - [[package]] name = "quick-xml" version = "0.30.0" @@ -7381,7 +7338,7 @@ dependencies = [ "mysql_common", "nexmark", "num-bigint", - "opendal 0.39.0", + "opendal", "parking_lot 0.12.1", "paste", "prometheus", @@ -7884,7 +7841,7 @@ dependencies = [ "itertools 0.11.0", "madsim-aws-sdk-s3", "madsim-tokio", - "opendal 0.41.0", + "opendal", "prometheus", "risingwave_common", "spin 0.9.8", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 020e4be128fb0..e544cb2cf49c6 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -73,7 +73,7 @@ mysql_async = { version = "0.32", default-features = false, features = [ mysql_common = { version = "0.30", default-features = false, features = [ "chrono", ] } -opendal = "0.39" +opendal = "0.41" nexmark = { version = "0.2", features = ["serde"] } num-bigint = "0.4" parking_lot = "0.12" diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index ac724b7e06554..72ced815342db 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -13,20 +13,52 @@ // limitations under the License. use async_nats::jetstream::object_store::ObjectMetadata; -use opendal::{Operator, Lister}; +use opendal::{Operator, Lister, Metakey}; +use futures::{stream::{self, BoxStream}, StreamExt}; +use crate::source::FsListInner; pub struct OpendalSource { pub(crate) op: Operator, pub(crate) engine_type: EngineType, } impl OpendalSource{ - async fn list(&self, prefix: &str) -> ObjectResult { - let lister = self.op.scan(prefix).await?; - Ok(Box::pin(OpenDalObjectIter::new(lister, self.op.clone()))) + async fn list(&self, prefix: &str) -> anyhow::Result { + let object_lister = self + .op + .lister_with(prefix) + .delimiter("") + .metakey(Metakey::ContentLength | Metakey::ContentType) + .await?; + + let stream = stream::unfold(object_lister, |mut object_lister| async move { + match object_lister.next().await { + Some(Ok(object)) => { + let key = object.path().to_string(); + let om = object.metadata(); + let last_modified = match om.last_modified() { + Some(t) => t.timestamp() as f64, + None => 0_f64, + }; + let total_size = om.content_length() as usize; + let metadata = ObjectListMetadata { + key, + last_modified, + total_size, + }; + Some((Ok(metadata), object_lister)) + } + Some(Err(err)) => Some((Err(err.into()), object_lister)), + None => None, + } + }); + + Ok(stream.boxed()) } } +pub type ObjectMetadataIter = BoxStream<'static, anyhow::Result>; + #[derive(Clone)] pub enum EngineType { @@ -34,94 +66,55 @@ pub enum EngineType { } -struct OpenDalSourceLister { - lister: Option, - op: Option, - #[allow(clippy::type_complexity)] - next_future: Option>, Lister)>>, - #[allow(clippy::type_complexity)] - metadata_future: Option, Operator)>>, +#[derive(Debug, Clone, PartialEq)] +pub struct ObjectListMetadata { + // Full path + pub key: String, + // Seconds since unix epoch. + pub last_modified: f64, + pub total_size: usize, } -impl OpenDalSourceLister{ - fn new(lister: Lister, op: Operator) -> Self { - Self { - lister: Some(lister), - op: Some(op), - next_future: None, - metadata_future: None, +#[async_trait] +impl FsListInner for OpendalSource { + async fn get_next_page From<&'a Object>>( + &mut self, + ) -> anyhow::Result<(Vec, bool)> { + let mut has_finished = false; + let mut req = self + .client + .list_objects_v2() + .bucket(&self.bucket_name) + .set_prefix(self.prefix.clone()); + if let Some(continuation_token) = self.next_continuation_token.take() { + req = req.continuation_token(continuation_token); } + let mut res = req + .send() + .await + .map_err(|e| anyhow!(DisplayErrorContext(e)))?; + if res.is_truncated() { + self.next_continuation_token = res.next_continuation_token.clone(); + } else { + has_finished = true; + self.next_continuation_token = None; + } + let objects = res.contents.take().unwrap_or_default(); + let matched_objs: Vec = objects + .iter() + .filter(|obj| obj.key().is_some()) + .filter(|obj| { + self.matcher + .as_ref() + .map(|m| m.matches(obj.key().unwrap())) + .unwrap_or(true) + }) + .map(T::from) + .collect_vec(); + Ok((matched_objs, has_finished)) } -} - -impl Stream for OpenDalObjectIter { - type Item = ObjectResult; - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - if let Some(metadata_future) = self.metadata_future.as_mut() { - let (result, op) = ready!(metadata_future.poll_unpin(cx)); - self.op = Some(op); - return match result { - Ok(m) => { - self.metadata_future = None; - Poll::Ready(Some(Ok(m))) - } - Err(e) => { - self.metadata_future = None; - Poll::Ready(Some(Err(e.into()))) - } - }; - } - if let Some(next_future) = self.next_future.as_mut() { - let (option, lister) = ready!(next_future.poll_unpin(cx)); - self.lister = Some(lister); - return match option { - None => { - self.next_future = None; - Poll::Ready(None) - } - Some(result) => { - self.next_future = None; - match result { - Ok(object) => { - let op = self.op.take().expect("op should not be None"); - let f = async move { - let key = object.path().to_string(); - // FIXME: How does opendal metadata cache work? - // Will below line result in one IO per object? - let om = match op - .metadata( - &object, - Metakey::LastModified | Metakey::ContentLength, - ) - .await - { - Ok(om) => om, - Err(e) => return (Err(e), op), - }; - let last_modified = match om.last_modified() { - Some(t) => t.timestamp() as f64, - None => 0_f64, - }; - let total_size = om.content_length() as usize; - let metadata = ObjectMetadata { - key, - last_modified, - total_size, - }; - (Ok(metadata), op) - }; - self.metadata_future = Some(Box::pin(f)); - self.poll_next(cx) - } - Err(e) => Poll::Ready(Some(Err(e.into()))), - } - } - }; - } - let mut lister = self.lister.take().expect("list should not be None"); - let f = async move { (lister.next().await, lister) }; - self.next_future = Some(Box::pin(f)); - self.poll_next(cx) + fn filter_policy(&self, _ctx: &FsFilterCtrlCtx, _page_num: usize, _item: &FsPageItem) -> bool { + true } -} \ No newline at end of file +} From 2871d05ec61d43b5da3a8f53790c163d4fcd281c Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 31 Oct 2023 17:49:59 +0800 Subject: [PATCH 03/47] save work --- src/connector/src/macros.rs | 2 + .../opendal_source/{gcs.rs => gcs_source.rs} | 27 +++++++- .../source/filesystem/opendal_source/mod.rs | 5 +- .../opendal_source/opendal_enumerator.rs | 63 ++++++------------- src/source/src/connector_source.rs | 54 +++++++++++++++- .../src/executor/source/list_executor.rs | 35 +++++++++++ 6 files changed, 136 insertions(+), 50 deletions(-) rename src/connector/src/source/filesystem/opendal_source/{gcs.rs => gcs_source.rs} (70%) diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index fdc3ed8867297..fa570e7936237 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. + #[macro_export] macro_rules! for_all_classified_sources { ($macro:path $(,$extra_args:tt)*) => { @@ -32,6 +33,7 @@ macro_rules! for_all_classified_sources { { GooglePubsub, $crate::source::google_pubsub::PubsubProperties, $crate::source::google_pubsub::PubsubSplit }, { Nats, $crate::source::nats::NatsProperties, $crate::source::nats::split::NatsSplit }, { S3, $crate::source::filesystem::S3Properties, $crate::source::filesystem::FsSplit }, + { OpenDAL, $crate::source::filesystem::GCSProperties, $crate::source::filesystem::FsSplit }, { Test, $crate::source::test_source::TestSourceProperties, $crate::source::test_source::TestSourceSplit} } $( diff --git a/src/connector/src/source/filesystem/opendal_source/gcs.rs b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs similarity index 70% rename from src/connector/src/source/filesystem/opendal_source/gcs.rs rename to src/connector/src/source/filesystem/opendal_source/gcs_source.rs index 89104e9308cfb..034b26332f1b0 100644 --- a/src/connector/src/source/filesystem/opendal_source/gcs.rs +++ b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs @@ -13,10 +13,11 @@ // limitations under the License. use opendal::{services::Gcs, layers::{LoggingLayer, RetryLayer}, Operator}; +use serde::Deserialize; -use super::opendal_enumerator::{OpendalSource, EngineType}; +use super::opendal_enumerator::{OpenDALSplitEnumerator, EngineType}; -impl OpendalSource { +impl OpenDALSplitEnumerator { /// create opendal gcs engine. pub fn new_gcs_source(bucket: String, root: String) -> anyhow::Result { // Create gcs backend builder. @@ -41,4 +42,24 @@ impl OpendalSource { engine_type: EngineType::Gcs, }) } -} \ No newline at end of file +} + +use crate::source::filesystem::FsSplit; +use crate::source::SourceProperties; + +pub const GCS_CONNECTOR: &str = "s3"; + +#[derive(Clone, Debug, Deserialize)] +pub struct GCSProperties { + #[serde(rename = "gcs.bucket_name")] + pub bucket_name: String, +} + +impl SourceProperties for GCSProperties { + type Split = FsSplit; + type SplitEnumerator = OpenDALSplitEnumerator; + type SplitReader = OpenDALSplitEnumerator; + + const SOURCE_NAME: &'static str = GCS_CONNECTOR; +} + diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index 2ec531490ef03..cbe8b12c9e602 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -12,5 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub mod gcs; +pub mod gcs_source; + pub mod opendal_enumerator; + + diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index 72ced815342db..6832d6d01a0bf 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -13,16 +13,30 @@ // limitations under the License. use async_nats::jetstream::object_store::ObjectMetadata; +use async_trait::async_trait; use opendal::{Operator, Lister, Metakey}; use futures::{stream::{self, BoxStream}, StreamExt}; -use crate::source::FsListInner; -pub struct OpendalSource { +use crate::source::{FsListInner, SplitEnumerator, SourceEnumeratorContextRef}; +pub struct OpenDALSplitEnumerator { pub(crate) op: Operator, pub(crate) engine_type: EngineType, } -impl OpendalSource{ +#[async_trait] +impl SplitEnumerator for OpenDALSplitEnumerator{ + async fn new( + properties: Self::Properties, + _context: SourceEnumeratorContextRef, + ) -> anyhow::Result { + todo!() + } + + async fn list_splits(&mut self) -> anyhow::Result> { + todo!() + } +} +impl OpenDALSplitEnumerator{ async fn list(&self, prefix: &str) -> anyhow::Result { let object_lister = self .op @@ -75,46 +89,5 @@ pub struct ObjectListMetadata { pub total_size: usize, } -#[async_trait] -impl FsListInner for OpendalSource { - async fn get_next_page From<&'a Object>>( - &mut self, - ) -> anyhow::Result<(Vec, bool)> { - let mut has_finished = false; - let mut req = self - .client - .list_objects_v2() - .bucket(&self.bucket_name) - .set_prefix(self.prefix.clone()); - if let Some(continuation_token) = self.next_continuation_token.take() { - req = req.continuation_token(continuation_token); - } - let mut res = req - .send() - .await - .map_err(|e| anyhow!(DisplayErrorContext(e)))?; - if res.is_truncated() { - self.next_continuation_token = res.next_continuation_token.clone(); - } else { - has_finished = true; - self.next_continuation_token = None; - } - let objects = res.contents.take().unwrap_or_default(); - let matched_objs: Vec = objects - .iter() - .filter(|obj| obj.key().is_some()) - .filter(|obj| { - self.matcher - .as_ref() - .map(|m| m.matches(obj.key().unwrap())) - .unwrap_or(true) - }) - .map(T::from) - .collect_vec(); - Ok((matched_objs, has_finished)) - } - fn filter_policy(&self, _ctx: &FsFilterCtrlCtx, _page_num: usize, _item: &FsPageItem) -> bool { - true - } -} + diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 31ee763d2a0b9..87522d851fcc7 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -24,7 +24,7 @@ use risingwave_common::catalog::ColumnId; use risingwave_common::error::ErrorCode::ConnectorError; use risingwave_common::error::{internal_error, Result, RwError}; use risingwave_common::util::select_all; -use risingwave_connector::dispatch_source_prop; +use risingwave_connector::{dispatch_source_prop, source::filesystem::opendal_source::opendal_enumerator::OpendalSource}; use risingwave_connector::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; use risingwave_connector::source::filesystem::{FsPage, FsPageItem, S3SplitEnumerator}; use risingwave_connector::source::{ @@ -107,6 +107,28 @@ impl ConnectorSource { )) } + pub async fn get_source_list_v2(&self) -> Result> { + let config = self.config.clone(); + let lister = match config { + ConnectorProperties::S3(prop) => { + S3SplitEnumerator::new(*prop, Arc::new(SourceEnumeratorContext::default())).await? + } + ConnectorProperties::GCS(prop) => { + GCSSplitEnumerator::new(*prop, Arc::new(SourceEnumeratorContext::default())).await? + } + other => return Err(internal_error(format!("Unsupported source: {:?}", other))), + }; + + Ok(build_fs_list_stream_v2( + FsListCtrlContext { + interval: Duration::from_secs(60), + last_tick: None, + filter_ctx: FsFilterCtrlCtx, + }, + lister, + )) + } + pub async fn stream_reader( &self, state: ConnectorState, @@ -212,3 +234,33 @@ async fn build_fs_list_stream( interval.tick().await; } } + + +#[try_stream(boxed, ok = FsPageItem, error = RwError)] +async fn build_fs_list_stream_v2( + mut ctrl_ctx: FsListCtrlContext, + mut lister: OpendalSource, +) { + let mut interval = time::interval(ctrl_ctx.interval); + interval.set_missed_tick_behavior(MissedTickBehavior::Skip); + + + + loop { + ctrl_ctx.last_tick = Some(time::Instant::now()); + 'inner: loop { + let a = lister.list("123").await?; + let (fs_page, has_finished) = lister.get_next_page::().await?; + let matched_items = fs_page + .into_iter() + .filter(|item| list_op.filter_policy(&ctrl_ctx.filter_ctx, page_num, item)) + .collect_vec(); + yield matched_items; + page_num += 1; + if !page_ctrl_logic(&ctrl_ctx, has_finished, page_num) { + break 'inner; + } + } + interval.tick().await; + } +} \ No newline at end of file diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/list_executor.rs index 53e8854594ce4..6b4becfd442e4 100644 --- a/src/stream/src/executor/source/list_executor.rs +++ b/src/stream/src/executor/source/list_executor.rs @@ -106,6 +106,21 @@ impl FsListExecutor { .boxed()) } + async fn build_chunked_paginate_stream_v2( + &self, + source_desc: &SourceDesc, + ) -> StreamExecutorResult> { + let stream = source_desc + .source + .get_source_list_v2() + .await + .map_err(StreamExecutorError::connector_error)?; + + Ok(stream + .map(|item| item.map(Self::map_fs_page_to_chunk)) + .boxed()) + } + fn map_fs_page_to_chunk(page: FsPage) -> StreamChunk { let rows = page .into_iter() @@ -126,6 +141,26 @@ impl FsListExecutor { ) } + fn map_fs_item_to_chunk(page: FsPage) -> StreamChunk { + let rows = page + .into_iter() + .map(|split| { + ( + Op::Insert, + OwnedRow::new(vec![ + Some(ScalarImpl::Utf8(split.name.into_boxed_str())), + Some(ScalarImpl::Timestamp(split.timestamp)), + Some(ScalarImpl::Int64(split.size)), + ]), + ) + }) + .collect::>(); + StreamChunk::from_rows( + &rows, + &[DataType::Varchar, DataType::Timestamp, DataType::Int64], + ) + } + #[try_stream(ok = Message, error = StreamExecutorError)] async fn into_stream(mut self) { let mut barrier_receiver = self.barrier_receiver.take().unwrap(); From 7a5321d210487baf494aa0cdb7d1807642fa0235 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 1 Nov 2023 15:47:15 +0800 Subject: [PATCH 04/47] save work --- src/connector/src/macros.rs | 3 +- src/connector/src/source/filesystem/mod.rs | 4 +- .../filesystem/opendal_source/gcs_source.rs | 14 ++--- .../source/filesystem/opendal_source/mod.rs | 4 +- .../opendal_source/opendal_enumerator.rs | 60 ++++++------------ src/source/src/connector_source.rs | 63 ++++++++----------- 6 files changed, 53 insertions(+), 95 deletions(-) diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index fa570e7936237..7f4fe65102499 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - #[macro_export] macro_rules! for_all_classified_sources { ($macro:path $(,$extra_args:tt)*) => { @@ -33,7 +32,7 @@ macro_rules! for_all_classified_sources { { GooglePubsub, $crate::source::google_pubsub::PubsubProperties, $crate::source::google_pubsub::PubsubSplit }, { Nats, $crate::source::nats::NatsProperties, $crate::source::nats::split::NatsSplit }, { S3, $crate::source::filesystem::S3Properties, $crate::source::filesystem::FsSplit }, - { OpenDAL, $crate::source::filesystem::GCSProperties, $crate::source::filesystem::FsSplit }, + { GCS, $crate::source::filesystem::GCSProperties, $crate::source::filesystem::FsSplit }, { Test, $crate::source::test_source::TestSourceProperties, $crate::source::test_source::TestSourceSplit} } $( diff --git a/src/connector/src/source/filesystem/mod.rs b/src/connector/src/source/filesystem/mod.rs index 8b2ca5b56c1d3..ceddd4cc0d385 100644 --- a/src/connector/src/source/filesystem/mod.rs +++ b/src/connector/src/source/filesystem/mod.rs @@ -12,12 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub use opendal_source::GCSProperties; pub use s3::{S3FileReader, S3Properties, S3SplitEnumerator, S3_CONNECTOR}; - mod file_common; pub mod nd_streaming; pub use file_common::{FsPage, FsPageItem, FsSplit}; +pub mod opendal_source; mod s3; pub mod s3_v2; -pub mod opendal_source; pub const S3_V2_CONNECTOR: &str = "s3_v2"; diff --git a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs index 034b26332f1b0..d5a6acf33751d 100644 --- a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs @@ -1,4 +1,4 @@ - // Copyright 2023 RisingWave Labs +// Copyright 2023 RisingWave Labs // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -12,22 +12,21 @@ // See the License for the specific language governing permissions and // limitations under the License. -use opendal::{services::Gcs, layers::{LoggingLayer, RetryLayer}, Operator}; +use opendal::layers::{LoggingLayer, RetryLayer}; +use opendal::services::Gcs; +use opendal::Operator; use serde::Deserialize; -use super::opendal_enumerator::{OpenDALSplitEnumerator, EngineType}; +use super::opendal_enumerator::{EngineType, OpenDALSplitEnumerator}; impl OpenDALSplitEnumerator { /// create opendal gcs engine. - pub fn new_gcs_source(bucket: String, root: String) -> anyhow::Result { + pub fn new_gcs_source(bucket: String) -> anyhow::Result { // Create gcs backend builder. let mut builder = Gcs::default(); builder.bucket(&bucket); - builder.root(&root); - - // if credential env is set, use it. Otherwise, ADC will be used. let cred = std::env::var("GOOGLE_APPLICATION_CREDENTIALS"); if let Ok(cred) = cred { @@ -62,4 +61,3 @@ impl SourceProperties for GCSProperties { const SOURCE_NAME: &'static str = GCS_CONNECTOR; } - diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index cbe8b12c9e602..a88e67595cd52 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -13,7 +13,5 @@ // limitations under the License. pub mod gcs_source; - +pub use gcs_source::*; pub mod opendal_enumerator; - - diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index 6832d6d01a0bf..86d851d644d1d 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -1,4 +1,4 @@ - // Copyright 2023 RisingWave Labs +// Copyright 2023 RisingWave Labs // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -14,30 +14,20 @@ use async_nats::jetstream::object_store::ObjectMetadata; use async_trait::async_trait; -use opendal::{Operator, Lister, Metakey}; -use futures::{stream::{self, BoxStream}, StreamExt}; +use futures::stream::{self, BoxStream}; +use futures::StreamExt; +use opendal::{Lister, Metakey, Operator}; +use risingwave_common::types::Timestamp; -use crate::source::{FsListInner, SplitEnumerator, SourceEnumeratorContextRef}; +use crate::source::filesystem::FsPageItem; +use crate::source::{FsListInner, SourceEnumeratorContextRef, SplitEnumerator}; pub struct OpenDALSplitEnumerator { pub(crate) op: Operator, pub(crate) engine_type: EngineType, } -#[async_trait] -impl SplitEnumerator for OpenDALSplitEnumerator{ - async fn new( - properties: Self::Properties, - _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { - todo!() - } - - async fn list_splits(&mut self) -> anyhow::Result> { - todo!() - } -} -impl OpenDALSplitEnumerator{ - async fn list(&self, prefix: &str) -> anyhow::Result { +impl OpenDALSplitEnumerator { + pub async fn list(&self, prefix: &str) -> anyhow::Result { let object_lister = self .op .lister_with(prefix) @@ -48,17 +38,19 @@ impl OpenDALSplitEnumerator{ let stream = stream::unfold(object_lister, |mut object_lister| async move { match object_lister.next().await { Some(Ok(object)) => { - let key = object.path().to_string(); + let name = object.path().to_string(); let om = object.metadata(); - let last_modified = match om.last_modified() { + + let t = match om.last_modified() { Some(t) => t.timestamp() as f64, None => 0_f64, }; - let total_size = om.content_length() as usize; - let metadata = ObjectListMetadata { - key, - last_modified, - total_size, + let timestamp = Timestamp::new(t); + let size = om.content_length() as i64; + let metadata = FsPageItem { + name, + timestamp, + size, }; Some((Ok(metadata), object_lister)) } @@ -71,23 +63,7 @@ impl OpenDALSplitEnumerator{ } } -pub type ObjectMetadataIter = BoxStream<'static, anyhow::Result>; - #[derive(Clone)] pub enum EngineType { - Gcs, } - - -#[derive(Debug, Clone, PartialEq)] -pub struct ObjectListMetadata { - // Full path - pub key: String, - // Seconds since unix epoch. - pub last_modified: f64, - pub total_size: usize, -} - - - diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 87522d851fcc7..b19b6d38e954a 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -24,8 +24,11 @@ use risingwave_common::catalog::ColumnId; use risingwave_common::error::ErrorCode::ConnectorError; use risingwave_common::error::{internal_error, Result, RwError}; use risingwave_common::util::select_all; -use risingwave_connector::{dispatch_source_prop, source::filesystem::opendal_source::opendal_enumerator::OpendalSource}; +use risingwave_connector::dispatch_source_prop; use risingwave_connector::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; +use risingwave_connector::source::filesystem::opendal_source::opendal_enumerator::{ + OpenDALSplitEnumerator, OpendalSource, +}; use risingwave_connector::source::filesystem::{FsPage, FsPageItem, S3SplitEnumerator}; use risingwave_connector::source::{ create_split_reader, BoxSourceWithStateStream, BoxTryStream, Column, ConnectorProperties, @@ -109,24 +112,24 @@ impl ConnectorSource { pub async fn get_source_list_v2(&self) -> Result> { let config = self.config.clone(); - let lister = match config { - ConnectorProperties::S3(prop) => { - S3SplitEnumerator::new(*prop, Arc::new(SourceEnumeratorContext::default())).await? - } - ConnectorProperties::GCS(prop) => { - GCSSplitEnumerator::new(*prop, Arc::new(SourceEnumeratorContext::default())).await? + match config { + ConnectorProperties::OpenDAL(prop) => { + let lister = OpenDALSplitEnumerator::new( + *prop, + Arc::new(SourceEnumeratorContext::default()), + ) + .await?; + return Ok(build_opendal_fs_list_stream( + FsListCtrlContext { + interval: Duration::from_secs(60), + last_tick: None, + filter_ctx: FsFilterCtrlCtx, + }, + lister, + )); } other => return Err(internal_error(format!("Unsupported source: {:?}", other))), }; - - Ok(build_fs_list_stream_v2( - FsListCtrlContext { - interval: Duration::from_secs(60), - last_tick: None, - filter_ctx: FsFilterCtrlCtx, - }, - lister, - )) } pub async fn stream_reader( @@ -235,32 +238,16 @@ async fn build_fs_list_stream( } } - #[try_stream(boxed, ok = FsPageItem, error = RwError)] -async fn build_fs_list_stream_v2( +async fn build_opendal_fs_list_stream( mut ctrl_ctx: FsListCtrlContext, - mut lister: OpendalSource, + mut lister: OpenDALSplitEnumerator, ) { let mut interval = time::interval(ctrl_ctx.interval); interval.set_missed_tick_behavior(MissedTickBehavior::Skip); - + let res = lister.list("123").await?; - loop { - ctrl_ctx.last_tick = Some(time::Instant::now()); - 'inner: loop { - let a = lister.list("123").await?; - let (fs_page, has_finished) = lister.get_next_page::().await?; - let matched_items = fs_page - .into_iter() - .filter(|item| list_op.filter_policy(&ctrl_ctx.filter_ctx, page_num, item)) - .collect_vec(); - yield matched_items; - page_num += 1; - if !page_ctrl_logic(&ctrl_ctx, has_finished, page_num) { - break 'inner; - } - } - interval.tick().await; - } -} \ No newline at end of file + interval.tick().await; + yield res; +} From 30235754e156b2ffdae9039bc8900c0d3e9ac6b0 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 2 Nov 2023 18:11:26 +0800 Subject: [PATCH 05/47] implement lister, reader wi[ --- src/connector/src/macros.rs | 2 +- .../src/source/filesystem/file_common.rs | 49 ++++++++++ src/connector/src/source/filesystem/mod.rs | 2 +- .../filesystem/opendal_source/gcs_source.rs | 33 ++----- .../source/filesystem/opendal_source/mod.rs | 43 +++++++++ .../opendal_source/opendal_enumerator.rs | 92 +++++++++++++++--- .../opendal_source/opendal_reader.rs | 53 +++++++++++ .../filesystem/opendal_source/s3_source.rs | 58 ++++++++++++ src/source/src/connector_source.rs | 88 +++++------------- .../src/executor/source/list_executor.rs | 93 +++++++------------ 10 files changed, 351 insertions(+), 162 deletions(-) create mode 100644 src/connector/src/source/filesystem/opendal_source/opendal_reader.rs create mode 100644 src/connector/src/source/filesystem/opendal_source/s3_source.rs diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index 7f4fe65102499..331dfc1cc7f0c 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -32,7 +32,7 @@ macro_rules! for_all_classified_sources { { GooglePubsub, $crate::source::google_pubsub::PubsubProperties, $crate::source::google_pubsub::PubsubSplit }, { Nats, $crate::source::nats::NatsProperties, $crate::source::nats::split::NatsSplit }, { S3, $crate::source::filesystem::S3Properties, $crate::source::filesystem::FsSplit }, - { GCS, $crate::source::filesystem::GCSProperties, $crate::source::filesystem::FsSplit }, + { GCS, $crate::source::filesystem::GCSProperties, $crate::source::filesystem::GcsSplit }, { Test, $crate::source::test_source::TestSourceProperties, $crate::source::test_source::TestSourceSplit} } $( diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index 85d65f40a3479..9a176776226f7 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -67,6 +67,55 @@ impl FsSplit { } } +/// [`GcsSplit`] Describes a file or a split of a file. A file is a generic concept, +/// and can be a local file, a distributed file system, or am object in S3 bucket. +#[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)] +pub struct GcsSplit { + pub name: String, + pub offset: usize, + pub size: usize, +} + +impl From<&Object> for GcsSplit { + fn from(value: &Object) -> Self { + Self { + name: value.key().unwrap().to_owned(), + offset: 0, + size: value.size() as usize, + } + } +} + +impl SplitMetaData for GcsSplit { + fn id(&self) -> SplitId { + self.name.as_str().into() + } + + fn restore_from_json(value: JsonbVal) -> anyhow::Result { + serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + } + + fn encode_to_json(&self) -> JsonbVal { + serde_json::to_value(self.clone()).unwrap().into() + } + + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + let offset = start_offset.parse().unwrap(); + self.offset = offset; + Ok(()) + } +} + +impl GcsSplit { + pub fn new(name: String, start: usize, size: usize) -> Self { + Self { + name, + offset: start, + size, + } + } +} + #[derive(Clone, Debug)] pub struct FsPageItem { pub name: String, diff --git a/src/connector/src/source/filesystem/mod.rs b/src/connector/src/source/filesystem/mod.rs index ceddd4cc0d385..b3721d6e2ca0e 100644 --- a/src/connector/src/source/filesystem/mod.rs +++ b/src/connector/src/source/filesystem/mod.rs @@ -16,7 +16,7 @@ pub use opendal_source::GCSProperties; pub use s3::{S3FileReader, S3Properties, S3SplitEnumerator, S3_CONNECTOR}; mod file_common; pub mod nd_streaming; -pub use file_common::{FsPage, FsPageItem, FsSplit}; +pub use file_common::{FsPage, FsPageItem, FsSplit, GcsSplit}; pub mod opendal_source; mod s3; pub mod s3_v2; diff --git a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs index d5a6acf33751d..2db3ad10465a7 100644 --- a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs @@ -12,20 +12,26 @@ // See the License for the specific language governing permissions and // limitations under the License. +use anyhow::{anyhow, Result}; +use async_trait::async_trait; use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::Gcs; use opendal::Operator; use serde::Deserialize; -use super::opendal_enumerator::{EngineType, OpenDALSplitEnumerator}; +use super::opendal_enumerator::{EngineType, OpenDALConnector}; +use super::GCSProperties; +use crate::parser::ParserConfig; +use crate::source::filesystem::GcsSplit; +use crate::source::{Column, SourceContextRef, SplitReader}; -impl OpenDALSplitEnumerator { +impl OpenDALConnector { /// create opendal gcs engine. - pub fn new_gcs_source(bucket: String) -> anyhow::Result { + pub fn new_gcs_source(gcs_properties: GCSProperties) -> anyhow::Result { // Create gcs backend builder. let mut builder = Gcs::default(); - builder.bucket(&bucket); + builder.bucket(&gcs_properties.bucket_name); // if credential env is set, use it. Otherwise, ADC will be used. let cred = std::env::var("GOOGLE_APPLICATION_CREDENTIALS"); @@ -42,22 +48,3 @@ impl OpenDALSplitEnumerator { }) } } - -use crate::source::filesystem::FsSplit; -use crate::source::SourceProperties; - -pub const GCS_CONNECTOR: &str = "s3"; - -#[derive(Clone, Debug, Deserialize)] -pub struct GCSProperties { - #[serde(rename = "gcs.bucket_name")] - pub bucket_name: String, -} - -impl SourceProperties for GCSProperties { - type Split = FsSplit; - type SplitEnumerator = OpenDALSplitEnumerator; - type SplitReader = OpenDALSplitEnumerator; - - const SOURCE_NAME: &'static str = GCS_CONNECTOR; -} diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index a88e67595cd52..e3ca99e4de5ba 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -14,4 +14,47 @@ pub mod gcs_source; pub use gcs_source::*; +pub mod s3_source; +pub use s3_source::*; +use serde::Deserialize; pub mod opendal_enumerator; +pub mod opendal_reader; + +use self::opendal_enumerator::OpenDALConnector; +use super::{GcsSplit, S3Properties, S3_CONNECTOR}; +use crate::source::{SourceProperties, SplitImpl}; + +pub const GCS_CONNECTOR: &str = "gcs"; + +#[derive(Clone, Debug, Deserialize)] +pub struct GCSProperties { + #[serde(rename = "gcs.bucket_name")] + pub bucket_name: String, +} + +impl SourceProperties for GCSProperties { + type Split = GcsSplit; + type SplitEnumerator = OpenDALConnector; + type SplitReader = OpenDALConnector; + + const SOURCE_NAME: &'static str = GCS_CONNECTOR; + + fn init_from_pb_source(&mut self, _source: &risingwave_pb::catalog::PbSource) {} +} + +#[derive(Clone, Debug, Deserialize)] +pub enum OpenDALProperties { + GCSProperties(GCSProperties), + S3Properties(S3Properties), +} + +// impl SourceProperties for OpenDALProperties{ + +// const SOURCE_NAME: &'static str = GCS_CONNECTOR; +// type Split = GcsSplit; + +// type SplitEnumerator = OpenDALConnector; +// type SplitReader = OpenDALConnector; + +// fn init_from_pb_source(&mut self, _source: &risingwave_pb::catalog::PbSource) {} +// } diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index 86d851d644d1d..67e9448589f8f 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -12,22 +12,54 @@ // See the License for the specific language governing permissions and // limitations under the License. -use async_nats::jetstream::object_store::ObjectMetadata; use async_trait::async_trait; +use chrono::NaiveDateTime; use futures::stream::{self, BoxStream}; use futures::StreamExt; use opendal::{Lister, Metakey, Operator}; use risingwave_common::types::Timestamp; -use crate::source::filesystem::FsPageItem; +use super::{GCSProperties, OpenDALProperties}; +use crate::source::filesystem::{FsPageItem, GcsSplit}; use crate::source::{FsListInner, SourceEnumeratorContextRef, SplitEnumerator}; -pub struct OpenDALSplitEnumerator { +pub struct OpenDALConnector { pub(crate) op: Operator, pub(crate) engine_type: EngineType, } -impl OpenDALSplitEnumerator { - pub async fn list(&self, prefix: &str) -> anyhow::Result { +#[derive(Clone)] +pub enum EngineType { + Gcs, + S3, +} + +#[async_trait] +impl SplitEnumerator for OpenDALConnector { + type Properties = GCSProperties; + type Split = GcsSplit; + + async fn new( + properties: Self::Properties, + _context: SourceEnumeratorContextRef, + ) -> anyhow::Result { + // match properties { + // OpenDALProperties::GCSProperties(gcs_properties) => { + // OpenDALConnector::new_gcs_source(gcs_properties) + // } + // OpenDALProperties::S3Properties(s3_properties) => { + // OpenDALConnector::new_s3_source(s3_properties) + // } + // } + OpenDALConnector::new_gcs_source(properties) + } + + async fn list_splits(&mut self) -> anyhow::Result> { + todo!() + } +} + +impl OpenDALConnector { + pub async fn list(&self, prefix: &str) -> anyhow::Result { let object_lister = self .op .lister_with(prefix) @@ -42,8 +74,11 @@ impl OpenDALSplitEnumerator { let om = object.metadata(); let t = match om.last_modified() { - Some(t) => t.timestamp() as f64, - None => 0_f64, + Some(t) => t.naive_utc(), + None => { + let timestamp = 0; + NaiveDateTime::from_timestamp(timestamp, 0) + } }; let timestamp = Timestamp::new(t); let size = om.content_length() as i64; @@ -61,9 +96,44 @@ impl OpenDALSplitEnumerator { Ok(stream.boxed()) } -} -#[derive(Clone)] -pub enum EngineType { - Gcs, + // #[try_stream(boxed, ok = StreamChunkWithState, 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(); + // let source_id = self.source_ctx.source_info.source_id.to_string(); + // let source_ctx = self.source_ctx.clone(); + + // let split_id = split.id(); + + // let data_stream = Self::stream_read_object( + // self.s3_client.clone(), + // self.bucket_name.clone(), + // split, + // self.source_ctx.clone(), + // ); + + // let parser = + // ByteStreamSourceParserImpl::create(self.parser_config.clone(), source_ctx).await?; + // let msg_stream = if matches!( + // parser, + // ByteStreamSourceParserImpl::Json(_) | ByteStreamSourceParserImpl::Csv(_) + // ) { + // parser.into_stream(nd_streaming::split_stream(data_stream)) + // } else { + // parser.into_stream(data_stream) + // }; + // #[for_await] + // for msg in msg_stream { + // let msg = msg?; + // self.source_ctx + // .metrics + // .partition_input_count + // .with_label_values(&[&actor_id, &source_id, &split_id]) + // .inc_by(msg.chunk.cardinality() as u64); + // yield msg; + // } + // } + // } } +pub type ObjectMetadataIter = BoxStream<'static, anyhow::Result>; diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs new file mode 100644 index 0000000000000..c24152a610068 --- /dev/null +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -0,0 +1,53 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use anyhow::{anyhow, Result}; +use async_trait::async_trait; +use futures_async_stream::try_stream; + +use super::opendal_enumerator::OpenDALConnector; +use super::{GCSProperties, OpenDALProperties}; +use crate::parser::ParserConfig; +use crate::source::filesystem::GcsSplit; +use crate::source::{ + BoxSourceWithStateStream, Column, CommonSplitReader, SourceContextRef, SourceMessage, + SplitReader, +}; +#[async_trait] +impl SplitReader for OpenDALConnector { + type Properties = GCSProperties; + type Split = GcsSplit; + + async fn new( + properties: GCSProperties, + splits: Vec, + parser_config: ParserConfig, + source_ctx: SourceContextRef, + _columns: Option>, + ) -> Result { + // match properties { + // OpenDALProperties::GCSProperties(gcs_properties) => { + // OpenDALConnector::new_gcs_source(gcs_properties) + // } + // OpenDALProperties::S3Properties(s3_properties) => { + // OpenDALConnector::new_s3_source(s3_properties) + // } + // } + OpenDALConnector::new_gcs_source(properties) + } + + fn into_stream(self) -> BoxSourceWithStateStream { + todo!() + } +} diff --git a/src/connector/src/source/filesystem/opendal_source/s3_source.rs b/src/connector/src/source/filesystem/opendal_source/s3_source.rs new file mode 100644 index 0000000000000..2d7f088e40ec9 --- /dev/null +++ b/src/connector/src/source/filesystem/opendal_source/s3_source.rs @@ -0,0 +1,58 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use anyhow::{anyhow, Result}; +use async_trait::async_trait; +use opendal::layers::{LoggingLayer, RetryLayer}; +use opendal::services::S3; +use opendal::Operator; +use serde::Deserialize; + +use super::opendal_enumerator::{EngineType, OpenDALConnector}; +use crate::parser::ParserConfig; +use crate::source::filesystem::{GcsSplit, S3Properties}; +use crate::source::{Column, SourceContextRef, SplitReader}; + +impl OpenDALConnector { + /// create opendal gcs engine. + pub fn new_s3_source(s3_properties: S3Properties) -> anyhow::Result { + // Create gcs backend builder. + let mut builder = S3::default(); + + builder.bucket(&s3_properties.bucket_name); + + builder.region(&s3_properties.region_name); + + if let Some(endpoint_url) = s3_properties.endpoint_url { + builder.endpoint(&endpoint_url); + } + + if let Some(access) = s3_properties.access { + builder.access_key_id(&access); + } + + if let Some(secret) = s3_properties.secret { + builder.secret_access_key(&secret); + } + + let op: Operator = Operator::new(builder)? + .layer(LoggingLayer::default()) + .layer(RetryLayer::default()) + .finish(); + Ok(Self { + op, + engine_type: EngineType::S3, + }) + } +} diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index b19b6d38e954a..b3279ee455e12 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -26,9 +26,7 @@ use risingwave_common::error::{internal_error, Result, RwError}; use risingwave_common::util::select_all; use risingwave_connector::dispatch_source_prop; use risingwave_connector::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; -use risingwave_connector::source::filesystem::opendal_source::opendal_enumerator::{ - OpenDALSplitEnumerator, OpendalSource, -}; +use risingwave_connector::source::filesystem::opendal_source::opendal_enumerator::OpenDALConnector; use risingwave_connector::source::filesystem::{FsPage, FsPageItem, S3SplitEnumerator}; use risingwave_connector::source::{ create_split_reader, BoxSourceWithStateStream, BoxTryStream, Column, ConnectorProperties, @@ -91,34 +89,23 @@ impl ConnectorSource { .collect::>>() } - pub async fn get_source_list(&self) -> Result> { + pub async fn get_opendal_source_list(&self) -> Result> { let config = self.config.clone(); - let lister = match config { - ConnectorProperties::S3(prop) => { - S3SplitEnumerator::new(*prop, Arc::new(SourceEnumeratorContext::default())).await? - } - other => return Err(internal_error(format!("Unsupported source: {:?}", other))), - }; - - Ok(build_fs_list_stream( - FsListCtrlContext { - interval: Duration::from_secs(60), - last_tick: None, - filter_ctx: FsFilterCtrlCtx, - }, - lister, - )) - } - pub async fn get_source_list_v2(&self) -> Result> { - let config = self.config.clone(); match config { - ConnectorProperties::OpenDAL(prop) => { - let lister = OpenDALSplitEnumerator::new( - *prop, - Arc::new(SourceEnumeratorContext::default()), - ) - .await?; + ConnectorProperties::GCS(prop) => { + let lister = OpenDALConnector::new_gcs_source(*prop)?; + return Ok(build_opendal_fs_list_stream( + FsListCtrlContext { + interval: Duration::from_secs(60), + last_tick: None, + filter_ctx: FsFilterCtrlCtx, + }, + lister, + )); + } + ConnectorProperties::S3(prop) => { + let lister = OpenDALConnector::new_s3_source(*prop)?; return Ok(build_opendal_fs_list_stream( FsListCtrlContext { interval: Duration::from_secs(60), @@ -206,48 +193,21 @@ impl ConnectorSource { } } -#[try_stream(boxed, ok = FsPage, error = RwError)] -async fn build_fs_list_stream( - mut ctrl_ctx: FsListCtrlContext, - mut list_op: impl FsListInner + Send + 'static, -) { +#[try_stream(boxed, ok = FsPageItem, error = RwError)] +async fn build_opendal_fs_list_stream(ctrl_ctx: FsListCtrlContext, lister: OpenDALConnector) { let mut interval = time::interval(ctrl_ctx.interval); interval.set_missed_tick_behavior(MissedTickBehavior::Skip); - // controlling whether request for next page - fn page_ctrl_logic(_ctx: &FsListCtrlContext, has_finished: bool, _page_num: usize) -> bool { - !has_finished - } - + let mut object_metadata_iter = lister.list("123").await?; loop { - let mut page_num = 0; - ctrl_ctx.last_tick = Some(time::Instant::now()); - 'inner: loop { - let (fs_page, has_finished) = list_op.get_next_page::().await?; - let matched_items = fs_page - .into_iter() - .filter(|item| list_op.filter_policy(&ctrl_ctx.filter_ctx, page_num, item)) - .collect_vec(); - yield matched_items; - page_num += 1; - if !page_ctrl_logic(&ctrl_ctx, has_finished, page_num) { - break 'inner; + match object_metadata_iter.next().await { + Some(list_res) => { + yield list_res.unwrap(); + } + None => { + break; } } interval.tick().await; } } - -#[try_stream(boxed, ok = FsPageItem, error = RwError)] -async fn build_opendal_fs_list_stream( - mut ctrl_ctx: FsListCtrlContext, - mut lister: OpenDALSplitEnumerator, -) { - let mut interval = time::interval(ctrl_ctx.interval); - interval.set_missed_tick_behavior(MissedTickBehavior::Skip); - - let res = lister.list("123").await?; - - interval.tick().await; - yield res; -} diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/list_executor.rs index 6b4becfd442e4..81918e7e57e77 100644 --- a/src/stream/src/executor/source/list_executor.rs +++ b/src/stream/src/executor/source/list_executor.rs @@ -22,7 +22,7 @@ use futures_async_stream::try_stream; use risingwave_common::array::Op; use risingwave_common::catalog::Schema; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; -use risingwave_connector::source::filesystem::FsPage; +use risingwave_connector::source::filesystem::{FsPage, FsPageItem}; use risingwave_connector::source::{BoxTryStream, SourceCtrlOpts}; use risingwave_connector::ConnectorParams; use risingwave_source::source_desc::{SourceDesc, SourceDescBuilder}; @@ -95,70 +95,39 @@ impl FsListExecutor { &self, source_desc: &SourceDesc, ) -> StreamExecutorResult> { - let stream = source_desc + let stream: std::pin::Pin< + Box> + Send>, + > = source_desc .source - .get_source_list() + .get_opendal_source_list() .await .map_err(StreamExecutorError::connector_error)?; - - Ok(stream - .map(|item| item.map(Self::map_fs_page_to_chunk)) - .boxed()) - } - - async fn build_chunked_paginate_stream_v2( - &self, - source_desc: &SourceDesc, - ) -> StreamExecutorResult> { - let stream = source_desc - .source - .get_source_list_v2() - .await - .map_err(StreamExecutorError::connector_error)?; - - Ok(stream - .map(|item| item.map(Self::map_fs_page_to_chunk)) - .boxed()) - } - - fn map_fs_page_to_chunk(page: FsPage) -> StreamChunk { - let rows = page - .into_iter() - .map(|split| { - ( - Op::Insert, - OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(split.name.into_boxed_str())), - Some(ScalarImpl::Timestamp(split.timestamp)), - Some(ScalarImpl::Int64(split.size)), - ]), - ) - }) - .collect::>(); - StreamChunk::from_rows( - &rows, - &[DataType::Varchar, DataType::Timestamp, DataType::Int64], - ) - } - - fn map_fs_item_to_chunk(page: FsPage) -> StreamChunk { - let rows = page - .into_iter() - .map(|split| { - ( - Op::Insert, - OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(split.name.into_boxed_str())), - Some(ScalarImpl::Timestamp(split.timestamp)), - Some(ScalarImpl::Int64(split.size)), - ]), - ) - }) - .collect::>(); - StreamChunk::from_rows( - &rows, - &[DataType::Varchar, DataType::Timestamp, DataType::Int64], - ) + let chunked_stream = stream + .chunks(1024) // Group FsPageItems into chunks of size 1024 + .map(|chunk| { + let rows = chunk + .into_iter() + .map(|item| { + // Implement the conversion of FsPageItem to row here + let page_item = item.unwrap(); + ( + Op::Insert, + OwnedRow::new(vec![ + Some(ScalarImpl::Utf8(page_item.name.into_boxed_str())), + Some(ScalarImpl::Timestamp(page_item.timestamp)), + Some(ScalarImpl::Int64(page_item.size)), + ]), + ) + }) + .collect::>(); + + Ok(StreamChunk::from_rows( + &rows, + &[DataType::Varchar, DataType::Timestamp, DataType::Int64], + )) + }); + + Ok(chunked_stream.boxed()) } #[try_stream(ok = Message, error = StreamExecutorError)] From 3a7bdef1eadd491bd4081840c346301146249add Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 6 Nov 2023 16:23:52 +0800 Subject: [PATCH 06/47] save work --- .../opendal_source/opendal_reader.rs | 29 +++++++++++++++++-- 1 file changed, 27 insertions(+), 2 deletions(-) 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 c24152a610068..36a911e9efd4e 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -12,13 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{anyhow, Result}; +use anyhow::{anyhow, Result, Ok}; use async_trait::async_trait; use futures_async_stream::try_stream; +use risingwave_common::error::RwError; use super::opendal_enumerator::OpenDALConnector; use super::{GCSProperties, OpenDALProperties}; -use crate::parser::ParserConfig; +use crate::{parser::ParserConfig, source::StreamChunkWithState}; use crate::source::filesystem::GcsSplit; use crate::source::{ BoxSourceWithStateStream, Column, CommonSplitReader, SourceContextRef, SourceMessage, @@ -51,3 +52,27 @@ impl SplitReader for OpenDALConnector { todo!() } } + +impl OpenDALConnector{ + #[try_stream(boxed, ok = StreamChunkWithState, error = RwError)] + async fn into_chunk_stream(self) { + + } + + async fn streaming_read( + &self, + path: &str, + start_pos: Option, + ) -> Result<()>{ + let reader = match start_pos { + Some(start_position) => { + self.op + .reader_with(path) + .range(start_position as u64..) + .await? + } + None => self.op.reader(path).await?, + }; + Ok(()) + } +} \ No newline at end of file From 6b5a435046a309681bec35e22921ce2d6f91fe86 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 7 Nov 2023 10:18:40 +0800 Subject: [PATCH 07/47] save work --- .../opendal_source/opendal_reader.rs | 49 ++++++++++++++++++- 1 file changed, 48 insertions(+), 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 36a911e9efd4e..1c8fbf8bdffe0 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -14,9 +14,12 @@ use anyhow::{anyhow, Result, Ok}; use async_trait::async_trait; +use aws_smithy_http::byte_stream::ByteStream; use futures_async_stream::try_stream; use risingwave_common::error::RwError; - +use tokio::io::AsyncReadExt; +use tokio_stream::Stream; +use tokio::io::{AsyncRead, AsyncSeek}; use super::opendal_enumerator::OpenDALConnector; use super::{GCSProperties, OpenDALProperties}; use crate::{parser::ParserConfig, source::StreamChunkWithState}; @@ -75,4 +78,48 @@ impl OpenDALConnector{ }; Ok(()) } + + pub async fn get_object( + &self, + object_name: &str, + start: usize, + ) -> Result { + let mut reader = self.op + .reader_with(object_name) + .range(start as u64..) + .await?; + // Seek to the beginning of the object + reader.seek(0).await?; + + let mut buffer = vec![0u8; 1024]; + let mut chunk = Vec::new(); + let mut total_bytes_read = 0; + + // Read the object and convert it into chunks + loop { + let bytes_read = reader.read(&mut buffer).await?; + if bytes_read == 0 { + break; + } + + chunk.extend_from_slice(&buffer[..bytes_read]); + total_bytes_read += bytes_read; + + if total_bytes_read >= 1024 { + // Create a new StreamChunk and emit it + let stream_chunk = StreamChunk::new(chunk); + yield stream_chunk; + + // Reset the chunk and byte counter + chunk = Vec::new(); + total_bytes_read = 0; + } + } + + // If there are remaining bytes in the last chunk, emit it + if !chunk.is_empty() { + let stream_chunk = StreamChunk::new(chunk); + yield stream_chunk; + } + } } \ No newline at end of file From 28e72862b5d7adb50580d31fef5deae8cc748a6e Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 9 Nov 2023 16:44:01 +0800 Subject: [PATCH 08/47] implement opendal reader --- src/connector/src/macros.rs | 2 +- .../src/source/filesystem/file_common.rs | 10 +- src/connector/src/source/filesystem/mod.rs | 4 +- .../filesystem/opendal_source/gcs_source.rs | 14 +- .../source/filesystem/opendal_source/mod.rs | 33 +-- .../opendal_source/opendal_enumerator.rs | 34 +-- .../opendal_source/opendal_reader.rs | 206 +++++++++++------- .../filesystem/opendal_source/s3_source.rs | 11 +- .../opendal_engine/opendal_object_store.rs | 8 +- src/risedevtool/src/bin/risedev-compose.rs | 2 +- src/risedevtool/src/bin/risedev-dev.rs | 4 +- src/risedevtool/src/config.rs | 2 +- src/risedevtool/src/service_config.rs | 4 +- src/source/src/connector_source.rs | 15 +- .../src/executor/source/list_executor.rs | 2 +- 15 files changed, 199 insertions(+), 152 deletions(-) diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index 331dfc1cc7f0c..a45d3fa7ee631 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -32,7 +32,7 @@ macro_rules! for_all_classified_sources { { GooglePubsub, $crate::source::google_pubsub::PubsubProperties, $crate::source::google_pubsub::PubsubSplit }, { Nats, $crate::source::nats::NatsProperties, $crate::source::nats::split::NatsSplit }, { S3, $crate::source::filesystem::S3Properties, $crate::source::filesystem::FsSplit }, - { GCS, $crate::source::filesystem::GCSProperties, $crate::source::filesystem::GcsSplit }, + { Gcs, $crate::source::filesystem::GcsProperties, $crate::source::filesystem::OpendalSplit }, { Test, $crate::source::test_source::TestSourceProperties, $crate::source::test_source::TestSourceSplit} } $( diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index 9a176776226f7..51db37cb5eff8 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -67,16 +67,16 @@ impl FsSplit { } } -/// [`GcsSplit`] Describes a file or a split of a file. A file is a generic concept, +/// [`OpendalSplit`] Describes a file or a split of a file. A file is a generic concept, /// and can be a local file, a distributed file system, or am object in S3 bucket. #[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)] -pub struct GcsSplit { +pub struct OpendalSplit { pub name: String, pub offset: usize, pub size: usize, } -impl From<&Object> for GcsSplit { +impl From<&Object> for OpendalSplit { fn from(value: &Object) -> Self { Self { name: value.key().unwrap().to_owned(), @@ -86,7 +86,7 @@ impl From<&Object> for GcsSplit { } } -impl SplitMetaData for GcsSplit { +impl SplitMetaData for OpendalSplit { fn id(&self) -> SplitId { self.name.as_str().into() } @@ -106,7 +106,7 @@ impl SplitMetaData for GcsSplit { } } -impl GcsSplit { +impl OpendalSplit { pub fn new(name: String, start: usize, size: usize) -> Self { Self { name, diff --git a/src/connector/src/source/filesystem/mod.rs b/src/connector/src/source/filesystem/mod.rs index b3721d6e2ca0e..b0004cf56b3e6 100644 --- a/src/connector/src/source/filesystem/mod.rs +++ b/src/connector/src/source/filesystem/mod.rs @@ -12,11 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub use opendal_source::GCSProperties; +pub use opendal_source::GcsProperties; pub use s3::{S3FileReader, S3Properties, S3SplitEnumerator, S3_CONNECTOR}; mod file_common; pub mod nd_streaming; -pub use file_common::{FsPage, FsPageItem, FsSplit, GcsSplit}; +pub use file_common::{FsPage, FsPageItem, FsSplit, OpendalSplit}; pub mod opendal_source; mod s3; pub mod s3_v2; diff --git a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs index 2db3ad10465a7..44bcd766cc700 100644 --- a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs @@ -12,22 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{anyhow, Result}; -use async_trait::async_trait; use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::Gcs; use opendal::Operator; -use serde::Deserialize; -use super::opendal_enumerator::{EngineType, OpenDALConnector}; -use super::GCSProperties; -use crate::parser::ParserConfig; -use crate::source::filesystem::GcsSplit; -use crate::source::{Column, SourceContextRef, SplitReader}; +use super::opendal_enumerator::{EngineType, OpendalConnector}; +use super::GcsProperties; -impl OpenDALConnector { +impl OpendalConnector { /// create opendal gcs engine. - pub fn new_gcs_source(gcs_properties: GCSProperties) -> anyhow::Result { + pub fn new_gcs_source(gcs_properties: GcsProperties) -> anyhow::Result { // Create gcs backend builder. let mut builder = Gcs::default(); diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index e3ca99e4de5ba..d62dda83c0930 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -20,41 +20,42 @@ use serde::Deserialize; pub mod opendal_enumerator; pub mod opendal_reader; -use self::opendal_enumerator::OpenDALConnector; -use super::{GcsSplit, S3Properties, S3_CONNECTOR}; -use crate::source::{SourceProperties, SplitImpl}; +use self::opendal_enumerator::OpendalConnector; +use self::opendal_reader::OpendalReader; +use super::OpendalSplit; +use crate::source::SourceProperties; pub const GCS_CONNECTOR: &str = "gcs"; #[derive(Clone, Debug, Deserialize)] -pub struct GCSProperties { +pub struct GcsProperties { #[serde(rename = "gcs.bucket_name")] pub bucket_name: String, } -impl SourceProperties for GCSProperties { - type Split = GcsSplit; - type SplitEnumerator = OpenDALConnector; - type SplitReader = OpenDALConnector; +impl SourceProperties for GcsProperties { + type Split = OpendalSplit; + type SplitEnumerator = OpendalConnector; + type SplitReader = OpendalReader; const SOURCE_NAME: &'static str = GCS_CONNECTOR; fn init_from_pb_source(&mut self, _source: &risingwave_pb::catalog::PbSource) {} } -#[derive(Clone, Debug, Deserialize)] -pub enum OpenDALProperties { - GCSProperties(GCSProperties), - S3Properties(S3Properties), -} +// #[derive(Clone, Debug, Deserialize)] +// pub enum OpenDALProperties { +// GcsProperties(GcsProperties), +// S3Properties(S3Properties), +// } // impl SourceProperties for OpenDALProperties{ // const SOURCE_NAME: &'static str = GCS_CONNECTOR; -// type Split = GcsSplit; +// type Split = OpendalSplit; -// type SplitEnumerator = OpenDALConnector; -// type SplitReader = OpenDALConnector; +// type SplitEnumerator = OpendalConnector; +// type SplitReader = OpendalReader; // fn init_from_pb_source(&mut self, _source: &risingwave_pb::catalog::PbSource) {} // } diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index 67e9448589f8f..af4f779244b50 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -16,13 +16,13 @@ use async_trait::async_trait; use chrono::NaiveDateTime; use futures::stream::{self, BoxStream}; use futures::StreamExt; -use opendal::{Lister, Metakey, Operator}; +use opendal::{Metakey, Operator}; use risingwave_common::types::Timestamp; -use super::{GCSProperties, OpenDALProperties}; -use crate::source::filesystem::{FsPageItem, GcsSplit}; -use crate::source::{FsListInner, SourceEnumeratorContextRef, SplitEnumerator}; -pub struct OpenDALConnector { +use super::GcsProperties; +use crate::source::filesystem::{FsPageItem, OpendalSplit}; +use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; +pub struct OpendalConnector { pub(crate) op: Operator, pub(crate) engine_type: EngineType, } @@ -34,31 +34,31 @@ pub enum EngineType { } #[async_trait] -impl SplitEnumerator for OpenDALConnector { - type Properties = GCSProperties; - type Split = GcsSplit; +impl SplitEnumerator for OpendalConnector { + type Properties = GcsProperties; + type Split = OpendalSplit; async fn new( properties: Self::Properties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> anyhow::Result { // match properties { - // OpenDALProperties::GCSProperties(gcs_properties) => { - // OpenDALConnector::new_gcs_source(gcs_properties) + // OpenDALProperties::GcsProperties(gcs_properties) => { + // OpendalConnector::new_gcs_source(gcs_properties) // } // OpenDALProperties::S3Properties(s3_properties) => { - // OpenDALConnector::new_s3_source(s3_properties) + // OpendalConnector::new_s3_source(s3_properties) // } // } - OpenDALConnector::new_gcs_source(properties) + OpendalConnector::new_gcs_source(properties) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> anyhow::Result> { todo!() } } -impl OpenDALConnector { +impl OpendalConnector { pub async fn list(&self, prefix: &str) -> anyhow::Result { let object_lister = self .op @@ -77,15 +77,15 @@ impl OpenDALConnector { Some(t) => t.naive_utc(), None => { let timestamp = 0; - NaiveDateTime::from_timestamp(timestamp, 0) + NaiveDateTime::from_timestamp_opt(timestamp, 0).unwrap() } }; let timestamp = Timestamp::new(t); let size = om.content_length() as i64; let metadata = FsPageItem { name, - timestamp, size, + timestamp, }; Some((Ok(metadata), object_lister)) } 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 1c8fbf8bdffe0..b8c390b9ecb0c 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -12,114 +12,172 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{anyhow, Result, Ok}; +use std::pin::pin; + +use anyhow::{Ok, Result}; use async_trait::async_trait; use aws_smithy_http::byte_stream::ByteStream; +use futures::TryStreamExt; use futures_async_stream::try_stream; +use opendal::Operator; use risingwave_common::error::RwError; -use tokio::io::AsyncReadExt; -use tokio_stream::Stream; -use tokio::io::{AsyncRead, AsyncSeek}; -use super::opendal_enumerator::OpenDALConnector; -use super::{GCSProperties, OpenDALProperties}; -use crate::{parser::ParserConfig, source::StreamChunkWithState}; -use crate::source::filesystem::GcsSplit; +use tokio::io::{AsyncReadExt, BufReader}; +use tokio_util::io::{ReaderStream, StreamReader}; + +use super::opendal_enumerator::OpendalConnector; +use super::GcsProperties; +use crate::parser::{ByteStreamSourceParserImpl, ParserConfig}; +use crate::source::filesystem::{nd_streaming, OpendalSplit}; use crate::source::{ - BoxSourceWithStateStream, Column, CommonSplitReader, SourceContextRef, SourceMessage, - SplitReader, + BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SourceMeta, SplitMetaData, + SplitReader, StreamChunkWithState, }; + +const MAX_CHANNEL_BUFFER_SIZE: usize = 2048; +const STREAM_READER_CAPACITY: usize = 4096; +pub struct OpendalReader { + connector: OpendalConnector, + splits: Vec, + parser_config: ParserConfig, + source_ctx: SourceContextRef, +} #[async_trait] -impl SplitReader for OpenDALConnector { - type Properties = GCSProperties; - type Split = GcsSplit; +impl SplitReader for OpendalReader { + type Properties = GcsProperties; + type Split = OpendalSplit; async fn new( - properties: GCSProperties, - splits: Vec, + properties: GcsProperties, + splits: Vec, parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, ) -> Result { // match properties { - // OpenDALProperties::GCSProperties(gcs_properties) => { - // OpenDALConnector::new_gcs_source(gcs_properties) + // OpenDALProperties::GcsProperties(gcs_properties) => { + // OpendalConnector::new_gcs_source(gcs_properties) // } // OpenDALProperties::S3Properties(s3_properties) => { - // OpenDALConnector::new_s3_source(s3_properties) + // OpendalConnector::new_s3_source(s3_properties) // } // } - OpenDALConnector::new_gcs_source(properties) + let connector = OpendalConnector::new_gcs_source(properties)?; + let opendal_reader = OpendalReader { + connector, + splits, + parser_config, + source_ctx, + }; + Ok(opendal_reader) } fn into_stream(self) -> BoxSourceWithStateStream { - todo!() + self.into_chunk_stream() } } -impl OpenDALConnector{ +impl OpendalReader { #[try_stream(boxed, ok = StreamChunkWithState, 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(); + let source_id = self.source_ctx.source_info.source_id.to_string(); + let source_ctx = self.source_ctx.clone(); + + let split_id = split.id(); + + let data_stream = + Self::stream_read_object(self.connector.op.clone(), split, self.source_ctx.clone()); - async fn streaming_read( - &self, - path: &str, - start_pos: Option, - ) -> Result<()>{ - let reader = match start_pos { - Some(start_position) => { - self.op - .reader_with(path) - .range(start_position as u64..) - .await? + let parser = + ByteStreamSourceParserImpl::create(self.parser_config.clone(), source_ctx).await?; + let msg_stream = if matches!( + parser, + ByteStreamSourceParserImpl::Json(_) | ByteStreamSourceParserImpl::Csv(_) + ) { + parser.into_stream(nd_streaming::split_stream(data_stream)) + } else { + parser.into_stream(data_stream) + }; + #[for_await] + for msg in msg_stream { + let msg = msg?; + self.source_ctx + .metrics + .partition_input_count + .with_label_values(&[&actor_id, &source_id, &split_id]) + .inc_by(msg.chunk.cardinality() as u64); + yield msg; } - None => self.op.reader(path).await?, - }; - Ok(()) + } } - pub async fn get_object( - &self, - object_name: &str, - start: usize, - ) -> Result { - let mut reader = self.op - .reader_with(object_name) - .range(start as u64..) - .await?; - // Seek to the beginning of the object - reader.seek(0).await?; - - let mut buffer = vec![0u8; 1024]; - let mut chunk = Vec::new(); - let mut total_bytes_read = 0; - - // Read the object and convert it into chunks - loop { - let bytes_read = reader.read(&mut buffer).await?; - if bytes_read == 0 { - break; - } + #[try_stream(boxed, ok = Vec, error = anyhow::Error)] + pub async fn stream_read_object( + op: Operator, + split: OpendalSplit, + source_ctx: SourceContextRef, + ) { + let actor_id = source_ctx.source_info.actor_id.to_string(); + let source_id = source_ctx.source_info.source_id.to_string(); + let max_chunk_size = source_ctx.source_ctrl_opts.chunk_size; + let split_id = split.id(); + + let object_name = split.name.clone(); + + let byte_stream = Self::get_object(op, &object_name, split.offset).await?; + + let stream_reader = StreamReader::new( + byte_stream.map_err(|e| std::io::Error::new(std::io::ErrorKind::Other, e)), + ); - chunk.extend_from_slice(&buffer[..bytes_read]); - total_bytes_read += bytes_read; + let reader = pin!(BufReader::new(stream_reader)); - if total_bytes_read >= 1024 { - // Create a new StreamChunk and emit it - let stream_chunk = StreamChunk::new(chunk); - yield stream_chunk; + let stream = ReaderStream::with_capacity(reader, STREAM_READER_CAPACITY); - // Reset the chunk and byte counter - chunk = Vec::new(); - total_bytes_read = 0; + let mut offset: usize = split.offset; + let mut batch_size: usize = 0; + let mut batch = Vec::new(); + #[for_await] + for read in stream { + let bytes = read?; + let len = bytes.len(); + let msg = SourceMessage { + key: None, + payload: Some(bytes.as_ref().to_vec()), + offset: offset.to_string(), + split_id: split.id(), + meta: SourceMeta::Empty, + }; + offset += len; + batch_size += len; + batch.push(msg); + if batch.len() >= max_chunk_size { + source_ctx + .metrics + .partition_input_bytes + .with_label_values(&[&actor_id, &source_id, &split_id]) + .inc_by(batch_size as u64); + batch_size = 0; + yield batch.clone(); + batch.clear(); + } + } + if !batch.is_empty() { + source_ctx + .metrics + .partition_input_bytes + .with_label_values(&[&actor_id, &source_id, &split_id]) + .inc_by(batch_size as u64); + yield batch; } } - // If there are remaining bytes in the last chunk, emit it - if !chunk.is_empty() { - let stream_chunk = StreamChunk::new(chunk); - yield stream_chunk; + pub async fn get_object(op: Operator, object_name: &str, start: usize) -> Result { + let mut reader = op.reader_with(object_name).range(start as u64..).await?; + let mut buffer = Vec::new(); + reader.read_to_end(&mut buffer).await?; + let res = ByteStream::from(buffer); + Ok(res) } - } -} \ No newline at end of file +} diff --git a/src/connector/src/source/filesystem/opendal_source/s3_source.rs b/src/connector/src/source/filesystem/opendal_source/s3_source.rs index 2d7f088e40ec9..b25de60e8a290 100644 --- a/src/connector/src/source/filesystem/opendal_source/s3_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/s3_source.rs @@ -12,19 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{anyhow, Result}; -use async_trait::async_trait; use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::S3; use opendal::Operator; -use serde::Deserialize; -use super::opendal_enumerator::{EngineType, OpenDALConnector}; -use crate::parser::ParserConfig; -use crate::source::filesystem::{GcsSplit, S3Properties}; -use crate::source::{Column, SourceContextRef, SplitReader}; +use super::opendal_enumerator::{EngineType, OpendalConnector}; +use crate::source::filesystem::S3Properties; -impl OpenDALConnector { +impl OpendalConnector { /// create opendal gcs engine. pub fn new_s3_source(s3_properties: S3Properties) -> anyhow::Result { // Create gcs backend builder. diff --git a/src/object_store/src/object/opendal_engine/opendal_object_store.rs b/src/object_store/src/object/opendal_engine/opendal_object_store.rs index ff682946b0651..c557aba3de0ed 100644 --- a/src/object_store/src/object/opendal_engine/opendal_object_store.rs +++ b/src/object_store/src/object/opendal_engine/opendal_object_store.rs @@ -72,7 +72,7 @@ impl ObjectStore for OpendalObjectStore { async fn streaming_upload(&self, path: &str) -> ObjectResult { Ok(Box::new( - OpenDalStreamingUploader::new(self.op.clone(), path.to_string()).await?, + OpendalStreamingUploader::new(self.op.clone(), path.to_string()).await?, )) } @@ -199,10 +199,10 @@ impl ObjectStore for OpendalObjectStore { } /// Store multiple parts in a map, and concatenate them on finish. -pub struct OpenDalStreamingUploader { +pub struct OpendalStreamingUploader { writer: Writer, } -impl OpenDalStreamingUploader { +impl OpendalStreamingUploader { pub async fn new(op: Operator, path: String) -> ObjectResult { let writer = op.writer_with(&path).buffer(OPENDAL_BUFFER_SIZE).await?; Ok(Self { writer }) @@ -212,7 +212,7 @@ impl OpenDalStreamingUploader { const OPENDAL_BUFFER_SIZE: usize = 16 * 1024 * 1024; #[async_trait::async_trait] -impl StreamingUploader for OpenDalStreamingUploader { +impl StreamingUploader for OpendalStreamingUploader { async fn write_bytes(&mut self, data: Bytes) -> ObjectResult<()> { self.writer.write(data).await?; Ok(()) diff --git a/src/risedevtool/src/bin/risedev-compose.rs b/src/risedevtool/src/bin/risedev-compose.rs index 63925d919bb2b..7f3df20893438 100644 --- a/src/risedevtool/src/bin/risedev-compose.rs +++ b/src/risedevtool/src/bin/risedev-compose.rs @@ -203,7 +203,7 @@ fn main() -> Result<()> { ServiceConfig::ZooKeeper(_) => { return Err(anyhow!("not supported, please use redpanda instead")) } - ServiceConfig::OpenDal(_) => continue, + ServiceConfig::Opendal(_) => continue, ServiceConfig::AwsS3(_) => continue, ServiceConfig::RedPanda(c) => { if opts.deploy { diff --git a/src/risedevtool/src/bin/risedev-dev.rs b/src/risedevtool/src/bin/risedev-dev.rs index 474e8dd0cbd15..6ab704c696272 100644 --- a/src/risedevtool/src/bin/risedev-dev.rs +++ b/src/risedevtool/src/bin/risedev-dev.rs @@ -112,7 +112,7 @@ fn task_main( ServiceConfig::Redis(c) => Some((c.port, c.id.clone())), ServiceConfig::ZooKeeper(c) => Some((c.port, c.id.clone())), ServiceConfig::AwsS3(_) => None, - ServiceConfig::OpenDal(_) => None, + ServiceConfig::Opendal(_) => None, ServiceConfig::RedPanda(_) => None, }; @@ -271,7 +271,7 @@ fn task_main( ctx.pb .set_message(format!("using AWS s3 bucket {}", c.bucket)); } - ServiceConfig::OpenDal(c) => { + ServiceConfig::Opendal(c) => { let mut ctx = ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); diff --git a/src/risedevtool/src/config.rs b/src/risedevtool/src/config.rs index 09e530487d4f0..2b1248627e927 100644 --- a/src/risedevtool/src/config.rs +++ b/src/risedevtool/src/config.rs @@ -166,7 +166,7 @@ impl ConfigExpander { "prometheus" => ServiceConfig::Prometheus(serde_yaml::from_str(&out_str)?), "grafana" => ServiceConfig::Grafana(serde_yaml::from_str(&out_str)?), "tempo" => ServiceConfig::Tempo(serde_yaml::from_str(&out_str)?), - "opendal" => ServiceConfig::OpenDal(serde_yaml::from_str(&out_str)?), + "opendal" => ServiceConfig::Opendal(serde_yaml::from_str(&out_str)?), "aws-s3" => ServiceConfig::AwsS3(serde_yaml::from_str(&out_str)?), "kafka" => ServiceConfig::Kafka(serde_yaml::from_str(&out_str)?), "pubsub" => ServiceConfig::Pubsub(serde_yaml::from_str(&out_str)?), diff --git a/src/risedevtool/src/service_config.rs b/src/risedevtool/src/service_config.rs index 516ae872d6c31..8421b53d44572 100644 --- a/src/risedevtool/src/service_config.rs +++ b/src/risedevtool/src/service_config.rs @@ -327,7 +327,7 @@ pub enum ServiceConfig { Prometheus(PrometheusConfig), Grafana(GrafanaConfig), Tempo(TempoConfig), - OpenDal(OpendalConfig), + Opendal(OpendalConfig), AwsS3(AwsS3Config), Kafka(KafkaConfig), Pubsub(PubsubConfig), @@ -354,7 +354,7 @@ impl ServiceConfig { Self::Pubsub(c) => &c.id, Self::Redis(c) => &c.id, Self::RedPanda(c) => &c.id, - Self::OpenDal(c) => &c.id, + Self::Opendal(c) => &c.id, } } } diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index b3279ee455e12..5acd00c31dc54 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -26,12 +26,11 @@ use risingwave_common::error::{internal_error, Result, RwError}; use risingwave_common::util::select_all; use risingwave_connector::dispatch_source_prop; use risingwave_connector::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; -use risingwave_connector::source::filesystem::opendal_source::opendal_enumerator::OpenDALConnector; -use risingwave_connector::source::filesystem::{FsPage, FsPageItem, S3SplitEnumerator}; +use risingwave_connector::source::filesystem::opendal_source::opendal_enumerator::OpendalConnector; +use risingwave_connector::source::filesystem::FsPageItem; use risingwave_connector::source::{ create_split_reader, BoxSourceWithStateStream, BoxTryStream, Column, ConnectorProperties, - ConnectorState, FsFilterCtrlCtx, FsListInner, SourceColumnDesc, SourceContext, - SourceEnumeratorContext, SplitEnumerator, SplitReader, + ConnectorState, FsFilterCtrlCtx, SourceColumnDesc, SourceContext, SplitReader, }; use tokio::time; use tokio::time::{Duration, MissedTickBehavior}; @@ -93,8 +92,8 @@ impl ConnectorSource { let config = self.config.clone(); match config { - ConnectorProperties::GCS(prop) => { - let lister = OpenDALConnector::new_gcs_source(*prop)?; + ConnectorProperties::Gcs(prop) => { + let lister = OpendalConnector::new_gcs_source(*prop)?; return Ok(build_opendal_fs_list_stream( FsListCtrlContext { interval: Duration::from_secs(60), @@ -105,7 +104,7 @@ impl ConnectorSource { )); } ConnectorProperties::S3(prop) => { - let lister = OpenDALConnector::new_s3_source(*prop)?; + let lister = OpendalConnector::new_s3_source(*prop)?; return Ok(build_opendal_fs_list_stream( FsListCtrlContext { interval: Duration::from_secs(60), @@ -194,7 +193,7 @@ impl ConnectorSource { } #[try_stream(boxed, ok = FsPageItem, error = RwError)] -async fn build_opendal_fs_list_stream(ctrl_ctx: FsListCtrlContext, lister: OpenDALConnector) { +async fn build_opendal_fs_list_stream(ctrl_ctx: FsListCtrlContext, lister: OpendalConnector) { let mut interval = time::interval(ctrl_ctx.interval); interval.set_missed_tick_behavior(MissedTickBehavior::Skip); diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/list_executor.rs index 81918e7e57e77..0f3d567903d30 100644 --- a/src/stream/src/executor/source/list_executor.rs +++ b/src/stream/src/executor/source/list_executor.rs @@ -22,7 +22,7 @@ use futures_async_stream::try_stream; use risingwave_common::array::Op; use risingwave_common::catalog::Schema; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; -use risingwave_connector::source::filesystem::{FsPage, FsPageItem}; +use risingwave_connector::source::filesystem::FsPageItem; use risingwave_connector::source::{BoxTryStream, SourceCtrlOpts}; use risingwave_connector::ConnectorParams; use risingwave_source::source_desc::{SourceDesc, SourceDescBuilder}; From 0314c2a55ce9803246f5b4bdf2b40544a68d5e9d Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 9 Nov 2023 16:59:43 +0800 Subject: [PATCH 09/47] minor --- src/connector/src/macros.rs | 2 +- .../src/source/filesystem/file_common.rs | 10 ++--- src/connector/src/source/filesystem/mod.rs | 2 +- .../source/filesystem/opendal_source/mod.rs | 6 +-- .../opendal_source/opendal_enumerator.rs | 45 ++----------------- .../opendal_source/opendal_reader.rs | 10 ++--- 6 files changed, 18 insertions(+), 57 deletions(-) diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index a45d3fa7ee631..59c01f9852b7f 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -32,7 +32,7 @@ macro_rules! for_all_classified_sources { { GooglePubsub, $crate::source::google_pubsub::PubsubProperties, $crate::source::google_pubsub::PubsubSplit }, { Nats, $crate::source::nats::NatsProperties, $crate::source::nats::split::NatsSplit }, { S3, $crate::source::filesystem::S3Properties, $crate::source::filesystem::FsSplit }, - { Gcs, $crate::source::filesystem::GcsProperties, $crate::source::filesystem::OpendalSplit }, + { Gcs, $crate::source::filesystem::GcsProperties, $crate::source::filesystem::OpendalFsSplit }, { Test, $crate::source::test_source::TestSourceProperties, $crate::source::test_source::TestSourceSplit} } $( diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index 51db37cb5eff8..cd1868a0533be 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -67,16 +67,16 @@ impl FsSplit { } } -/// [`OpendalSplit`] Describes a file or a split of a file. A file is a generic concept, +/// [`OpendalFsSplit`] Describes a file or a split of a file. A file is a generic concept, /// and can be a local file, a distributed file system, or am object in S3 bucket. #[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)] -pub struct OpendalSplit { +pub struct OpendalFsSplit { pub name: String, pub offset: usize, pub size: usize, } -impl From<&Object> for OpendalSplit { +impl From<&Object> for OpendalFsSplit { fn from(value: &Object) -> Self { Self { name: value.key().unwrap().to_owned(), @@ -86,7 +86,7 @@ impl From<&Object> for OpendalSplit { } } -impl SplitMetaData for OpendalSplit { +impl SplitMetaData for OpendalFsSplit { fn id(&self) -> SplitId { self.name.as_str().into() } @@ -106,7 +106,7 @@ impl SplitMetaData for OpendalSplit { } } -impl OpendalSplit { +impl OpendalFsSplit { pub fn new(name: String, start: usize, size: usize) -> Self { Self { name, diff --git a/src/connector/src/source/filesystem/mod.rs b/src/connector/src/source/filesystem/mod.rs index b0004cf56b3e6..0d43b2ed69816 100644 --- a/src/connector/src/source/filesystem/mod.rs +++ b/src/connector/src/source/filesystem/mod.rs @@ -16,7 +16,7 @@ pub use opendal_source::GcsProperties; pub use s3::{S3FileReader, S3Properties, S3SplitEnumerator, S3_CONNECTOR}; mod file_common; pub mod nd_streaming; -pub use file_common::{FsPage, FsPageItem, FsSplit, OpendalSplit}; +pub use file_common::{FsPage, FsPageItem, FsSplit, OpendalFsSplit}; pub mod opendal_source; mod s3; pub mod s3_v2; diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index d62dda83c0930..0a9a504fdf04e 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -22,7 +22,7 @@ pub mod opendal_reader; use self::opendal_enumerator::OpendalConnector; use self::opendal_reader::OpendalReader; -use super::OpendalSplit; +use super::OpendalFsSplit; use crate::source::SourceProperties; pub const GCS_CONNECTOR: &str = "gcs"; @@ -34,7 +34,7 @@ pub struct GcsProperties { } impl SourceProperties for GcsProperties { - type Split = OpendalSplit; + type Split = OpendalFsSplit; type SplitEnumerator = OpendalConnector; type SplitReader = OpendalReader; @@ -52,7 +52,7 @@ impl SourceProperties for GcsProperties { // impl SourceProperties for OpenDALProperties{ // const SOURCE_NAME: &'static str = GCS_CONNECTOR; -// type Split = OpendalSplit; +// type Split = OpendalFsSplit; // type SplitEnumerator = OpendalConnector; // type SplitReader = OpendalReader; diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index af4f779244b50..e29e9b83a7e15 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -20,7 +20,7 @@ use opendal::{Metakey, Operator}; use risingwave_common::types::Timestamp; use super::GcsProperties; -use crate::source::filesystem::{FsPageItem, OpendalSplit}; +use crate::source::filesystem::{FsPageItem, OpendalFsSplit}; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; pub struct OpendalConnector { pub(crate) op: Operator, @@ -36,7 +36,7 @@ pub enum EngineType { #[async_trait] impl SplitEnumerator for OpendalConnector { type Properties = GcsProperties; - type Split = OpendalSplit; + type Split = OpendalFsSplit; async fn new( properties: Self::Properties, @@ -53,7 +53,7 @@ impl SplitEnumerator for OpendalConnector { OpendalConnector::new_gcs_source(properties) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> anyhow::Result> { todo!() } } @@ -96,44 +96,5 @@ impl OpendalConnector { Ok(stream.boxed()) } - - // #[try_stream(boxed, ok = StreamChunkWithState, 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(); - // let source_id = self.source_ctx.source_info.source_id.to_string(); - // let source_ctx = self.source_ctx.clone(); - - // let split_id = split.id(); - - // let data_stream = Self::stream_read_object( - // self.s3_client.clone(), - // self.bucket_name.clone(), - // split, - // self.source_ctx.clone(), - // ); - - // let parser = - // ByteStreamSourceParserImpl::create(self.parser_config.clone(), source_ctx).await?; - // let msg_stream = if matches!( - // parser, - // ByteStreamSourceParserImpl::Json(_) | ByteStreamSourceParserImpl::Csv(_) - // ) { - // parser.into_stream(nd_streaming::split_stream(data_stream)) - // } else { - // parser.into_stream(data_stream) - // }; - // #[for_await] - // for msg in msg_stream { - // let msg = msg?; - // self.source_ctx - // .metrics - // .partition_input_count - // .with_label_values(&[&actor_id, &source_id, &split_id]) - // .inc_by(msg.chunk.cardinality() as u64); - // yield msg; - // } - // } - // } } pub type ObjectMetadataIter = BoxStream<'static, anyhow::Result>; 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 b8c390b9ecb0c..422e6777ff72c 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -27,7 +27,7 @@ use tokio_util::io::{ReaderStream, StreamReader}; use super::opendal_enumerator::OpendalConnector; use super::GcsProperties; use crate::parser::{ByteStreamSourceParserImpl, ParserConfig}; -use crate::source::filesystem::{nd_streaming, OpendalSplit}; +use crate::source::filesystem::{nd_streaming, OpendalFsSplit}; use crate::source::{ BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SourceMeta, SplitMetaData, SplitReader, StreamChunkWithState, @@ -37,18 +37,18 @@ const MAX_CHANNEL_BUFFER_SIZE: usize = 2048; const STREAM_READER_CAPACITY: usize = 4096; pub struct OpendalReader { connector: OpendalConnector, - splits: Vec, + splits: Vec, parser_config: ParserConfig, source_ctx: SourceContextRef, } #[async_trait] impl SplitReader for OpendalReader { type Properties = GcsProperties; - type Split = OpendalSplit; + type Split = OpendalFsSplit; async fn new( properties: GcsProperties, - splits: Vec, + splits: Vec, parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, @@ -115,7 +115,7 @@ impl OpendalReader { #[try_stream(boxed, ok = Vec, error = anyhow::Error)] pub async fn stream_read_object( op: Operator, - split: OpendalSplit, + split: OpendalFsSplit, source_ctx: SourceContextRef, ) { let actor_id = source_ctx.source_info.actor_id.to_string(); From bb62cc2750bbb886a8bb42e6d3a3528f010f75f8 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 13 Nov 2023 14:35:07 +0800 Subject: [PATCH 10/47] introduce OpenDALConnectorTypeMarker --- src/connector/src/macros.rs | 3 +- .../src/source/filesystem/file_common.rs | 66 +++++++++++++++++-- src/connector/src/source/filesystem/mod.rs | 2 +- .../source/filesystem/opendal_source/mod.rs | 7 +- .../opendal_source/opendal_enumerator.rs | 5 +- .../opendal_source/opendal_reader.rs | 27 ++++++-- 6 files changed, 92 insertions(+), 18 deletions(-) diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index 59c01f9852b7f..0fdb4bf36e1ac 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -32,7 +32,8 @@ macro_rules! for_all_classified_sources { { GooglePubsub, $crate::source::google_pubsub::PubsubProperties, $crate::source::google_pubsub::PubsubSplit }, { Nats, $crate::source::nats::NatsProperties, $crate::source::nats::split::NatsSplit }, { S3, $crate::source::filesystem::S3Properties, $crate::source::filesystem::FsSplit }, - { Gcs, $crate::source::filesystem::GcsProperties, $crate::source::filesystem::OpendalFsSplit }, + { Gcs, $crate::source::filesystem::GcsProperties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::file_common::Gcs> }, + // { OpenDalS3, $crate::source::filesystem::GcsProperties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::file_common::OpenDalS3> }, { Test, $crate::source::test_source::TestSourceProperties, $crate::source::test_source::TestSourceSplit} } $( diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index cd1868a0533be..e459f572538d8 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -11,6 +11,10 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. +use std::fmt::Debug; +use std::hash::Hash; +use std::marker::PhantomData; + use anyhow::anyhow; use aws_sdk_s3::types::Object; use risingwave_common::types::{JsonbVal, Timestamp}; @@ -70,23 +74,73 @@ impl FsSplit { /// [`OpendalFsSplit`] Describes a file or a split of a file. A file is a generic concept, /// and can be a local file, a distributed file system, or am object in S3 bucket. #[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)] -pub struct OpendalFsSplit { +pub struct OpendalFsSplit +where + C: OpenDALConnectorTypeMarker + std::marker::Send, +{ pub name: String, pub offset: usize, pub size: usize, + marker: PhantomData, } -impl From<&Object> for OpendalFsSplit { +impl From<&Object> for OpendalFsSplit +where + C: OpenDALConnectorTypeMarker + std::marker::Send, +{ fn from(value: &Object) -> Self { Self { name: value.key().unwrap().to_owned(), offset: 0, size: value.size() as usize, + marker: PhantomData, } } } -impl SplitMetaData for OpendalFsSplit { +#[derive(Clone, Copy, Debug, PartialEq)] +pub enum OpenDALConnectorType { + OpenDalS3, + Gcs, +} + +/// A marker trait for different conventions, used for enforcing type safety. +/// +/// Implementors are [`S3`], [`Gcs`]. +pub trait OpenDALConnectorTypeMarker: 'static + Sized { + /// The extra fields in the [`PlanBase`] of this convention. + // type Extra: 'static + Eq + Hash + Clone + Debug; + + /// Get the [`OpenDALConnectorType`] enum value. + fn value() -> OpenDALConnectorType; +} + +/// The marker for batch convention. +#[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)] +pub struct Gcs; +impl OpenDALConnectorTypeMarker for Gcs { + // type Extra = Gcs; + + fn value() -> OpenDALConnectorType { + OpenDALConnectorType::Gcs + } +} + +/// The marker for batch convention. +#[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)] +pub struct OpenDalS3; +impl OpenDALConnectorTypeMarker for OpenDalS3 { + // type Extra = OpenDalS3; + + fn value() -> OpenDALConnectorType { + OpenDALConnectorType::OpenDalS3 + } +} + +impl SplitMetaData for OpendalFsSplit +where + C: OpenDALConnectorTypeMarker + std::marker::Send, +{ fn id(&self) -> SplitId { self.name.as_str().into() } @@ -106,12 +160,16 @@ impl SplitMetaData for OpendalFsSplit { } } -impl OpendalFsSplit { +impl OpendalFsSplit +where + C: OpenDALConnectorTypeMarker + std::marker::Send, +{ pub fn new(name: String, start: usize, size: usize) -> Self { Self { name, offset: start, size, + marker: PhantomData, } } } diff --git a/src/connector/src/source/filesystem/mod.rs b/src/connector/src/source/filesystem/mod.rs index 0d43b2ed69816..befa7fa715cb7 100644 --- a/src/connector/src/source/filesystem/mod.rs +++ b/src/connector/src/source/filesystem/mod.rs @@ -14,7 +14,7 @@ pub use opendal_source::GcsProperties; pub use s3::{S3FileReader, S3Properties, S3SplitEnumerator, S3_CONNECTOR}; -mod file_common; +pub mod file_common; pub mod nd_streaming; pub use file_common::{FsPage, FsPageItem, FsSplit, OpendalFsSplit}; pub mod opendal_source; diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index 0a9a504fdf04e..aeb1b1dae61fe 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -22,7 +22,8 @@ pub mod opendal_reader; use self::opendal_enumerator::OpendalConnector; use self::opendal_reader::OpendalReader; -use super::OpendalFsSplit; +use super::file_common::Gcs; +use super::{OpendalFsSplit, S3Properties}; use crate::source::SourceProperties; pub const GCS_CONNECTOR: &str = "gcs"; @@ -34,9 +35,9 @@ pub struct GcsProperties { } impl SourceProperties for GcsProperties { - type Split = OpendalFsSplit; + type Split = OpendalFsSplit; type SplitEnumerator = OpendalConnector; - type SplitReader = OpendalReader; + type SplitReader = OpendalReader; const SOURCE_NAME: &'static str = GCS_CONNECTOR; diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index e29e9b83a7e15..2fd97bb11fac5 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -20,6 +20,7 @@ use opendal::{Metakey, Operator}; use risingwave_common::types::Timestamp; use super::GcsProperties; +use crate::source::filesystem::file_common::Gcs; use crate::source::filesystem::{FsPageItem, OpendalFsSplit}; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; pub struct OpendalConnector { @@ -36,7 +37,7 @@ pub enum EngineType { #[async_trait] impl SplitEnumerator for OpendalConnector { type Properties = GcsProperties; - type Split = OpendalFsSplit; + type Split = OpendalFsSplit; async fn new( properties: Self::Properties, @@ -53,7 +54,7 @@ impl SplitEnumerator for OpendalConnector { OpendalConnector::new_gcs_source(properties) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> anyhow::Result>> { todo!() } } 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 422e6777ff72c..b70abaeef42ac 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::marker::PhantomData; use std::pin::pin; use anyhow::{Ok, Result}; @@ -27,6 +28,7 @@ use tokio_util::io::{ReaderStream, StreamReader}; use super::opendal_enumerator::OpendalConnector; use super::GcsProperties; use crate::parser::{ByteStreamSourceParserImpl, ParserConfig}; +use crate::source::filesystem::file_common::OpenDALConnectorTypeMarker; use crate::source::filesystem::{nd_streaming, OpendalFsSplit}; use crate::source::{ BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SourceMeta, SplitMetaData, @@ -35,20 +37,27 @@ use crate::source::{ const MAX_CHANNEL_BUFFER_SIZE: usize = 2048; const STREAM_READER_CAPACITY: usize = 4096; -pub struct OpendalReader { +pub struct OpendalReader +where + C: OpenDALConnectorTypeMarker + std::marker::Send, +{ connector: OpendalConnector, - splits: Vec, + splits: Vec>, parser_config: ParserConfig, source_ctx: SourceContextRef, + marker: PhantomData, } #[async_trait] -impl SplitReader for OpendalReader { +impl SplitReader for OpendalReader +where + C: OpenDALConnectorTypeMarker + std::marker::Send, +{ type Properties = GcsProperties; - type Split = OpendalFsSplit; + type Split = OpendalFsSplit; async fn new( properties: GcsProperties, - splits: Vec, + splits: Vec>, parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, @@ -67,6 +76,7 @@ impl SplitReader for OpendalReader { splits, parser_config, source_ctx, + marker: PhantomData, }; Ok(opendal_reader) } @@ -76,7 +86,10 @@ impl SplitReader for OpendalReader { } } -impl OpendalReader { +impl OpendalReader +where + C: OpenDALConnectorTypeMarker + std::marker::Send, +{ #[try_stream(boxed, ok = StreamChunkWithState, error = RwError)] async fn into_chunk_stream(self) { for split in self.splits { @@ -115,7 +128,7 @@ impl OpendalReader { #[try_stream(boxed, ok = Vec, error = anyhow::Error)] pub async fn stream_read_object( op: Operator, - split: OpendalFsSplit, + split: OpendalFsSplit, source_ctx: SourceContextRef, ) { let actor_id = source_ctx.source_info.actor_id.to_string(); From ee2f87ada466e9fa53f6f6397a332ef084437a82 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 13 Nov 2023 16:51:10 +0800 Subject: [PATCH 11/47] can apply gcs and s3 --- src/connector/src/macros.rs | 3 +- .../src/source/filesystem/file_common.rs | 51 ++++--------------- .../filesystem/opendal_source/gcs_source.rs | 12 +++-- .../source/filesystem/opendal_source/mod.rs | 37 ++++++-------- .../opendal_source/opendal_enumerator.rs | 43 +++++++++------- .../opendal_source/opendal_reader.rs | 41 ++++++--------- .../filesystem/opendal_source/s3_source.rs | 11 +++- src/connector/src/source/filesystem/s3/mod.rs | 10 +++- src/source/src/connector_source.rs | 16 ++++-- 9 files changed, 102 insertions(+), 122 deletions(-) diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index 0fdb4bf36e1ac..6747f7bb46382 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -32,8 +32,7 @@ macro_rules! for_all_classified_sources { { GooglePubsub, $crate::source::google_pubsub::PubsubProperties, $crate::source::google_pubsub::PubsubSplit }, { Nats, $crate::source::nats::NatsProperties, $crate::source::nats::split::NatsSplit }, { S3, $crate::source::filesystem::S3Properties, $crate::source::filesystem::FsSplit }, - { Gcs, $crate::source::filesystem::GcsProperties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::file_common::Gcs> }, - // { OpenDalS3, $crate::source::filesystem::GcsProperties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::file_common::OpenDalS3> }, + { Gcs, $crate::source::filesystem::opendal_source::GcsProperties , $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::GcsProperties> }, { Test, $crate::source::test_source::TestSourceProperties, $crate::source::test_source::TestSourceSplit} } $( diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index e459f572538d8..4b370ef79c8ca 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -74,9 +74,9 @@ impl FsSplit { /// [`OpendalFsSplit`] Describes a file or a split of a file. A file is a generic concept, /// and can be a local file, a distributed file system, or am object in S3 bucket. #[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)] -pub struct OpendalFsSplit +pub struct OpendalFsSplit where - C: OpenDALConnectorTypeMarker + std::marker::Send, + C: Send + Clone + 'static, { pub name: String, pub offset: usize, @@ -84,9 +84,9 @@ where marker: PhantomData, } -impl From<&Object> for OpendalFsSplit +impl From<&Object> for OpendalFsSplit where - C: OpenDALConnectorTypeMarker + std::marker::Send, + C: Send + Clone + 'static, { fn from(value: &Object) -> Self { Self { @@ -99,47 +99,14 @@ where } #[derive(Clone, Copy, Debug, PartialEq)] -pub enum OpenDALConnectorType { +pub enum OpendalEnumeratorType { OpenDalS3, Gcs, } -/// A marker trait for different conventions, used for enforcing type safety. -/// -/// Implementors are [`S3`], [`Gcs`]. -pub trait OpenDALConnectorTypeMarker: 'static + Sized { - /// The extra fields in the [`PlanBase`] of this convention. - // type Extra: 'static + Eq + Hash + Clone + Debug; - - /// Get the [`OpenDALConnectorType`] enum value. - fn value() -> OpenDALConnectorType; -} - -/// The marker for batch convention. -#[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)] -pub struct Gcs; -impl OpenDALConnectorTypeMarker for Gcs { - // type Extra = Gcs; - - fn value() -> OpenDALConnectorType { - OpenDALConnectorType::Gcs - } -} - -/// The marker for batch convention. -#[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)] -pub struct OpenDalS3; -impl OpenDALConnectorTypeMarker for OpenDalS3 { - // type Extra = OpenDalS3; - - fn value() -> OpenDALConnectorType { - OpenDALConnectorType::OpenDalS3 - } -} - -impl SplitMetaData for OpendalFsSplit +impl SplitMetaData for OpendalFsSplit where - C: OpenDALConnectorTypeMarker + std::marker::Send, + C: Sized + Send + Clone + 'static, { fn id(&self) -> SplitId { self.name.as_str().into() @@ -160,9 +127,9 @@ where } } -impl OpendalFsSplit +impl OpendalFsSplit where - C: OpenDALConnectorTypeMarker + std::marker::Send, + C: Send + Clone + 'static, { pub fn new(name: String, start: usize, size: usize) -> Self { Self { diff --git a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs index 44bcd766cc700..709ea283bb57c 100644 --- a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs @@ -12,14 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::marker::PhantomData; + use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::Gcs; use opendal::Operator; -use super::opendal_enumerator::{EngineType, OpendalConnector}; -use super::GcsProperties; +use super::opendal_enumerator::{EngineType, OpendalEnumerator}; +use super::{GcsProperties, OpenDALProperties}; -impl OpendalConnector { +impl OpendalEnumerator +where + C: Send + Clone + PartialEq + 'static + Sync, +{ /// create opendal gcs engine. pub fn new_gcs_source(gcs_properties: GcsProperties) -> anyhow::Result { // Create gcs backend builder. @@ -39,6 +44,7 @@ impl OpendalConnector { Ok(Self { op, engine_type: EngineType::Gcs, + marker: PhantomData, }) } } diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index aeb1b1dae61fe..2295ee33cad5a 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -13,6 +13,7 @@ // limitations under the License. pub mod gcs_source; + pub use gcs_source::*; pub mod s3_source; pub use s3_source::*; @@ -20,43 +21,35 @@ use serde::Deserialize; pub mod opendal_enumerator; pub mod opendal_reader; -use self::opendal_enumerator::OpendalConnector; +use self::opendal_enumerator::OpendalEnumerator; use self::opendal_reader::OpendalReader; -use super::file_common::Gcs; -use super::{OpendalFsSplit, S3Properties}; +use super::OpendalFsSplit; use crate::source::SourceProperties; pub const GCS_CONNECTOR: &str = "gcs"; -#[derive(Clone, Debug, Deserialize)] +#[derive(Clone, Debug, Deserialize, PartialEq)] pub struct GcsProperties { #[serde(rename = "gcs.bucket_name")] pub bucket_name: String, } impl SourceProperties for GcsProperties { - type Split = OpendalFsSplit; - type SplitEnumerator = OpendalConnector; - type SplitReader = OpendalReader; + type Split = OpendalFsSplit; + type SplitEnumerator = OpendalEnumerator; + type SplitReader = OpendalReader; const SOURCE_NAME: &'static str = GCS_CONNECTOR; fn init_from_pb_source(&mut self, _source: &risingwave_pb::catalog::PbSource) {} } -// #[derive(Clone, Debug, Deserialize)] -// pub enum OpenDALProperties { -// GcsProperties(GcsProperties), -// S3Properties(S3Properties), -// } - -// impl SourceProperties for OpenDALProperties{ - -// const SOURCE_NAME: &'static str = GCS_CONNECTOR; -// type Split = OpendalFsSplit; - -// type SplitEnumerator = OpendalConnector; -// type SplitReader = OpendalReader; +pub trait OpenDALProperties: Sized + Send + Clone + PartialEq + 'static + Sync { + fn new_enumerator(properties: Self) -> anyhow::Result>; +} -// fn init_from_pb_source(&mut self, _source: &risingwave_pb::catalog::PbSource) {} -// } +impl OpenDALProperties for GcsProperties { + fn new_enumerator(properties: Self) -> anyhow::Result> { + OpendalEnumerator::new_gcs_source(properties) + } +} diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index 2fd97bb11fac5..ed6fdabffb8e4 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::marker::PhantomData; + use async_trait::async_trait; use chrono::NaiveDateTime; use futures::stream::{self, BoxStream}; @@ -19,47 +21,50 @@ use futures::StreamExt; use opendal::{Metakey, Operator}; use risingwave_common::types::Timestamp; -use super::GcsProperties; -use crate::source::filesystem::file_common::Gcs; +use super::OpenDALProperties; use crate::source::filesystem::{FsPageItem, OpendalFsSplit}; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; -pub struct OpendalConnector { + +#[derive(Debug, Clone)] +pub struct OpendalEnumerator +where + C: Send + Clone + Sized + PartialEq + 'static + Sync, +{ pub(crate) op: Operator, pub(crate) engine_type: EngineType, + pub(crate) marker: PhantomData, } -#[derive(Clone)] +#[derive(Debug, Clone)] pub enum EngineType { Gcs, S3, } #[async_trait] -impl SplitEnumerator for OpendalConnector { - type Properties = GcsProperties; - type Split = OpendalFsSplit; +impl SplitEnumerator for OpendalEnumerator +where + C: Sized + Send + Clone + PartialEq + 'static + Sync, +{ + type Properties = C; + type Split = OpendalFsSplit; async fn new( properties: Self::Properties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { - // match properties { - // OpenDALProperties::GcsProperties(gcs_properties) => { - // OpendalConnector::new_gcs_source(gcs_properties) - // } - // OpenDALProperties::S3Properties(s3_properties) => { - // OpendalConnector::new_s3_source(s3_properties) - // } - // } - OpendalConnector::new_gcs_source(properties) + ) -> anyhow::Result> { + Self::Properties::new_enumerator(properties) } - async fn list_splits(&mut self) -> anyhow::Result>> { + async fn list_splits(&mut self) -> anyhow::Result>> { todo!() } } -impl OpendalConnector { +impl OpendalEnumerator +where + C: Send + Clone + Sized + PartialEq + 'static + Sync, +{ pub async fn list(&self, prefix: &str) -> anyhow::Result { let object_lister = self .op 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 b70abaeef42ac..eee0c76c00e86 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::marker::PhantomData; use std::pin::pin; use anyhow::{Ok, Result}; @@ -25,10 +24,9 @@ use risingwave_common::error::RwError; use tokio::io::{AsyncReadExt, BufReader}; use tokio_util::io::{ReaderStream, StreamReader}; -use super::opendal_enumerator::OpendalConnector; -use super::GcsProperties; +use super::opendal_enumerator::OpendalEnumerator; +use super::OpenDALProperties; use crate::parser::{ByteStreamSourceParserImpl, ParserConfig}; -use crate::source::filesystem::file_common::OpenDALConnectorTypeMarker; use crate::source::filesystem::{nd_streaming, OpendalFsSplit}; use crate::source::{ BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SourceMeta, SplitMetaData, @@ -37,46 +35,37 @@ use crate::source::{ const MAX_CHANNEL_BUFFER_SIZE: usize = 2048; const STREAM_READER_CAPACITY: usize = 4096; -pub struct OpendalReader -where - C: OpenDALConnectorTypeMarker + std::marker::Send, -{ - connector: OpendalConnector, +#[derive(Debug, Clone)] +pub struct OpendalReader { + connector: OpendalEnumerator, splits: Vec>, parser_config: ParserConfig, source_ctx: SourceContextRef, - marker: PhantomData, + // marker: PhantomData, } #[async_trait] -impl SplitReader for OpendalReader +impl SplitReader for OpendalReader where - C: OpenDALConnectorTypeMarker + std::marker::Send, + C: Send + Clone + PartialEq + 'static + Sync, { - type Properties = GcsProperties; + type Properties = C; type Split = OpendalFsSplit; async fn new( - properties: GcsProperties, + properties: C, splits: Vec>, parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, ) -> Result { - // match properties { - // OpenDALProperties::GcsProperties(gcs_properties) => { - // OpendalConnector::new_gcs_source(gcs_properties) - // } - // OpenDALProperties::S3Properties(s3_properties) => { - // OpendalConnector::new_s3_source(s3_properties) - // } - // } - let connector = OpendalConnector::new_gcs_source(properties)?; + let connector = Self::Properties::new_enumerator(properties)?; + // let connector = OpendalConnector::new_gcs_source(properties)?; let opendal_reader = OpendalReader { connector, splits, parser_config, source_ctx, - marker: PhantomData, + // marker: PhantomData, }; Ok(opendal_reader) } @@ -86,9 +75,9 @@ where } } -impl OpendalReader +impl OpendalReader where - C: OpenDALConnectorTypeMarker + std::marker::Send, + C: Send + Clone + Sized + PartialEq + 'static, { #[try_stream(boxed, ok = StreamChunkWithState, error = RwError)] async fn into_chunk_stream(self) { diff --git a/src/connector/src/source/filesystem/opendal_source/s3_source.rs b/src/connector/src/source/filesystem/opendal_source/s3_source.rs index b25de60e8a290..54e9735bd7696 100644 --- a/src/connector/src/source/filesystem/opendal_source/s3_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/s3_source.rs @@ -12,14 +12,20 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::marker::PhantomData; + use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::S3; use opendal::Operator; -use super::opendal_enumerator::{EngineType, OpendalConnector}; +use super::opendal_enumerator::{EngineType, OpendalEnumerator}; +use super::OpenDALProperties; use crate::source::filesystem::S3Properties; -impl OpendalConnector { +impl OpendalEnumerator +where + C: Sized + Send + Clone + PartialEq + 'static + Sync, +{ /// create opendal gcs engine. pub fn new_s3_source(s3_properties: S3Properties) -> anyhow::Result { // Create gcs backend builder. @@ -48,6 +54,7 @@ impl OpendalConnector { Ok(Self { op, engine_type: EngineType::S3, + marker: PhantomData, }) } } diff --git a/src/connector/src/source/filesystem/s3/mod.rs b/src/connector/src/source/filesystem/s3/mod.rs index 464a11f99fc09..35a4e45933a49 100644 --- a/src/connector/src/source/filesystem/s3/mod.rs +++ b/src/connector/src/source/filesystem/s3/mod.rs @@ -18,13 +18,15 @@ mod source; use serde::Deserialize; pub use source::S3FileReader; +use super::opendal_source::opendal_enumerator::OpendalEnumerator; +use super::opendal_source::OpenDALProperties; use crate::aws_auth::AwsAuthProps; use crate::source::filesystem::FsSplit; use crate::source::SourceProperties; pub const S3_CONNECTOR: &str = "s3"; -#[derive(Clone, Debug, Deserialize)] +#[derive(Clone, Debug, Deserialize, PartialEq)] pub struct S3Properties { #[serde(rename = "s3.region_name")] pub region_name: String, @@ -48,6 +50,12 @@ impl SourceProperties for S3Properties { const SOURCE_NAME: &'static str = S3_CONNECTOR; } +impl OpenDALProperties for S3Properties { + fn new_enumerator(properties: Self) -> anyhow::Result> { + OpendalEnumerator::new_s3_source(properties) + } +} + impl From<&S3Properties> for AwsAuthProps { fn from(props: &S3Properties) -> Self { Self { diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 5acd00c31dc54..3729c4640440c 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -26,8 +26,9 @@ use risingwave_common::error::{internal_error, Result, RwError}; use risingwave_common::util::select_all; use risingwave_connector::dispatch_source_prop; use risingwave_connector::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; -use risingwave_connector::source::filesystem::opendal_source::opendal_enumerator::OpendalConnector; -use risingwave_connector::source::filesystem::FsPageItem; +use risingwave_connector::source::filesystem::opendal_source::opendal_enumerator::OpendalEnumerator; +use risingwave_connector::source::filesystem::opendal_source::OpenDALProperties; +use risingwave_connector::source::filesystem::{FsPageItem, GcsProperties, S3Properties}; use risingwave_connector::source::{ create_split_reader, BoxSourceWithStateStream, BoxTryStream, Column, ConnectorProperties, ConnectorState, FsFilterCtrlCtx, SourceColumnDesc, SourceContext, SplitReader, @@ -93,7 +94,8 @@ impl ConnectorSource { match config { ConnectorProperties::Gcs(prop) => { - let lister = OpendalConnector::new_gcs_source(*prop)?; + let lister: OpendalEnumerator = + OpendalEnumerator::new_gcs_source(*prop)?; return Ok(build_opendal_fs_list_stream( FsListCtrlContext { interval: Duration::from_secs(60), @@ -104,7 +106,8 @@ impl ConnectorSource { )); } ConnectorProperties::S3(prop) => { - let lister = OpendalConnector::new_s3_source(*prop)?; + let lister: OpendalEnumerator = + OpendalEnumerator::new_s3_source(*prop)?; return Ok(build_opendal_fs_list_stream( FsListCtrlContext { interval: Duration::from_secs(60), @@ -193,7 +196,10 @@ impl ConnectorSource { } #[try_stream(boxed, ok = FsPageItem, error = RwError)] -async fn build_opendal_fs_list_stream(ctrl_ctx: FsListCtrlContext, lister: OpendalConnector) { +async fn build_opendal_fs_list_stream( + ctrl_ctx: FsListCtrlContext, + lister: OpendalEnumerator, +) { let mut interval = time::interval(ctrl_ctx.interval); interval.set_missed_tick_behavior(MissedTickBehavior::Skip); From c435b105f734fa75c37c9f88594638e9f81af5bd Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 13 Nov 2023 17:00:55 +0800 Subject: [PATCH 12/47] make clippy happy --- .../filesystem/opendal_source/gcs_source.rs | 4 ++-- .../source/filesystem/opendal_source/mod.rs | 4 ++-- .../opendal_source/opendal_enumerator.rs | 8 ++++---- .../opendal_source/opendal_reader.rs | 8 ++++---- .../filesystem/opendal_source/s3_source.rs | 4 ++-- src/connector/src/source/filesystem/s3/mod.rs | 4 ++-- src/source/src/connector_source.rs | 18 +++++++++--------- .../src/executor/source/list_executor.rs | 6 +++--- 8 files changed, 28 insertions(+), 28 deletions(-) diff --git a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs index 709ea283bb57c..0a790739ec824 100644 --- a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs @@ -19,9 +19,9 @@ use opendal::services::Gcs; use opendal::Operator; use super::opendal_enumerator::{EngineType, OpendalEnumerator}; -use super::{GcsProperties, OpenDALProperties}; +use super::{GcsProperties, OpenDalProperties}; -impl OpendalEnumerator +impl OpendalEnumerator where C: Send + Clone + PartialEq + 'static + Sync, { diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index 2295ee33cad5a..87156e34f0432 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -44,11 +44,11 @@ impl SourceProperties for GcsProperties { fn init_from_pb_source(&mut self, _source: &risingwave_pb::catalog::PbSource) {} } -pub trait OpenDALProperties: Sized + Send + Clone + PartialEq + 'static + Sync { +pub trait OpenDalProperties: Sized + Send + Clone + PartialEq + 'static + Sync { fn new_enumerator(properties: Self) -> anyhow::Result>; } -impl OpenDALProperties for GcsProperties { +impl OpenDalProperties for GcsProperties { fn new_enumerator(properties: Self) -> anyhow::Result> { OpendalEnumerator::new_gcs_source(properties) } diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index ed6fdabffb8e4..3bcfaab0a7aa8 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -21,12 +21,12 @@ use futures::StreamExt; use opendal::{Metakey, Operator}; use risingwave_common::types::Timestamp; -use super::OpenDALProperties; +use super::OpenDalProperties; use crate::source::filesystem::{FsPageItem, OpendalFsSplit}; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; #[derive(Debug, Clone)] -pub struct OpendalEnumerator +pub struct OpendalEnumerator where C: Send + Clone + Sized + PartialEq + 'static + Sync, { @@ -42,7 +42,7 @@ pub enum EngineType { } #[async_trait] -impl SplitEnumerator for OpendalEnumerator +impl SplitEnumerator for OpendalEnumerator where C: Sized + Send + Clone + PartialEq + 'static + Sync, { @@ -61,7 +61,7 @@ where } } -impl OpendalEnumerator +impl OpendalEnumerator where C: Send + Clone + Sized + PartialEq + 'static + Sync, { 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 eee0c76c00e86..3920c0a8f2a01 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -25,7 +25,7 @@ use tokio::io::{AsyncReadExt, BufReader}; use tokio_util::io::{ReaderStream, StreamReader}; use super::opendal_enumerator::OpendalEnumerator; -use super::OpenDALProperties; +use super::OpenDalProperties; use crate::parser::{ByteStreamSourceParserImpl, ParserConfig}; use crate::source::filesystem::{nd_streaming, OpendalFsSplit}; use crate::source::{ @@ -36,7 +36,7 @@ use crate::source::{ const MAX_CHANNEL_BUFFER_SIZE: usize = 2048; const STREAM_READER_CAPACITY: usize = 4096; #[derive(Debug, Clone)] -pub struct OpendalReader { +pub struct OpendalReader { connector: OpendalEnumerator, splits: Vec>, parser_config: ParserConfig, @@ -44,7 +44,7 @@ pub struct OpendalReader { // marker: PhantomData, } #[async_trait] -impl SplitReader for OpendalReader +impl SplitReader for OpendalReader where C: Send + Clone + PartialEq + 'static + Sync, { @@ -75,7 +75,7 @@ where } } -impl OpendalReader +impl OpendalReader where C: Send + Clone + Sized + PartialEq + 'static, { diff --git a/src/connector/src/source/filesystem/opendal_source/s3_source.rs b/src/connector/src/source/filesystem/opendal_source/s3_source.rs index 54e9735bd7696..cbf5bf401d0a7 100644 --- a/src/connector/src/source/filesystem/opendal_source/s3_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/s3_source.rs @@ -19,10 +19,10 @@ use opendal::services::S3; use opendal::Operator; use super::opendal_enumerator::{EngineType, OpendalEnumerator}; -use super::OpenDALProperties; +use super::OpenDalProperties; use crate::source::filesystem::S3Properties; -impl OpendalEnumerator +impl OpendalEnumerator where C: Sized + Send + Clone + PartialEq + 'static + Sync, { diff --git a/src/connector/src/source/filesystem/s3/mod.rs b/src/connector/src/source/filesystem/s3/mod.rs index 35a4e45933a49..0479e3f659de2 100644 --- a/src/connector/src/source/filesystem/s3/mod.rs +++ b/src/connector/src/source/filesystem/s3/mod.rs @@ -19,7 +19,7 @@ use serde::Deserialize; pub use source::S3FileReader; use super::opendal_source::opendal_enumerator::OpendalEnumerator; -use super::opendal_source::OpenDALProperties; +use super::opendal_source::OpenDalProperties; use crate::aws_auth::AwsAuthProps; use crate::source::filesystem::FsSplit; use crate::source::SourceProperties; @@ -50,7 +50,7 @@ impl SourceProperties for S3Properties { const SOURCE_NAME: &'static str = S3_CONNECTOR; } -impl OpenDALProperties for S3Properties { +impl OpenDalProperties for S3Properties { fn new_enumerator(properties: Self) -> anyhow::Result> { OpendalEnumerator::new_s3_source(properties) } diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 3729c4640440c..135bdae33b846 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -27,7 +27,7 @@ use risingwave_common::util::select_all; use risingwave_connector::dispatch_source_prop; use risingwave_connector::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; use risingwave_connector::source::filesystem::opendal_source::opendal_enumerator::OpendalEnumerator; -use risingwave_connector::source::filesystem::opendal_source::OpenDALProperties; +use risingwave_connector::source::filesystem::opendal_source::OpenDalProperties; use risingwave_connector::source::filesystem::{FsPageItem, GcsProperties, S3Properties}; use risingwave_connector::source::{ create_split_reader, BoxSourceWithStateStream, BoxTryStream, Column, ConnectorProperties, @@ -89,36 +89,36 @@ impl ConnectorSource { .collect::>>() } - pub async fn get_opendal_source_list(&self) -> Result> { + pub fn get_opendal_source_list(&self) -> Result> { let config = self.config.clone(); match config { ConnectorProperties::Gcs(prop) => { let lister: OpendalEnumerator = OpendalEnumerator::new_gcs_source(*prop)?; - return Ok(build_opendal_fs_list_stream( + Ok(build_opendal_fs_list_stream( FsListCtrlContext { interval: Duration::from_secs(60), last_tick: None, filter_ctx: FsFilterCtrlCtx, }, lister, - )); + )) } ConnectorProperties::S3(prop) => { let lister: OpendalEnumerator = OpendalEnumerator::new_s3_source(*prop)?; - return Ok(build_opendal_fs_list_stream( + Ok(build_opendal_fs_list_stream( FsListCtrlContext { interval: Duration::from_secs(60), last_tick: None, filter_ctx: FsFilterCtrlCtx, }, lister, - )); + )) } - other => return Err(internal_error(format!("Unsupported source: {:?}", other))), - }; + other => Err(internal_error(format!("Unsupported source: {:?}", other))), + } } pub async fn stream_reader( @@ -196,7 +196,7 @@ impl ConnectorSource { } #[try_stream(boxed, ok = FsPageItem, error = RwError)] -async fn build_opendal_fs_list_stream( +async fn build_opendal_fs_list_stream( ctrl_ctx: FsListCtrlContext, lister: OpendalEnumerator, ) { diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/list_executor.rs index 0f3d567903d30..0a142569fbdf2 100644 --- a/src/stream/src/executor/source/list_executor.rs +++ b/src/stream/src/executor/source/list_executor.rs @@ -91,7 +91,8 @@ impl FsListExecutor { } } - async fn build_chunked_paginate_stream( + #[allow(clippy::disallowed_types)] + fn build_chunked_paginate_stream( &self, source_desc: &SourceDesc, ) -> StreamExecutorResult> { @@ -100,7 +101,6 @@ impl FsListExecutor { > = source_desc .source .get_opendal_source_list() - .await .map_err(StreamExecutorError::connector_error)?; let chunked_stream = stream .chunks(1024) // Group FsPageItems into chunks of size 1024 @@ -156,7 +156,7 @@ impl FsListExecutor { // Return the ownership of `stream_source_core` to the source executor. self.stream_source_core = Some(core); - let chunked_paginate_stream = self.build_chunked_paginate_stream(&source_desc).await?; + let chunked_paginate_stream = self.build_chunked_paginate_stream(&source_desc)?; let barrier_stream = barrier_to_message_stream(barrier_receiver).boxed(); let mut stream = From 536bed4ca249cd223ed13715800421f9e62b4a0f Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 14 Nov 2023 15:03:01 +0800 Subject: [PATCH 13/47] bring new s3 source --- src/connector/src/macros.rs | 1 + .../source/filesystem/opendal_source/mod.rs | 24 ++++++++++++++++++- src/connector/src/source/filesystem/s3/mod.rs | 8 ------- src/source/src/connector_source.rs | 12 ++++++---- 4 files changed, 31 insertions(+), 14 deletions(-) diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index 6747f7bb46382..81780c43bf28d 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -33,6 +33,7 @@ macro_rules! for_all_classified_sources { { Nats, $crate::source::nats::NatsProperties, $crate::source::nats::split::NatsSplit }, { S3, $crate::source::filesystem::S3Properties, $crate::source::filesystem::FsSplit }, { Gcs, $crate::source::filesystem::opendal_source::GcsProperties , $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::GcsProperties> }, + { OpenDalS3, $crate::source::filesystem::opendal_source::OpendalS3Properties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalS3Properties> }, { Test, $crate::source::test_source::TestSourceProperties, $crate::source::test_source::TestSourceSplit} } $( diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index 87156e34f0432..35ed1743bfc9b 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -23,10 +23,11 @@ pub mod opendal_reader; use self::opendal_enumerator::OpendalEnumerator; use self::opendal_reader::OpendalReader; -use super::OpendalFsSplit; +use super::{OpendalFsSplit, S3Properties}; use crate::source::SourceProperties; pub const GCS_CONNECTOR: &str = "gcs"; +pub const OPENDAL_S3_CONNECTOR: &str = "opendal_s3"; #[derive(Clone, Debug, Deserialize, PartialEq)] pub struct GcsProperties { @@ -44,6 +45,21 @@ impl SourceProperties for GcsProperties { fn init_from_pb_source(&mut self, _source: &risingwave_pb::catalog::PbSource) {} } +#[derive(Clone, Debug, Deserialize, PartialEq)] +pub struct OpendalS3Properties { + pub s3_properties: S3Properties, +} + +impl SourceProperties for OpendalS3Properties { + type Split = OpendalFsSplit; + type SplitEnumerator = OpendalEnumerator; + type SplitReader = OpendalReader; + + const SOURCE_NAME: &'static str = OPENDAL_S3_CONNECTOR; + + fn init_from_pb_source(&mut self, _source: &risingwave_pb::catalog::PbSource) {} +} + pub trait OpenDalProperties: Sized + Send + Clone + PartialEq + 'static + Sync { fn new_enumerator(properties: Self) -> anyhow::Result>; } @@ -53,3 +69,9 @@ impl OpenDalProperties for GcsProperties { OpendalEnumerator::new_gcs_source(properties) } } + +impl OpenDalProperties for OpendalS3Properties { + fn new_enumerator(properties: Self) -> anyhow::Result> { + OpendalEnumerator::new_s3_source(properties.s3_properties) + } +} diff --git a/src/connector/src/source/filesystem/s3/mod.rs b/src/connector/src/source/filesystem/s3/mod.rs index 0479e3f659de2..9dc1a7a4ee6da 100644 --- a/src/connector/src/source/filesystem/s3/mod.rs +++ b/src/connector/src/source/filesystem/s3/mod.rs @@ -18,8 +18,6 @@ mod source; use serde::Deserialize; pub use source::S3FileReader; -use super::opendal_source::opendal_enumerator::OpendalEnumerator; -use super::opendal_source::OpenDalProperties; use crate::aws_auth::AwsAuthProps; use crate::source::filesystem::FsSplit; use crate::source::SourceProperties; @@ -50,12 +48,6 @@ impl SourceProperties for S3Properties { const SOURCE_NAME: &'static str = S3_CONNECTOR; } -impl OpenDalProperties for S3Properties { - fn new_enumerator(properties: Self) -> anyhow::Result> { - OpendalEnumerator::new_s3_source(properties) - } -} - impl From<&S3Properties> for AwsAuthProps { fn from(props: &S3Properties) -> Self { Self { diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 135bdae33b846..fd4e6e7197d93 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -27,8 +27,10 @@ use risingwave_common::util::select_all; use risingwave_connector::dispatch_source_prop; use risingwave_connector::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; use risingwave_connector::source::filesystem::opendal_source::opendal_enumerator::OpendalEnumerator; -use risingwave_connector::source::filesystem::opendal_source::OpenDalProperties; -use risingwave_connector::source::filesystem::{FsPageItem, GcsProperties, S3Properties}; +use risingwave_connector::source::filesystem::opendal_source::{ + OpenDalProperties, OpendalS3Properties, +}; +use risingwave_connector::source::filesystem::{FsPageItem, GcsProperties}; use risingwave_connector::source::{ create_split_reader, BoxSourceWithStateStream, BoxTryStream, Column, ConnectorProperties, ConnectorState, FsFilterCtrlCtx, SourceColumnDesc, SourceContext, SplitReader, @@ -105,9 +107,9 @@ impl ConnectorSource { lister, )) } - ConnectorProperties::S3(prop) => { - let lister: OpendalEnumerator = - OpendalEnumerator::new_s3_source(*prop)?; + ConnectorProperties::OpenDalS3(prop) => { + let lister: OpendalEnumerator = + OpendalEnumerator::new_s3_source(prop.s3_properties)?; Ok(build_opendal_fs_list_stream( FsListCtrlContext { interval: Duration::from_secs(60), From e52a5997dd143372732600c9515c5553c0d44881 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 14 Nov 2023 15:11:42 +0800 Subject: [PATCH 14/47] minor --- .../src/source/filesystem/opendal_source/s3_source.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/connector/src/source/filesystem/opendal_source/s3_source.rs b/src/connector/src/source/filesystem/opendal_source/s3_source.rs index cbf5bf401d0a7..2b0113aacca0d 100644 --- a/src/connector/src/source/filesystem/opendal_source/s3_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/s3_source.rs @@ -26,9 +26,9 @@ impl OpendalEnumerator where C: Sized + Send + Clone + PartialEq + 'static + Sync, { - /// create opendal gcs engine. + /// create opendal s3 engine. pub fn new_s3_source(s3_properties: S3Properties) -> anyhow::Result { - // Create gcs backend builder. + // Create s3 backend builder. let mut builder = S3::default(); builder.bucket(&s3_properties.bucket_name); From 078f506a35b9f8069afca9b94f7860fba490e17b Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 17 Nov 2023 15:25:37 +0800 Subject: [PATCH 15/47] can use opendal s3 as source, todo: add filter for prefix --- src/connector/src/source/base.rs | 40 ++++++++++++++++--- .../filesystem/opendal_source/gcs_source.rs | 12 ++++++ .../source/filesystem/opendal_source/mod.rs | 35 +++++++++++++++- .../opendal_source/opendal_enumerator.rs | 25 ++++++++++-- .../opendal_source/opendal_reader.rs | 1 + .../filesystem/opendal_source/s3_source.rs | 17 +++++++- src/connector/src/source/mod.rs | 1 + src/frontend/src/handler/create_source.rs | 10 ++++- src/source/src/connector_source.rs | 34 ++++++---------- 9 files changed, 139 insertions(+), 36 deletions(-) diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index e5193291a6d14..ec71d623c220d 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -39,9 +39,10 @@ use super::google_pubsub::GooglePubsubMeta; use super::kafka::KafkaMeta; use super::monitor::SourceMetrics; use super::nexmark::source::message::NexmarkMeta; +use super::{GCS_CONNECTOR, OPENDAL_S3_CONNECTOR}; use crate::parser::ParserConfig; pub(crate) use crate::source::common::CommonSplitReader; -use crate::source::filesystem::{FsPageItem, S3Properties, S3_V2_CONNECTOR}; +use crate::source::filesystem::{FsPageItem, S3Properties}; use crate::source::monitor::EnumeratorMetrics; use crate::source::S3_CONNECTOR; use crate::{ @@ -365,27 +366,54 @@ impl ConnectorProperties { pub fn is_new_fs_connector_b_tree_map(props: &BTreeMap) -> bool { props .get(UPSTREAM_SOURCE_KEY) - .map(|s| s.eq_ignore_ascii_case(S3_V2_CONNECTOR)) + .map(|s| s.eq_ignore_ascii_case(OPENDAL_S3_CONNECTOR)) .unwrap_or(false) + || props + .get(UPSTREAM_SOURCE_KEY) + .map(|s| s.eq_ignore_ascii_case(GCS_CONNECTOR)) + .unwrap_or(false) } pub fn is_new_fs_connector_hash_map(props: &HashMap) -> bool { props .get(UPSTREAM_SOURCE_KEY) - .map(|s| s.eq_ignore_ascii_case(S3_V2_CONNECTOR)) + .map(|s| s.eq_ignore_ascii_case(OPENDAL_S3_CONNECTOR)) .unwrap_or(false) + || props + .get(UPSTREAM_SOURCE_KEY) + .map(|s| s.eq_ignore_ascii_case(GCS_CONNECTOR)) + .unwrap_or(false) } pub fn rewrite_upstream_source_key_hash_map(props: &mut HashMap) { let connector = props.remove(UPSTREAM_SOURCE_KEY).unwrap(); match connector.as_str() { - S3_V2_CONNECTOR => { + // S3_V2_CONNECTOR => { + // tracing::info!( + // "using new fs source, rewrite connector from '{}' to '{}'", + // S3_V2_CONNECTOR, + // S3_CONNECTOR + // ); + // props.insert(UPSTREAM_SOURCE_KEY.to_string(), S3_CONNECTOR.to_string()); + // } + OPENDAL_S3_CONNECTOR => { tracing::info!( "using new fs source, rewrite connector from '{}' to '{}'", - S3_V2_CONNECTOR, + OPENDAL_S3_CONNECTOR, S3_CONNECTOR ); - props.insert(UPSTREAM_SOURCE_KEY.to_string(), S3_CONNECTOR.to_string()); + props.insert( + UPSTREAM_SOURCE_KEY.to_string(), + OPENDAL_S3_CONNECTOR.to_string(), + ); + } + GCS_CONNECTOR => { + tracing::info!( + "using new fs source, rewrite connector from '{}' to '{}'", + GCS_CONNECTOR, + S3_CONNECTOR + ); + props.insert(UPSTREAM_SOURCE_KEY.to_string(), GCS_CONNECTOR.to_string()); } _ => { props.insert(UPSTREAM_SOURCE_KEY.to_string(), connector); diff --git a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs index 0a790739ec824..4903fa1a215d6 100644 --- a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs @@ -41,9 +41,21 @@ where .layer(LoggingLayer::default()) .layer(RetryLayer::default()) .finish(); + + // todo(wcy-fdu): add (prefix, matcher) for gcs_properties + // let (prefix, matcher) = if let Some(pattern) = gcs_properties.match_pattern.as_ref() { + // let prefix = get_prefix(pattern); + // let matcher = glob::Pattern::new(pattern) + // .with_context(|| format!("Invalid match_pattern: {}", pattern))?; + // (Some(prefix), Some(matcher)) + // } else { + // (None, None) + // }; Ok(Self { op, engine_type: EngineType::Gcs, + prefix: None, + matcher: None, marker: PhantomData, }) } diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index 35ed1743bfc9b..f287025707e57 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -27,7 +27,7 @@ use super::{OpendalFsSplit, S3Properties}; use crate::source::SourceProperties; pub const GCS_CONNECTOR: &str = "gcs"; -pub const OPENDAL_S3_CONNECTOR: &str = "opendal_s3"; +pub const OPENDAL_S3_CONNECTOR: &str = "s3_v2"; #[derive(Clone, Debug, Deserialize, PartialEq)] pub struct GcsProperties { @@ -75,3 +75,36 @@ impl OpenDalProperties for OpendalS3Properties { OpendalEnumerator::new_s3_source(properties.s3_properties) } } + +/// Get the prefix from a glob +pub fn get_prefix(glob: &str) -> String { + let mut escaped = false; + let mut escaped_filter = false; + glob.chars() + .take_while(|c| match (c, &escaped) { + ('*', false) => false, + ('[', false) => false, + ('{', false) => false, + ('\\', false) => { + escaped = true; + true + } + (_, false) => true, + (_, true) => { + escaped = false; + true + } + }) + .filter(|c| match (c, &escaped_filter) { + (_, true) => { + escaped_filter = false; + true + } + ('\\', false) => { + escaped_filter = true; + false + } + (_, _) => true, + }) + .collect() +} diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index 3bcfaab0a7aa8..38c63d5792eac 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -32,6 +32,9 @@ where { pub(crate) op: Operator, pub(crate) engine_type: EngineType, + // prefix is used to reduce the number of objects to be listed + pub(crate) prefix: Option, + pub(crate) matcher: Option, pub(crate) marker: PhantomData, } @@ -65,10 +68,17 @@ impl OpendalEnumerator where C: Send + Clone + Sized + PartialEq + 'static + Sync, { - pub async fn list(&self, prefix: &str) -> anyhow::Result { + pub async fn list(&self) -> anyhow::Result { + let _prefix = match &self.prefix { + Some(prefix) => prefix, + None => "", + }; + // Currently, we need to do full list and then filter the prefix and matcher, + // After OpenDAL implementing the list prefix, we can use the user-specified prefix. + // https://github.com/apache/incubator-opendal/issues/3247 let object_lister = self .op - .lister_with(prefix) + .lister_with("/") .delimiter("") .metakey(Metakey::ContentLength | Metakey::ContentType) .await?; @@ -76,6 +86,7 @@ where let stream = stream::unfold(object_lister, |mut object_lister| async move { match object_lister.next().await { Some(Ok(object)) => { + // todo: manual filtering prefix let name = object.path().to_string(); let om = object.metadata(); @@ -95,8 +106,14 @@ where }; Some((Ok(metadata), object_lister)) } - Some(Err(err)) => Some((Err(err.into()), object_lister)), - None => None, + Some(Err(err)) => { + tracing::error!("list fail"); + Some((Err(err.into()), object_lister)) + } + None => { + tracing::info!("list to the end"); + None + } } }); 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 3920c0a8f2a01..4559cbd3278e2 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -176,6 +176,7 @@ where } pub async fn get_object(op: Operator, object_name: &str, start: usize) -> Result { + println!("读{:?}", object_name); let mut reader = op.reader_with(object_name).range(start as u64..).await?; let mut buffer = Vec::new(); reader.read_to_end(&mut buffer).await?; diff --git a/src/connector/src/source/filesystem/opendal_source/s3_source.rs b/src/connector/src/source/filesystem/opendal_source/s3_source.rs index 2b0113aacca0d..17fefb0623fdc 100644 --- a/src/connector/src/source/filesystem/opendal_source/s3_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/s3_source.rs @@ -14,12 +14,14 @@ use std::marker::PhantomData; +use anyhow::Context; use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::S3; use opendal::Operator; use super::opendal_enumerator::{EngineType, OpendalEnumerator}; use super::OpenDalProperties; +use crate::source::filesystem::opendal_source::get_prefix; use crate::source::filesystem::S3Properties; impl OpendalEnumerator @@ -30,9 +32,7 @@ where pub fn new_s3_source(s3_properties: S3Properties) -> anyhow::Result { // Create s3 backend builder. let mut builder = S3::default(); - builder.bucket(&s3_properties.bucket_name); - builder.region(&s3_properties.region_name); if let Some(endpoint_url) = s3_properties.endpoint_url { @@ -47,13 +47,26 @@ where builder.secret_access_key(&secret); } + builder.disable_config_load(); + let (prefix, matcher) = if let Some(pattern) = s3_properties.match_pattern.as_ref() { + let prefix = get_prefix(pattern); + let matcher = glob::Pattern::new(pattern) + .with_context(|| format!("Invalid match_pattern: {}", pattern))?; + (Some(prefix), Some(matcher)) + } else { + (None, None) + }; + let op: Operator = Operator::new(builder)? .layer(LoggingLayer::default()) .layer(RetryLayer::default()) .finish(); + Ok(Self { op, engine_type: EngineType::S3, + prefix, + matcher, marker: PhantomData, }) } diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index c866ed6c3c223..1e6cecb310125 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -39,6 +39,7 @@ pub mod test_source; pub use manager::{SourceColumnDesc, SourceColumnType}; pub use mock_external_table::MockExternalTableReader; +pub use crate::source::filesystem::opendal_source::{GCS_CONNECTOR, OPENDAL_S3_CONNECTOR}; pub use crate::source::filesystem::{S3_CONNECTOR, S3_V2_CONNECTOR}; pub use crate::source::nexmark::NEXMARK_CONNECTOR; pub use crate::source::pulsar::PULSAR_CONNECTOR; diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 8bca367351641..c3d7efcb3dc1a 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -37,8 +37,8 @@ use risingwave_connector::source::datagen::DATAGEN_CONNECTOR; use risingwave_connector::source::nexmark::source::{get_event_data_types_with_names, EventType}; use risingwave_connector::source::test_source::TEST_CONNECTOR; use risingwave_connector::source::{ - GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, NATS_CONNECTOR, NEXMARK_CONNECTOR, - PULSAR_CONNECTOR, S3_CONNECTOR, S3_V2_CONNECTOR, + GCS_CONNECTOR, GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, NATS_CONNECTOR, + NEXMARK_CONNECTOR, OPENDAL_S3_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, S3_V2_CONNECTOR, }; use risingwave_pb::catalog::{ PbSchemaRegistryNameStrategy, PbSource, StreamSourceInfo, WatermarkDesc, @@ -885,6 +885,12 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock hashmap!( Format::Plain => vec![Encode::Csv, Encode::Json], ), + OPENDAL_S3_CONNECTOR => hashmap!( + Format::Plain => vec![Encode::Csv, Encode::Json], + ), + GCS_CONNECTOR => hashmap!( + Format::Plain => vec![Encode::Json], + ), MYSQL_CDC_CONNECTOR => hashmap!( Format::Plain => vec![Encode::Bytes], Format::Debezium => vec![Encode::Json], diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index fd4e6e7197d93..4c02b11b9bfdf 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -36,7 +36,7 @@ use risingwave_connector::source::{ ConnectorState, FsFilterCtrlCtx, SourceColumnDesc, SourceContext, SplitReader, }; use tokio::time; -use tokio::time::{Duration, MissedTickBehavior}; +use tokio::time::Duration; #[derive(Clone, Debug)] pub struct ConnectorSource { @@ -93,7 +93,6 @@ impl ConnectorSource { pub fn get_opendal_source_list(&self) -> Result> { let config = self.config.clone(); - match config { ConnectorProperties::Gcs(prop) => { let lister: OpendalEnumerator = @@ -107,9 +106,10 @@ impl ConnectorSource { lister, )) } - ConnectorProperties::OpenDalS3(prop) => { + ConnectorProperties::S3(prop) => { + // todo(wcy-fdu): use new s3 prop let lister: OpendalEnumerator = - OpendalEnumerator::new_s3_source(prop.s3_properties)?; + OpendalEnumerator::new_s3_source(*prop)?; Ok(build_opendal_fs_list_stream( FsListCtrlContext { interval: Duration::from_secs(60), @@ -157,7 +157,7 @@ impl ConnectorSource { dispatch_source_prop!(config, prop, { let readers = if support_multiple_splits { - tracing::debug!( + tracing::info!( "spawning connector split reader for multiple splits {:?}", splits ); @@ -171,7 +171,7 @@ impl ConnectorSource { let to_reader_splits = splits.into_iter().map(|split| vec![split]); try_join_all(to_reader_splits.into_iter().map(|splits| { - tracing::debug!(?splits, ?prop, "spawning connector split reader"); + tracing::info!(?splits, ?prop, "spawning connector split reader"); let props = prop.clone(); let data_gen_columns = data_gen_columns.clone(); let parser_config = parser_config.clone(); @@ -199,22 +199,14 @@ impl ConnectorSource { #[try_stream(boxed, ok = FsPageItem, error = RwError)] async fn build_opendal_fs_list_stream( - ctrl_ctx: FsListCtrlContext, + _ctrl_ctx: FsListCtrlContext, lister: OpendalEnumerator, ) { - let mut interval = time::interval(ctrl_ctx.interval); - interval.set_missed_tick_behavior(MissedTickBehavior::Skip); - - let mut object_metadata_iter = lister.list("123").await?; - loop { - match object_metadata_iter.next().await { - Some(list_res) => { - yield list_res.unwrap(); - } - None => { - break; - } - } - interval.tick().await; + // let mut interval = time::interval(ctrl_ctx.interval); + // interval.set_missed_tick_behavior(MissedTickBehavior::Skip); + + let mut object_metadata_iter = lister.list().await?; + while let Some(list_res) = object_metadata_iter.next().await { + yield list_res.unwrap(); } } From 98de2c5497efd65442853a85fffd27b0283de31c Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 21 Nov 2023 17:20:24 +0800 Subject: [PATCH 16/47] remove origin s3_v2, and use opendal as new s3_v2 --- src/connector/src/source/base.rs | 35 +++--- src/connector/src/source/filesystem/mod.rs | 1 - .../filesystem/opendal_source/gcs_source.rs | 24 ++-- .../source/filesystem/opendal_source/mod.rs | 2 + .../opendal_source/opendal_enumerator.rs | 14 ++- .../opendal_source/opendal_reader.rs | 3 - src/connector/src/source/mod.rs | 2 +- src/frontend/src/handler/create_source.rs | 5 +- src/source/src/connector_source.rs | 40 +++++-- src/source/src/source_desc.rs | 1 - .../src/executor/source/fetch_executor.rs | 103 ++++++++++++++---- 11 files changed, 163 insertions(+), 67 deletions(-) diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index ec71d623c220d..df25fab5eefec 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -42,7 +42,8 @@ use super::nexmark::source::message::NexmarkMeta; use super::{GCS_CONNECTOR, OPENDAL_S3_CONNECTOR}; use crate::parser::ParserConfig; pub(crate) use crate::source::common::CommonSplitReader; -use crate::source::filesystem::{FsPageItem, S3Properties}; +use crate::source::filesystem::opendal_source::OpendalS3Properties; +use crate::source::filesystem::{FsPageItem, GcsProperties, S3Properties}; use crate::source::monitor::EnumeratorMetrics; use crate::source::S3_CONNECTOR; use crate::{ @@ -74,7 +75,7 @@ impl TryFromHashmap for P { } } -pub async fn create_split_reader( +pub async fn create_split_reader( prop: P, splits: Vec, parser_config: ParserConfig, @@ -388,14 +389,6 @@ impl ConnectorProperties { pub fn rewrite_upstream_source_key_hash_map(props: &mut HashMap) { let connector = props.remove(UPSTREAM_SOURCE_KEY).unwrap(); match connector.as_str() { - // S3_V2_CONNECTOR => { - // tracing::info!( - // "using new fs source, rewrite connector from '{}' to '{}'", - // S3_V2_CONNECTOR, - // S3_CONNECTOR - // ); - // props.insert(UPSTREAM_SOURCE_KEY.to_string(), S3_CONNECTOR.to_string()); - // } OPENDAL_S3_CONNECTOR => { tracing::info!( "using new fs source, rewrite connector from '{}' to '{}'", @@ -425,12 +418,26 @@ impl ConnectorProperties { impl ConnectorProperties { pub fn extract(mut props: HashMap) -> Result { if Self::is_new_fs_connector_hash_map(&props) { - _ = props + let connector = props .remove(UPSTREAM_SOURCE_KEY) .ok_or_else(|| anyhow!("Must specify 'connector' in WITH clause"))?; - return Ok(ConnectorProperties::S3(Box::new( - S3Properties::try_from_hashmap(props)?, - ))); + match connector.as_str() { + "s3_v2" => { + return Ok(ConnectorProperties::OpenDalS3(Box::new( + OpendalS3Properties { + s3_properties: S3Properties::try_from_hashmap(props)?, + }, + ))); + } + "gcs" => { + return Ok(ConnectorProperties::Gcs(Box::new( + GcsProperties::try_from_hashmap(props)?, + ))); + } + _ => { + unreachable!() + } + } } let connector = props diff --git a/src/connector/src/source/filesystem/mod.rs b/src/connector/src/source/filesystem/mod.rs index befa7fa715cb7..e7c2a4fa72288 100644 --- a/src/connector/src/source/filesystem/mod.rs +++ b/src/connector/src/source/filesystem/mod.rs @@ -20,4 +20,3 @@ pub use file_common::{FsPage, FsPageItem, FsSplit, OpendalFsSplit}; pub mod opendal_source; mod s3; pub mod s3_v2; -pub const S3_V2_CONNECTOR: &str = "s3_v2"; diff --git a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs index 4903fa1a215d6..5dbc4426a2d87 100644 --- a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs @@ -14,12 +14,13 @@ use std::marker::PhantomData; +use anyhow::Context; use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::Gcs; use opendal::Operator; use super::opendal_enumerator::{EngineType, OpendalEnumerator}; -use super::{GcsProperties, OpenDalProperties}; +use super::{get_prefix, GcsProperties, OpenDalProperties}; impl OpendalEnumerator where @@ -42,20 +43,19 @@ where .layer(RetryLayer::default()) .finish(); - // todo(wcy-fdu): add (prefix, matcher) for gcs_properties - // let (prefix, matcher) = if let Some(pattern) = gcs_properties.match_pattern.as_ref() { - // let prefix = get_prefix(pattern); - // let matcher = glob::Pattern::new(pattern) - // .with_context(|| format!("Invalid match_pattern: {}", pattern))?; - // (Some(prefix), Some(matcher)) - // } else { - // (None, None) - // }; + let (prefix, matcher) = if let Some(pattern) = gcs_properties.match_pattern.as_ref() { + let prefix = get_prefix(pattern); + let matcher = glob::Pattern::new(pattern) + .with_context(|| format!("Invalid match_pattern: {}", pattern))?; + (Some(prefix), Some(matcher)) + } else { + (None, None) + }; Ok(Self { op, engine_type: EngineType::Gcs, - prefix: None, - matcher: None, + prefix, + matcher, marker: PhantomData, }) } diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index f287025707e57..7e11ab15b748f 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -33,6 +33,8 @@ pub const OPENDAL_S3_CONNECTOR: &str = "s3_v2"; pub struct GcsProperties { #[serde(rename = "gcs.bucket_name")] pub bucket_name: String, + #[serde(rename = "match_pattern", default)] + pub match_pattern: Option, } impl SourceProperties for GcsProperties { diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index 38c63d5792eac..1d259fd90a6b4 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -60,7 +60,7 @@ where } async fn list_splits(&mut self) -> anyhow::Result>> { - todo!() + Ok(vec![]) } } @@ -73,21 +73,23 @@ where Some(prefix) => prefix, None => "", }; + // Currently, we need to do full list and then filter the prefix and matcher, // After OpenDAL implementing the list prefix, we can use the user-specified prefix. // https://github.com/apache/incubator-opendal/issues/3247 + let object_lister = self .op .lister_with("/") .delimiter("") .metakey(Metakey::ContentLength | Metakey::ContentType) .await?; - let stream = stream::unfold(object_lister, |mut object_lister| async move { match object_lister.next().await { Some(Ok(object)) => { // todo: manual filtering prefix let name = object.path().to_string(); + let om = object.metadata(); let t = match om.last_modified() { @@ -119,5 +121,13 @@ where Ok(stream.boxed()) } + + pub fn get_prefix(&self) -> &Option { + &self.prefix + } + + pub fn get_matcher(&self) -> &Option { + &self.matcher + } } pub type ObjectMetadataIter = BoxStream<'static, anyhow::Result>; 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 4559cbd3278e2..38671a344fe1d 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -59,13 +59,11 @@ where _columns: Option>, ) -> Result { let connector = Self::Properties::new_enumerator(properties)?; - // let connector = OpendalConnector::new_gcs_source(properties)?; let opendal_reader = OpendalReader { connector, splits, parser_config, source_ctx, - // marker: PhantomData, }; Ok(opendal_reader) } @@ -176,7 +174,6 @@ where } pub async fn get_object(op: Operator, object_name: &str, start: usize) -> Result { - println!("读{:?}", object_name); let mut reader = op.reader_with(object_name).range(start as u64..).await?; let mut buffer = Vec::new(); reader.read_to_end(&mut buffer).await?; diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index 1e6cecb310125..95da14c943a68 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -40,6 +40,6 @@ pub use manager::{SourceColumnDesc, SourceColumnType}; pub use mock_external_table::MockExternalTableReader; pub use crate::source::filesystem::opendal_source::{GCS_CONNECTOR, OPENDAL_S3_CONNECTOR}; -pub use crate::source::filesystem::{S3_CONNECTOR, S3_V2_CONNECTOR}; +pub use crate::source::filesystem::S3_CONNECTOR; pub use crate::source::nexmark::NEXMARK_CONNECTOR; pub use crate::source::pulsar::PULSAR_CONNECTOR; diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index c3d7efcb3dc1a..f4c8a9571b579 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -38,7 +38,7 @@ use risingwave_connector::source::nexmark::source::{get_event_data_types_with_na use risingwave_connector::source::test_source::TEST_CONNECTOR; use risingwave_connector::source::{ GCS_CONNECTOR, GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, NATS_CONNECTOR, - NEXMARK_CONNECTOR, OPENDAL_S3_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, S3_V2_CONNECTOR, + NEXMARK_CONNECTOR, OPENDAL_S3_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, }; use risingwave_pb::catalog::{ PbSchemaRegistryNameStrategy, PbSource, StreamSourceInfo, WatermarkDesc, @@ -882,9 +882,6 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock hashmap!( Format::Plain => vec![Encode::Csv, Encode::Json], ), - S3_V2_CONNECTOR => hashmap!( - Format::Plain => vec![Encode::Csv, Encode::Json], - ), OPENDAL_S3_CONNECTOR => hashmap!( Format::Plain => vec![Encode::Csv, Encode::Json], ), diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 4c02b11b9bfdf..12c563f416de5 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -105,11 +105,10 @@ impl ConnectorSource { }, lister, )) - } - ConnectorProperties::S3(prop) => { - // todo(wcy-fdu): use new s3 prop + } + ConnectorProperties::OpenDalS3(prop) => { let lister: OpendalEnumerator = - OpendalEnumerator::new_s3_source(*prop)?; + OpendalEnumerator::new_s3_source(prop.s3_properties)?; Ok(build_opendal_fs_list_stream( FsListCtrlContext { interval: Duration::from_secs(60), @@ -154,14 +153,12 @@ impl ConnectorSource { }; let support_multiple_splits = config.support_multiple_splits(); - dispatch_source_prop!(config, prop, { let readers = if support_multiple_splits { tracing::info!( "spawning connector split reader for multiple splits {:?}", splits ); - let reader = create_split_reader(*prop, splits, parser_config, source_ctx, data_gen_columns) .await?; @@ -205,8 +202,37 @@ async fn build_opendal_fs_list_stream( // let mut interval = time::interval(ctrl_ctx.interval); // interval.set_missed_tick_behavior(MissedTickBehavior::Skip); + // let prefix = match lister.get_prefix().as_ref() { + // Some(prefix) => prefix, + // None => { + // let empty_prefix = "".to_string(); + // &empty_prefix + // } + // }; + let prefix = lister + .get_prefix() + .as_ref() + .map(|s| s.as_str()) + .unwrap_or(""); + + let matcher = lister.get_matcher(); let mut object_metadata_iter = lister.list().await?; + while let Some(list_res) = object_metadata_iter.next().await { - yield list_res.unwrap(); + match list_res { + Ok(res) => { + if res.name.starts_with(prefix) + && matcher + .as_ref() + .map(|m| m.matches(&res.name)) + .unwrap_or(true) + { + yield res + } else { + continue; + } + } + Err(_) => break, + } } } diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index e6646c03282a0..33e920bb8590f 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -109,7 +109,6 @@ impl SourceDescBuilder { // new fs source requires `connector='s3_v2' but we simply reuse S3 connector` ConnectorProperties::rewrite_upstream_source_key_hash_map(&mut self.properties); } - let source = ConnectorSource::new( self.properties, columns.clone(), diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 3e7ea84bcfcce..32fd1319352ea 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -24,7 +24,8 @@ use risingwave_common::catalog::{ColumnId, Schema, TableId}; use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{ScalarRef, ScalarRefImpl}; -use risingwave_connector::source::filesystem::FsSplit; +use risingwave_connector::source::filesystem::opendal_source::OpendalS3Properties; +use risingwave_connector::source::filesystem::{GcsProperties, OpendalFsSplit}; use risingwave_connector::source::{ BoxSourceWithStateStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData, StreamChunkWithState, @@ -110,13 +111,23 @@ impl FsFetchExecutor { ) .await?; pin_mut!(table_iter); - + let properties = source_desc.source.config.clone(); while let Some(item) = table_iter.next().await { let row = item?; let split = match row.datum_at(1) { - Some(ScalarRefImpl::Jsonb(jsonb_ref)) => { - SplitImpl::from(FsSplit::restore_from_json(jsonb_ref.to_owned_scalar())?) - } + Some(ScalarRefImpl::Jsonb(jsonb_ref)) => match properties { + risingwave_connector::source::ConnectorProperties::Gcs(_) => { + let split: OpendalFsSplit = + OpendalFsSplit::restore_from_json(jsonb_ref.to_owned_scalar())?; + SplitImpl::from(split) + } + risingwave_connector::source::ConnectorProperties::OpenDalS3(_) => { + let split: OpendalFsSplit = + OpendalFsSplit::restore_from_json(jsonb_ref.to_owned_scalar())?; + SplitImpl::from(split) + } + _ => unreachable!(), + }, _ => unreachable!(), }; batch.push(split); @@ -126,7 +137,6 @@ impl FsFetchExecutor { } } } - if batch.is_empty() { stream.replace_data_stream(stream::pending().boxed()); } else { @@ -265,16 +275,36 @@ impl FsFetchExecutor { // Receiving file assignments from upstream list executor, // store into state table and try building a new reader. Message::Chunk(chunk) => { - let file_assignment = chunk - .data_chunk() - .rows() - .map(|row| { - let filename = row.datum_at(0).unwrap().into_utf8(); - let size = row.datum_at(2).unwrap().into_int64(); - FsSplit::new(filename.to_owned(), 0, size as usize) - }) - .collect(); - state_store_handler.take_snapshot(file_assignment).await?; + let properties = source_desc.source.config.clone(); + match properties{ + risingwave_connector::source::ConnectorProperties::Gcs(_) => { + let file_assignment = chunk + .data_chunk() + .rows() + .map(|row| { + let filename = row.datum_at(0).unwrap().into_utf8(); + let size = row.datum_at(2).unwrap().into_int64(); + let split: OpendalFsSplit= OpendalFsSplit::new(filename.to_owned(), 0, size as usize); + split + }) + .collect(); + state_store_handler.take_snapshot(file_assignment).await?; + }, + risingwave_connector::source::ConnectorProperties::OpenDalS3(_) => { + let file_assignment = chunk + .data_chunk() + .rows() + .map(|row| { + let filename = row.datum_at(0).unwrap().into_utf8(); + let size = row.datum_at(2).unwrap().into_int64(); + let split: OpendalFsSplit= OpendalFsSplit::new(filename.to_owned(), 0, size as usize); + split + }) + .collect(); + state_store_handler.take_snapshot(file_assignment).await?; + }, + _ => unreachable!() + } } _ => unreachable!(), } @@ -291,12 +321,17 @@ impl FsFetchExecutor { .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)) => { - FsSplit::restore_from_json(jsonb_ref.to_owned_scalar())? - } - _ => unreachable!(), - }; + let properties = source_desc.source.config.clone(); + if let risingwave_connector::source::ConnectorProperties::OpenDalS3(_)=properties{ + let fs_split: OpendalFsSplit = + 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; @@ -306,6 +341,30 @@ impl FsFetchExecutor { .set(split_id, fs_split.encode_to_json()) .await?; } + } + else if let risingwave_connector::source::ConnectorProperties::Gcs(_)=properties{ + let fs_split: OpendalFsSplit = + 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?; + } + } + else{ + unreachable!() + } } yield Message::Chunk(chunk); From 743226fbd6bb1a125a25390e27085176401cf129 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 23 Nov 2023 16:12:43 +0800 Subject: [PATCH 17/47] add credential --- .../src/source/filesystem/opendal_source/gcs_source.rs | 4 ++-- src/connector/src/source/filesystem/opendal_source/mod.rs | 2 ++ src/source/src/connector_source.rs | 2 +- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs index 5dbc4426a2d87..3fa91d9a541a1 100644 --- a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs @@ -34,8 +34,8 @@ where builder.bucket(&gcs_properties.bucket_name); // if credential env is set, use it. Otherwise, ADC will be used. - let cred = std::env::var("GOOGLE_APPLICATION_CREDENTIALS"); - if let Ok(cred) = cred { + let cred = gcs_properties.credential; + if let Some(cred) = cred { builder.credential(&cred); } let op: Operator = Operator::new(builder)? diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index 7e11ab15b748f..8115884e1caf7 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -33,6 +33,8 @@ pub const OPENDAL_S3_CONNECTOR: &str = "s3_v2"; pub struct GcsProperties { #[serde(rename = "gcs.bucket_name")] pub bucket_name: String, + #[serde(rename = "gcs.credential")] + pub credential: Option, #[serde(rename = "match_pattern", default)] pub match_pattern: Option, } diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 12c563f416de5..f3e1560c55022 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -105,7 +105,7 @@ impl ConnectorSource { }, lister, )) - } + } ConnectorProperties::OpenDalS3(prop) => { let lister: OpendalEnumerator = OpendalEnumerator::new_s3_source(prop.s3_properties)?; From 4c774eb4bcd1cf79d7727aa78b7935b6f9d4f508 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 23 Nov 2023 17:59:39 +0800 Subject: [PATCH 18/47] fix list --- .../filesystem/opendal_source/gcs_source.rs | 4 +-- .../opendal_source/opendal_enumerator.rs | 7 ++-- .../filesystem/opendal_source/s3_source.rs | 4 +-- src/source/src/connector_source.rs | 33 ++----------------- .../src/executor/source/list_executor.rs | 4 ++- 5 files changed, 13 insertions(+), 39 deletions(-) diff --git a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs index 3fa91d9a541a1..c3b49c1eecb0a 100644 --- a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs @@ -26,9 +26,9 @@ impl OpendalEnumerator where C: Send + Clone + PartialEq + 'static + Sync, { - /// create opendal gcs engine. + /// create opendal gcs source. pub fn new_gcs_source(gcs_properties: GcsProperties) -> anyhow::Result { - // Create gcs backend builder. + // Create gcs builder. let mut builder = Gcs::default(); builder.bucket(&gcs_properties.bucket_name); diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index 1d259fd90a6b4..cf9586bfb2a5e 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -82,14 +82,13 @@ where .op .lister_with("/") .delimiter("") - .metakey(Metakey::ContentLength | Metakey::ContentType) + .metakey(Metakey::ContentLength | Metakey::LastModified) .await?; let stream = stream::unfold(object_lister, |mut object_lister| async move { match object_lister.next().await { Some(Ok(object)) => { // todo: manual filtering prefix let name = object.path().to_string(); - let om = object.metadata(); let t = match om.last_modified() { @@ -109,11 +108,11 @@ where Some((Ok(metadata), object_lister)) } Some(Err(err)) => { - tracing::error!("list fail"); + tracing::error!("list object fail, err {}", err); Some((Err(err.into()), object_lister)) } None => { - tracing::info!("list to the end"); + tracing::info!("list object completed."); None } } diff --git a/src/connector/src/source/filesystem/opendal_source/s3_source.rs b/src/connector/src/source/filesystem/opendal_source/s3_source.rs index 17fefb0623fdc..0681bbd0c3448 100644 --- a/src/connector/src/source/filesystem/opendal_source/s3_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/s3_source.rs @@ -28,9 +28,9 @@ impl OpendalEnumerator where C: Sized + Send + Clone + PartialEq + 'static + Sync, { - /// create opendal s3 engine. + /// create opendal s3 source. pub fn new_s3_source(s3_properties: S3Properties) -> anyhow::Result { - // Create s3 backend builder. + // Create s3 builder. let mut builder = S3::default(); builder.bucket(&s3_properties.bucket_name); builder.region(&s3_properties.region_name); diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index f3e1560c55022..d0003c32b8921 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -97,26 +97,12 @@ impl ConnectorSource { ConnectorProperties::Gcs(prop) => { let lister: OpendalEnumerator = OpendalEnumerator::new_gcs_source(*prop)?; - Ok(build_opendal_fs_list_stream( - FsListCtrlContext { - interval: Duration::from_secs(60), - last_tick: None, - filter_ctx: FsFilterCtrlCtx, - }, - lister, - )) + Ok(build_opendal_fs_list_stream(lister)) } ConnectorProperties::OpenDalS3(prop) => { let lister: OpendalEnumerator = OpendalEnumerator::new_s3_source(prop.s3_properties)?; - Ok(build_opendal_fs_list_stream( - FsListCtrlContext { - interval: Duration::from_secs(60), - last_tick: None, - filter_ctx: FsFilterCtrlCtx, - }, - lister, - )) + Ok(build_opendal_fs_list_stream(lister)) } other => Err(internal_error(format!("Unsupported source: {:?}", other))), } @@ -195,20 +181,7 @@ impl ConnectorSource { } #[try_stream(boxed, ok = FsPageItem, error = RwError)] -async fn build_opendal_fs_list_stream( - _ctrl_ctx: FsListCtrlContext, - lister: OpendalEnumerator, -) { - // let mut interval = time::interval(ctrl_ctx.interval); - // interval.set_missed_tick_behavior(MissedTickBehavior::Skip); - - // let prefix = match lister.get_prefix().as_ref() { - // Some(prefix) => prefix, - // None => { - // let empty_prefix = "".to_string(); - // &empty_prefix - // } - // }; +async fn build_opendal_fs_list_stream(lister: OpendalEnumerator) { let prefix = lister .get_prefix() .as_ref() diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/list_executor.rs index 0a142569fbdf2..931fec5c2c0b3 100644 --- a/src/stream/src/executor/source/list_executor.rs +++ b/src/stream/src/executor/source/list_executor.rs @@ -34,6 +34,8 @@ use crate::executor::monitor::StreamingMetrics; use crate::executor::stream_reader::StreamReaderWithPause; use crate::executor::*; +const CHUNK_SIZE: usize = 1024; + #[allow(dead_code)] pub struct FsListExecutor { actor_ctx: ActorContextRef, @@ -103,7 +105,7 @@ impl FsListExecutor { .get_opendal_source_list() .map_err(StreamExecutorError::connector_error)?; let chunked_stream = stream - .chunks(1024) // Group FsPageItems into chunks of size 1024 + .chunks(CHUNK_SIZE) // Group FsPageItems into chunks of size 1024 .map(|chunk| { let rows = chunk .into_iter() From a303fc3134333f5c99628d71fff13d66971c0bad Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 24 Nov 2023 14:13:42 +0800 Subject: [PATCH 19/47] use stream read --- e2e_test/s3/gcs_source.py | 136 ++++++++++++++++++ .../filesystem/opendal_source/gcs_source.rs | 7 +- .../source/filesystem/opendal_source/mod.rs | 6 +- .../opendal_source/opendal_enumerator.rs | 15 +- .../opendal_source/opendal_reader.rs | 35 ++--- .../filesystem/opendal_source/s3_source.rs | 7 +- src/source/src/connector_source.rs | 6 +- .../src/executor/source/list_executor.rs | 2 +- 8 files changed, 165 insertions(+), 49 deletions(-) create mode 100644 e2e_test/s3/gcs_source.py diff --git a/e2e_test/s3/gcs_source.py b/e2e_test/s3/gcs_source.py new file mode 100644 index 0000000000000..b862568de78a0 --- /dev/null +++ b/e2e_test/s3/gcs_source.py @@ -0,0 +1,136 @@ +import os +import sys +import csv +import json +import random +import psycopg2 + +from time import sleep +from io import StringIO +from minio import Minio +from functools import partial + +def gen_data(file_num, item_num_per_file): + assert item_num_per_file % 2 == 0, \ + f'item_num_per_file should be even to ensure sum(mark) == 0: {item_num_per_file}' + return [ + [{ + 'id': file_id * item_num_per_file + item_id, + 'name': f'{file_id}_{item_id}', + 'sex': item_id % 2, + 'mark': (-1) ** (item_id % 2), + } for item_id in range(item_num_per_file)] + for file_id in range(file_num) + ] + +def format_json(data): + return [ + '\n'.join([json.dumps(item) for item in file]) + for file in data + ] + + +def do_test(config, file_num, item_num_per_file, prefix, fmt): + conn = psycopg2.connect( + host="localhost", + port="4566", + user="root", + database="dev" + ) + + # Open a cursor to execute SQL statements + cur = conn.cursor() + + def _table(): + return f'gcs_test_{fmt}' + + def _encode(): + return 'JSON' + + # Execute a SELECT statement + cur.execute(f'''CREATE TABLE {_table()}( + id int, + name TEXT, + sex int, + mark int, + ) WITH ( + connector = 'gcs', + match_pattern = '{prefix}*.{fmt}', + gcs.bucket_name = '{config['GCS_BUCKET']}', + gcs.credentials = '{config['GOOGLE_APPLICATION_CREDENTIALS']}', + ) FORMAT PLAIN ENCODE {_encode()};''') + + total_rows = file_num * item_num_per_file + MAX_RETRIES = 40 + for retry_no in range(MAX_RETRIES): + cur.execute(f'select count(*) from {_table()}') + result = cur.fetchone() + if result[0] == total_rows: + break + print(f"[retry {retry_no}] Now got {result[0]} rows in table, {total_rows} expected, wait 30s") + sleep(30) + + stmt = f'select count(*), sum(id), sum(sex), sum(mark) from {_table()}' + print(f'Execute {stmt}') + cur.execute(stmt) + result = cur.fetchone() + + print('Got:', result) + + def _assert_eq(field, got, expect): + assert got == expect, f'{field} assertion failed: got {got}, expect {expect}.' + + _assert_eq('count(*)', result[0], total_rows) + _assert_eq('sum(id)', result[1], (total_rows - 1) * total_rows / 2) + _assert_eq('sum(sex)', result[2], total_rows / 2) + _assert_eq('sum(mark)', result[3], 0) + + print('Test pass') + + cur.execute(f'drop table {_table()}') + cur.close() + conn.close() + + +if __name__ == "__main__": + FILE_NUM = 4001 + ITEM_NUM_PER_FILE = 2 + data = gen_data(FILE_NUM, ITEM_NUM_PER_FILE) + + fmt = sys.argv[1] + FORMATTER = { + 'json': format_json, + } + assert fmt in FORMATTER, f"Unsupported format: {fmt}" + formatted_files = FORMATTER[fmt](data) + + config = json.loads(os.environ["GCS_SOURCE_TEST_CONF"]) + client = Minio( + config["GCS_ENDPOINT"], + access_key=config["GCS_ACCESS_KEY"], + secret_key=config["GCS_SECRET_KEY"], + secure=True, + ) + run_id = str(random.randint(1000, 9999)) + _local = lambda idx: f'data_{idx}.{fmt}' + _gcs = lambda idx: f"{run_id}_data_{idx}.{fmt}" + + # todo(wcy-fdu): write into gcs + # put gcs files + for idx, file_str in enumerate(formatted_files): + with open(_local(idx), "w") as f: + f.write(file_str) + os.fsync(f.fileno()) + + client.fput_object( + config["GCS_BUCKET"], + _gcs(idx), + _local(idx) + ) + + # do test + do_test(config, FILE_NUM, ITEM_NUM_PER_FILE, run_id, fmt) + + # clean up gcs files + for idx, _ in enumerate(formatted_files): + client.remove_object(config["gcs_BUCKET"], _gcs(idx)) diff --git a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs index c3b49c1eecb0a..7b6e3dea9ce2e 100644 --- a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs @@ -19,10 +19,10 @@ use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::Gcs; use opendal::Operator; -use super::opendal_enumerator::{EngineType, OpendalEnumerator}; -use super::{get_prefix, GcsProperties, OpenDalProperties}; +use super::opendal_enumerator::OpendalEnumerator; +use super::{get_prefix, GcsProperties, OpenDalSourceProperties}; -impl OpendalEnumerator +impl OpendalEnumerator where C: Send + Clone + PartialEq + 'static + Sync, { @@ -53,7 +53,6 @@ where }; Ok(Self { op, - engine_type: EngineType::Gcs, prefix, matcher, marker: PhantomData, diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index 8115884e1caf7..ba34056d2525f 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -64,17 +64,17 @@ impl SourceProperties for OpendalS3Properties { fn init_from_pb_source(&mut self, _source: &risingwave_pb::catalog::PbSource) {} } -pub trait OpenDalProperties: Sized + Send + Clone + PartialEq + 'static + Sync { +pub trait OpenDalSourceProperties: Sized + Send + Clone + PartialEq + 'static + Sync { fn new_enumerator(properties: Self) -> anyhow::Result>; } -impl OpenDalProperties for GcsProperties { +impl OpenDalSourceProperties for GcsProperties { fn new_enumerator(properties: Self) -> anyhow::Result> { OpendalEnumerator::new_gcs_source(properties) } } -impl OpenDalProperties for OpendalS3Properties { +impl OpenDalSourceProperties for OpendalS3Properties { fn new_enumerator(properties: Self) -> anyhow::Result> { OpendalEnumerator::new_s3_source(properties.s3_properties) } diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index cf9586bfb2a5e..112e7f710f8c2 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -21,31 +21,24 @@ use futures::StreamExt; use opendal::{Metakey, Operator}; use risingwave_common::types::Timestamp; -use super::OpenDalProperties; +use super::OpenDalSourceProperties; use crate::source::filesystem::{FsPageItem, OpendalFsSplit}; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; #[derive(Debug, Clone)] -pub struct OpendalEnumerator +pub struct OpendalEnumerator where C: Send + Clone + Sized + PartialEq + 'static + Sync, { pub(crate) op: Operator, - pub(crate) engine_type: EngineType, // prefix is used to reduce the number of objects to be listed pub(crate) prefix: Option, pub(crate) matcher: Option, pub(crate) marker: PhantomData, } -#[derive(Debug, Clone)] -pub enum EngineType { - Gcs, - S3, -} - #[async_trait] -impl SplitEnumerator for OpendalEnumerator +impl SplitEnumerator for OpendalEnumerator where C: Sized + Send + Clone + PartialEq + 'static + Sync, { @@ -64,7 +57,7 @@ where } } -impl OpendalEnumerator +impl OpendalEnumerator where C: Send + Clone + Sized + PartialEq + 'static + Sync, { 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 38671a344fe1d..438191589d268 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -12,20 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::pin::pin; - use anyhow::{Ok, Result}; use async_trait::async_trait; -use aws_smithy_http::byte_stream::ByteStream; use futures::TryStreamExt; use futures_async_stream::try_stream; use opendal::Operator; use risingwave_common::error::RwError; -use tokio::io::{AsyncReadExt, BufReader}; +use tokio::io::BufReader; use tokio_util::io::{ReaderStream, StreamReader}; use super::opendal_enumerator::OpendalEnumerator; -use super::OpenDalProperties; +use super::OpenDalSourceProperties; use crate::parser::{ByteStreamSourceParserImpl, ParserConfig}; use crate::source::filesystem::{nd_streaming, OpendalFsSplit}; use crate::source::{ @@ -36,15 +33,14 @@ use crate::source::{ const MAX_CHANNEL_BUFFER_SIZE: usize = 2048; const STREAM_READER_CAPACITY: usize = 4096; #[derive(Debug, Clone)] -pub struct OpendalReader { +pub struct OpendalReader { connector: OpendalEnumerator, splits: Vec>, parser_config: ParserConfig, source_ctx: SourceContextRef, - // marker: PhantomData, } #[async_trait] -impl SplitReader for OpendalReader +impl SplitReader for OpendalReader where C: Send + Clone + PartialEq + 'static + Sync, { @@ -73,7 +69,7 @@ where } } -impl OpendalReader +impl OpendalReader where C: Send + Clone + Sized + PartialEq + 'static, { @@ -125,15 +121,16 @@ where let object_name = split.name.clone(); - let byte_stream = Self::get_object(op, &object_name, split.offset).await?; + let reader = op + .reader_with(&object_name) + .range(split.offset as u64..) + .await?; let stream_reader = StreamReader::new( - byte_stream.map_err(|e| std::io::Error::new(std::io::ErrorKind::Other, e)), + reader.map_err(|e| std::io::Error::new(std::io::ErrorKind::Other, e)), ); - - let reader = pin!(BufReader::new(stream_reader)); - - let stream = ReaderStream::with_capacity(reader, STREAM_READER_CAPACITY); + let buf_reader = BufReader::new(stream_reader); + let stream = ReaderStream::with_capacity(buf_reader, STREAM_READER_CAPACITY); let mut offset: usize = split.offset; let mut batch_size: usize = 0; @@ -172,12 +169,4 @@ where yield batch; } } - - pub async fn get_object(op: Operator, object_name: &str, start: usize) -> Result { - let mut reader = op.reader_with(object_name).range(start as u64..).await?; - let mut buffer = Vec::new(); - reader.read_to_end(&mut buffer).await?; - let res = ByteStream::from(buffer); - Ok(res) - } } diff --git a/src/connector/src/source/filesystem/opendal_source/s3_source.rs b/src/connector/src/source/filesystem/opendal_source/s3_source.rs index 0681bbd0c3448..0417c52f294ab 100644 --- a/src/connector/src/source/filesystem/opendal_source/s3_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/s3_source.rs @@ -19,12 +19,12 @@ use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::S3; use opendal::Operator; -use super::opendal_enumerator::{EngineType, OpendalEnumerator}; -use super::OpenDalProperties; +use super::opendal_enumerator::OpendalEnumerator; +use super::OpenDalSourceProperties; use crate::source::filesystem::opendal_source::get_prefix; use crate::source::filesystem::S3Properties; -impl OpendalEnumerator +impl OpendalEnumerator where C: Sized + Send + Clone + PartialEq + 'static + Sync, { @@ -64,7 +64,6 @@ where Ok(Self { op, - engine_type: EngineType::S3, prefix, matcher, marker: PhantomData, diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index d0003c32b8921..80581af8783a6 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -28,7 +28,7 @@ use risingwave_connector::dispatch_source_prop; use risingwave_connector::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; use risingwave_connector::source::filesystem::opendal_source::opendal_enumerator::OpendalEnumerator; use risingwave_connector::source::filesystem::opendal_source::{ - OpenDalProperties, OpendalS3Properties, + OpenDalSourceProperties, OpendalS3Properties, }; use risingwave_connector::source::filesystem::{FsPageItem, GcsProperties}; use risingwave_connector::source::{ @@ -91,7 +91,7 @@ impl ConnectorSource { .collect::>>() } - pub fn get_opendal_source_list(&self) -> Result> { + pub fn get_source_list(&self) -> Result> { let config = self.config.clone(); match config { ConnectorProperties::Gcs(prop) => { @@ -181,7 +181,7 @@ impl ConnectorSource { } #[try_stream(boxed, ok = FsPageItem, error = RwError)] -async fn build_opendal_fs_list_stream(lister: OpendalEnumerator) { +async fn build_opendal_fs_list_stream(lister: OpendalEnumerator) { let prefix = lister .get_prefix() .as_ref() diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/list_executor.rs index 931fec5c2c0b3..53a5691790c07 100644 --- a/src/stream/src/executor/source/list_executor.rs +++ b/src/stream/src/executor/source/list_executor.rs @@ -102,7 +102,7 @@ impl FsListExecutor { Box> + Send>, > = source_desc .source - .get_opendal_source_list() + .get_source_list() .map_err(StreamExecutorError::connector_error)?; let chunked_stream = stream .chunks(CHUNK_SIZE) // Group FsPageItems into chunks of size 1024 From 64f5f2499cf448f03bdd0606024cfadc71344c05 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 24 Nov 2023 14:44:24 +0800 Subject: [PATCH 20/47] add python test script --- data_0.json | 2 ++ e2e_test/s3/gcs_source.py | 22 +++++++--------------- 2 files changed, 9 insertions(+), 15 deletions(-) create mode 100644 data_0.json diff --git a/data_0.json b/data_0.json new file mode 100644 index 0000000000000..9604b75cb64b7 --- /dev/null +++ b/data_0.json @@ -0,0 +1,2 @@ +{"id": 0, "name": "0_0", "sex": 0, "mark": 1} +{"id": 1, "name": "0_1", "sex": 1, "mark": -1} \ No newline at end of file diff --git a/e2e_test/s3/gcs_source.py b/e2e_test/s3/gcs_source.py index b862568de78a0..46903f23e8c36 100644 --- a/e2e_test/s3/gcs_source.py +++ b/e2e_test/s3/gcs_source.py @@ -4,10 +4,10 @@ import json import random import psycopg2 +import opendal from time import sleep from io import StringIO -from minio import Minio from functools import partial def gen_data(file_num, item_num_per_file): @@ -105,32 +105,24 @@ def _assert_eq(field, got, expect): formatted_files = FORMATTER[fmt](data) config = json.loads(os.environ["GCS_SOURCE_TEST_CONF"]) - client = Minio( - config["GCS_ENDPOINT"], - access_key=config["GCS_ACCESS_KEY"], - secret_key=config["GCS_SECRET_KEY"], - secure=True, - ) run_id = str(random.randint(1000, 9999)) _local = lambda idx: f'data_{idx}.{fmt}' _gcs = lambda idx: f"{run_id}_data_{idx}.{fmt}" - # todo(wcy-fdu): write into gcs # put gcs files + op = opendal.Operator("gcs", root="/ci/temp/", bucket=config["GCS_BUCKET"], credential=config["GOOGLE_APPLICATION_CREDENTIALS"]) + for idx, file_str in enumerate(formatted_files): with open(_local(idx), "w") as f: f.write(file_str) os.fsync(f.fileno()) - - client.fput_object( - config["GCS_BUCKET"], - _gcs(idx), - _local(idx) - ) + file_bytes = file_str.encode('utf-8') + op.write(_gcs(idx), file_bytes) + # do test do_test(config, FILE_NUM, ITEM_NUM_PER_FILE, run_id, fmt) # clean up gcs files for idx, _ in enumerate(formatted_files): - client.remove_object(config["gcs_BUCKET"], _gcs(idx)) + op.delete(_gcs(idx)) From a0dc1cdff828d980da68d0ed6f4b9f8dea0f781f Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 24 Nov 2023 14:44:44 +0800 Subject: [PATCH 21/47] minor --- data_0.json | 2 -- 1 file changed, 2 deletions(-) delete mode 100644 data_0.json diff --git a/data_0.json b/data_0.json deleted file mode 100644 index 9604b75cb64b7..0000000000000 --- a/data_0.json +++ /dev/null @@ -1,2 +0,0 @@ -{"id": 0, "name": "0_0", "sex": 0, "mark": 1} -{"id": 1, "name": "0_1", "sex": 1, "mark": -1} \ No newline at end of file From f37a0af3f16dba08be1e3999a673a98823d18dfb Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 24 Nov 2023 15:17:47 +0800 Subject: [PATCH 22/47] ready for review --- src/connector/src/source/base.rs | 29 ------------ .../src/source/filesystem/file_common.rs | 6 --- .../filesystem/opendal_source/gcs_source.rs | 3 +- .../source/filesystem/opendal_source/mod.rs | 34 +------------- .../opendal_source/opendal_enumerator.rs | 17 +++---- .../filesystem/opendal_source/s3_source.rs | 4 +- src/connector/src/source/filesystem/s3/mod.rs | 2 +- src/source/src/connector_source.rs | 10 ++-- src/source/src/source_desc.rs | 12 +---- .../src/executor/source/list_executor.rs | 47 +++++++++---------- 10 files changed, 45 insertions(+), 119 deletions(-) diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index df25fab5eefec..632dd98c00b60 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -45,7 +45,6 @@ pub(crate) use crate::source::common::CommonSplitReader; use crate::source::filesystem::opendal_source::OpendalS3Properties; use crate::source::filesystem::{FsPageItem, GcsProperties, S3Properties}; use crate::source::monitor::EnumeratorMetrics; -use crate::source::S3_CONNECTOR; use crate::{ dispatch_source_prop, dispatch_split_impl, for_all_sources, impl_connector_properties, impl_split, match_source_name_str, @@ -385,34 +384,6 @@ impl ConnectorProperties { .map(|s| s.eq_ignore_ascii_case(GCS_CONNECTOR)) .unwrap_or(false) } - - pub fn rewrite_upstream_source_key_hash_map(props: &mut HashMap) { - let connector = props.remove(UPSTREAM_SOURCE_KEY).unwrap(); - match connector.as_str() { - OPENDAL_S3_CONNECTOR => { - tracing::info!( - "using new fs source, rewrite connector from '{}' to '{}'", - OPENDAL_S3_CONNECTOR, - S3_CONNECTOR - ); - props.insert( - UPSTREAM_SOURCE_KEY.to_string(), - OPENDAL_S3_CONNECTOR.to_string(), - ); - } - GCS_CONNECTOR => { - tracing::info!( - "using new fs source, rewrite connector from '{}' to '{}'", - GCS_CONNECTOR, - S3_CONNECTOR - ); - props.insert(UPSTREAM_SOURCE_KEY.to_string(), GCS_CONNECTOR.to_string()); - } - _ => { - props.insert(UPSTREAM_SOURCE_KEY.to_string(), connector); - } - } - } } impl ConnectorProperties { diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index 4b370ef79c8ca..0deec9da23098 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -98,12 +98,6 @@ where } } -#[derive(Clone, Copy, Debug, PartialEq)] -pub enum OpendalEnumeratorType { - OpenDalS3, - Gcs, -} - impl SplitMetaData for OpendalFsSplit where C: Sized + Send + Clone + 'static, diff --git a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs index 7b6e3dea9ce2e..5a58e05c406c0 100644 --- a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs @@ -20,7 +20,8 @@ use opendal::services::Gcs; use opendal::Operator; use super::opendal_enumerator::OpendalEnumerator; -use super::{get_prefix, GcsProperties, OpenDalSourceProperties}; +use super::{GcsProperties, OpenDalSourceProperties}; +use crate::source::filesystem::s3::enumerator::get_prefix; impl OpendalEnumerator where diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index ba34056d2525f..5c3cae8eff039 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -27,6 +27,7 @@ use super::{OpendalFsSplit, S3Properties}; use crate::source::SourceProperties; pub const GCS_CONNECTOR: &str = "gcs"; +// The new s3_v2 will use opendal. pub const OPENDAL_S3_CONNECTOR: &str = "s3_v2"; #[derive(Clone, Debug, Deserialize, PartialEq)] @@ -79,36 +80,3 @@ impl OpenDalSourceProperties for OpendalS3Properties { OpendalEnumerator::new_s3_source(properties.s3_properties) } } - -/// Get the prefix from a glob -pub fn get_prefix(glob: &str) -> String { - let mut escaped = false; - let mut escaped_filter = false; - glob.chars() - .take_while(|c| match (c, &escaped) { - ('*', false) => false, - ('[', false) => false, - ('{', false) => false, - ('\\', false) => { - escaped = true; - true - } - (_, false) => true, - (_, true) => { - escaped = false; - true - } - }) - .filter(|c| match (c, &escaped_filter) { - (_, true) => { - escaped_filter = false; - true - } - ('\\', false) => { - escaped_filter = true; - false - } - (_, _) => true, - }) - .collect() -} diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index 112e7f710f8c2..0c44c77dcdb15 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -62,14 +62,15 @@ where C: Send + Clone + Sized + PartialEq + 'static + Sync, { pub async fn list(&self) -> anyhow::Result { - let _prefix = match &self.prefix { - Some(prefix) => prefix, - None => "", - }; - // Currently, we need to do full list and then filter the prefix and matcher, // After OpenDAL implementing the list prefix, we can use the user-specified prefix. // https://github.com/apache/incubator-opendal/issues/3247 + // todo(wcy-fdu): manual filtering prefix + + // let _prefix = match &self.prefix { + // Some(prefix) => prefix, + // None => "", + // }; let object_lister = self .op @@ -80,7 +81,6 @@ where let stream = stream::unfold(object_lister, |mut object_lister| async move { match object_lister.next().await { Some(Ok(object)) => { - // todo: manual filtering prefix let name = object.path().to_string(); let om = object.metadata(); @@ -100,10 +100,7 @@ where }; Some((Ok(metadata), object_lister)) } - Some(Err(err)) => { - tracing::error!("list object fail, err {}", err); - Some((Err(err.into()), object_lister)) - } + Some(Err(err)) => Some((Err(err.into()), object_lister)), None => { tracing::info!("list object completed."); None diff --git a/src/connector/src/source/filesystem/opendal_source/s3_source.rs b/src/connector/src/source/filesystem/opendal_source/s3_source.rs index 0417c52f294ab..d6978eb681ea6 100644 --- a/src/connector/src/source/filesystem/opendal_source/s3_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/s3_source.rs @@ -21,8 +21,8 @@ use opendal::Operator; use super::opendal_enumerator::OpendalEnumerator; use super::OpenDalSourceProperties; -use crate::source::filesystem::opendal_source::get_prefix; -use crate::source::filesystem::S3Properties; +// use crate::source::filesystem::opendal_source::get_prefix; +use crate::source::filesystem::{s3::enumerator::get_prefix, S3Properties}; impl OpendalEnumerator where diff --git a/src/connector/src/source/filesystem/s3/mod.rs b/src/connector/src/source/filesystem/s3/mod.rs index 9dc1a7a4ee6da..cc5d2695918b0 100644 --- a/src/connector/src/source/filesystem/s3/mod.rs +++ b/src/connector/src/source/filesystem/s3/mod.rs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. -mod enumerator; +pub mod enumerator; pub use enumerator::S3SplitEnumerator; mod source; diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 80581af8783a6..9741ac5eb3cc9 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -141,7 +141,7 @@ impl ConnectorSource { let support_multiple_splits = config.support_multiple_splits(); dispatch_source_prop!(config, prop, { let readers = if support_multiple_splits { - tracing::info!( + tracing::debug!( "spawning connector split reader for multiple splits {:?}", splits ); @@ -154,7 +154,7 @@ impl ConnectorSource { let to_reader_splits = splits.into_iter().map(|split| vec![split]); try_join_all(to_reader_splits.into_iter().map(|splits| { - tracing::info!(?splits, ?prop, "spawning connector split reader"); + tracing::debug!(?splits, ?prop, "spawning connector split reader"); let props = prop.clone(); let data_gen_columns = data_gen_columns.clone(); let parser_config = parser_config.clone(); @@ -202,10 +202,14 @@ async fn build_opendal_fs_list_stream(lister: Openda { yield res } else { + // Currrntly due to the lack of prefix list, we just skip the unmatched files. continue; } } - Err(_) => break, + Err(err) => { + tracing::error!("list object fail, err {}", err); + break; + } } } } diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index 33e920bb8590f..067666c4d4597 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -20,7 +20,7 @@ use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; use risingwave_connector::parser::{EncodingProperties, ProtocolProperties, SpecificParserConfig}; use risingwave_connector::source::monitor::SourceMetrics; -use risingwave_connector::source::{ConnectorProperties, SourceColumnDesc, SourceColumnType}; +use risingwave_connector::source::{SourceColumnDesc, SourceColumnType}; use risingwave_connector::ConnectorParams; use risingwave_pb::catalog::PbStreamSourceInfo; use risingwave_pb::plan_common::PbColumnCatalog; @@ -36,8 +36,6 @@ pub struct SourceDesc { pub source: ConnectorSource, pub columns: Vec, pub metrics: Arc, - - pub is_new_fs_source: bool, } /// `FsSourceDesc` describes a stream source. @@ -99,16 +97,11 @@ impl SourceDescBuilder { columns } - pub fn build(mut self) -> Result { + pub fn build(self) -> Result { let columns = self.column_catalogs_to_source_column_descs(); let psrser_config = SpecificParserConfig::new(&self.source_info, &self.properties)?; - let is_new_fs_source = ConnectorProperties::is_new_fs_connector_hash_map(&self.properties); - if is_new_fs_source { - // new fs source requires `connector='s3_v2' but we simply reuse S3 connector` - ConnectorProperties::rewrite_upstream_source_key_hash_map(&mut self.properties); - } let source = ConnectorSource::new( self.properties, columns.clone(), @@ -120,7 +113,6 @@ impl SourceDescBuilder { source, columns, metrics: self.metrics, - is_new_fs_source, }) } diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/list_executor.rs index 53a5691790c07..5be01da077cab 100644 --- a/src/stream/src/executor/source/list_executor.rs +++ b/src/stream/src/executor/source/list_executor.rs @@ -104,30 +104,29 @@ impl FsListExecutor { .source .get_source_list() .map_err(StreamExecutorError::connector_error)?; - let chunked_stream = stream - .chunks(CHUNK_SIZE) // Group FsPageItems into chunks of size 1024 - .map(|chunk| { - let rows = chunk - .into_iter() - .map(|item| { - // Implement the conversion of FsPageItem to row here - let page_item = item.unwrap(); - ( - Op::Insert, - OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(page_item.name.into_boxed_str())), - Some(ScalarImpl::Timestamp(page_item.timestamp)), - Some(ScalarImpl::Int64(page_item.size)), - ]), - ) - }) - .collect::>(); - - Ok(StreamChunk::from_rows( - &rows, - &[DataType::Varchar, DataType::Timestamp, DataType::Int64], - )) - }); + + // Group FsPageItem stream into chunks of size 1024. + let chunked_stream = stream.chunks(CHUNK_SIZE).map(|chunk| { + let rows = chunk + .into_iter() + .map(|item| { + let page_item = item.unwrap(); + ( + Op::Insert, + OwnedRow::new(vec![ + Some(ScalarImpl::Utf8(page_item.name.into_boxed_str())), + Some(ScalarImpl::Timestamp(page_item.timestamp)), + Some(ScalarImpl::Int64(page_item.size)), + ]), + ) + }) + .collect::>(); + + Ok(StreamChunk::from_rows( + &rows, + &[DataType::Varchar, DataType::Timestamp, DataType::Int64], + )) + }); Ok(chunked_stream.boxed()) } From 4ce0a99edfd4c41cfedf9593a95108c280b4e05d Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 24 Nov 2023 15:27:50 +0800 Subject: [PATCH 23/47] format python file --- e2e_test/s3/gcs_source.py | 1 - 1 file changed, 1 deletion(-) diff --git a/e2e_test/s3/gcs_source.py b/e2e_test/s3/gcs_source.py index 46903f23e8c36..6db9da57b1933 100644 --- a/e2e_test/s3/gcs_source.py +++ b/e2e_test/s3/gcs_source.py @@ -118,7 +118,6 @@ def _assert_eq(field, got, expect): os.fsync(f.fileno()) file_bytes = file_str.encode('utf-8') op.write(_gcs(idx), file_bytes) - # do test do_test(config, FILE_NUM, ITEM_NUM_PER_FILE, run_id, fmt) From b8792b80b58fcb657cd4120e3c6ae29de38626d5 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 27 Nov 2023 10:04:22 +0800 Subject: [PATCH 24/47] fmt --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 51ee4b19a5cc0..c2f26ebc1a29b 100644 --- a/README.md +++ b/README.md @@ -45,7 +45,7 @@ target="_blank" > Docker - + Date: Mon, 27 Nov 2023 10:07:37 +0800 Subject: [PATCH 25/47] fmt --- src/connector/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index e544cb2cf49c6..c8875ca5746f6 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -73,9 +73,9 @@ mysql_async = { version = "0.32", default-features = false, features = [ mysql_common = { version = "0.30", default-features = false, features = [ "chrono", ] } -opendal = "0.41" nexmark = { version = "0.2", features = ["serde"] } num-bigint = "0.4" +opendal = "0.41" parking_lot = "0.12" paste = "1" prometheus = { version = "0.13", features = ["process"] } From fdcd292cd6f8828bde7cc70bba00b6830e329df7 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 27 Nov 2023 10:56:23 +0800 Subject: [PATCH 26/47] fmt --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index c2f26ebc1a29b..b6b26340e8faf 100644 --- a/README.md +++ b/README.md @@ -45,7 +45,7 @@ target="_blank" > Docker - + Date: Mon, 27 Nov 2023 11:14:11 +0800 Subject: [PATCH 27/47] strange fmt --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index b6b26340e8faf..51ee4b19a5cc0 100644 --- a/README.md +++ b/README.md @@ -45,7 +45,7 @@ target="_blank" > Docker - + Date: Tue, 28 Nov 2023 17:02:06 +0800 Subject: [PATCH 28/47] add e2e test for gcs source in main cron --- ci/workflows/main-cron.yml | 17 +++++++++++++++++ e2e_test/s3/gcs_source.py | 5 ++++- 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index d8e78952c141f..31360ad1c2eb7 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -357,6 +357,23 @@ steps: timeout_in_minutes: 25 retry: *auto-retry + - label: "GCS source check on AWS (json parser)" + command: "ci/scripts/s3-source-test.sh -p ci-release -s 'gcs_source.py json'" + depends_on: build + plugins: + - seek-oss/aws-sm#v2.3.1: + env: + GCS_SOURCE_TEST_CONF: ci_gcs_source_test_gcp + - docker-compose#v4.9.0: + run: rw-build-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + environment: + - GCS_SOURCE_TEST_CONF + - ./ci/plugins/upload-failure-logs + timeout_in_minutes: 25 + retry: *auto-retry + - label: "S3_v2 source check on AWS (csv parser)" command: "ci/scripts/s3-source-test.sh -p ci-release -s 'fs_source_v2.py csv_without_header'" depends_on: build diff --git a/e2e_test/s3/gcs_source.py b/e2e_test/s3/gcs_source.py index 6db9da57b1933..830c05fed36aa 100644 --- a/e2e_test/s3/gcs_source.py +++ b/e2e_test/s3/gcs_source.py @@ -110,8 +110,9 @@ def _assert_eq(field, got, expect): _gcs = lambda idx: f"{run_id}_data_{idx}.{fmt}" # put gcs files - op = opendal.Operator("gcs", root="/ci/temp/", bucket=config["GCS_BUCKET"], credential=config["GOOGLE_APPLICATION_CREDENTIALS"]) + op = opendal.Operator("gcs", root="/", bucket=config["GCS_BUCKET"], credential=config["GOOGLE_APPLICATION_CREDENTIALS"]) + print("upload file to gcs") for idx, file_str in enumerate(formatted_files): with open(_local(idx), "w") as f: f.write(file_str) @@ -120,8 +121,10 @@ def _assert_eq(field, got, expect): op.write(_gcs(idx), file_bytes) # do test + print("do test") do_test(config, FILE_NUM, ITEM_NUM_PER_FILE, run_id, fmt) # clean up gcs files + print("clean up gcs files") for idx, _ in enumerate(formatted_files): op.delete(_gcs(idx)) From 1bf66bef99f89ebdda3cad97c1925a06025fe64d Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 28 Nov 2023 17:32:56 +0800 Subject: [PATCH 29/47] install opendal in python --- ci/scripts/s3-source-test-for-opendal-fs-engine.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/scripts/s3-source-test-for-opendal-fs-engine.sh b/ci/scripts/s3-source-test-for-opendal-fs-engine.sh index 6fbbdb35e0e45..741916db49b74 100755 --- a/ci/scripts/s3-source-test-for-opendal-fs-engine.sh +++ b/ci/scripts/s3-source-test-for-opendal-fs-engine.sh @@ -29,7 +29,7 @@ echo "--- starting risingwave cluster with connector node" cargo make ci-start ci-3cn-3fe-opendal-fs-backend echo "--- Run test" -python3 -m pip install minio psycopg2-binary +python3 -m pip install minio psycopg2-binary opendal python3 e2e_test/s3/$script.py echo "--- Kill cluster" From 0e6fa9899c36e6bbf4df35c73589aca81637a529 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 28 Nov 2023 17:56:31 +0800 Subject: [PATCH 30/47] install opendal in python --- ci/scripts/s3-source-test-for-opendal-fs-engine.sh | 2 +- ci/scripts/s3-source-test.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/ci/scripts/s3-source-test-for-opendal-fs-engine.sh b/ci/scripts/s3-source-test-for-opendal-fs-engine.sh index 741916db49b74..6fbbdb35e0e45 100755 --- a/ci/scripts/s3-source-test-for-opendal-fs-engine.sh +++ b/ci/scripts/s3-source-test-for-opendal-fs-engine.sh @@ -29,7 +29,7 @@ echo "--- starting risingwave cluster with connector node" cargo make ci-start ci-3cn-3fe-opendal-fs-backend echo "--- Run test" -python3 -m pip install minio psycopg2-binary opendal +python3 -m pip install minio psycopg2-binary python3 e2e_test/s3/$script.py echo "--- Kill cluster" diff --git a/ci/scripts/s3-source-test.sh b/ci/scripts/s3-source-test.sh index 9fce76f000e31..364883bb90781 100755 --- a/ci/scripts/s3-source-test.sh +++ b/ci/scripts/s3-source-test.sh @@ -29,7 +29,7 @@ echo "--- starting risingwave cluster with connector node" cargo make ci-start ci-1cn-1fe echo "--- Run test" -python3 -m pip install minio psycopg2-binary +python3 -m pip install minio psycopg2-binary opendal python3 e2e_test/s3/$script echo "--- Kill cluster" From 7682ce274bd4a9660bfccb246ff82d59887f3422 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 29 Nov 2023 10:34:22 +0800 Subject: [PATCH 31/47] fix python scripts --- e2e_test/s3/gcs_source.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/e2e_test/s3/gcs_source.py b/e2e_test/s3/gcs_source.py index 830c05fed36aa..be559078beb88 100644 --- a/e2e_test/s3/gcs_source.py +++ b/e2e_test/s3/gcs_source.py @@ -108,9 +108,9 @@ def _assert_eq(field, got, expect): run_id = str(random.randint(1000, 9999)) _local = lambda idx: f'data_{idx}.{fmt}' _gcs = lambda idx: f"{run_id}_data_{idx}.{fmt}" - + credential_str = json.dumps(config["GOOGLE_APPLICATION_CREDENTIALS"]) # put gcs files - op = opendal.Operator("gcs", root="/", bucket=config["GCS_BUCKET"], credential=config["GOOGLE_APPLICATION_CREDENTIALS"]) + op = opendal.Operator("gcs", root="/", bucket=config["GCS_BUCKET"], credential=credential_str) print("upload file to gcs") for idx, file_str in enumerate(formatted_files): From de9f1bfaca9e07437e0e29abb98ffb89bca30942 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 29 Nov 2023 11:10:55 +0800 Subject: [PATCH 32/47] fix python scripts --- e2e_test/s3/gcs_source.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/e2e_test/s3/gcs_source.py b/e2e_test/s3/gcs_source.py index be559078beb88..c917f2c2d33fd 100644 --- a/e2e_test/s3/gcs_source.py +++ b/e2e_test/s3/gcs_source.py @@ -30,7 +30,7 @@ def format_json(data): ] -def do_test(config, file_num, item_num_per_file, prefix, fmt): +def do_test(config, file_num, item_num_per_file, prefix, fmt, credential): conn = psycopg2.connect( host="localhost", port="4566", @@ -57,7 +57,7 @@ def _encode(): connector = 'gcs', match_pattern = '{prefix}*.{fmt}', gcs.bucket_name = '{config['GCS_BUCKET']}', - gcs.credentials = '{config['GOOGLE_APPLICATION_CREDENTIALS']}', + gcs.credentials = '{credential}', ) FORMAT PLAIN ENCODE {_encode()};''') total_rows = file_num * item_num_per_file @@ -122,7 +122,7 @@ def _assert_eq(field, got, expect): # do test print("do test") - do_test(config, FILE_NUM, ITEM_NUM_PER_FILE, run_id, fmt) + do_test(config, FILE_NUM, ITEM_NUM_PER_FILE, run_id, fmt, credential_str) # clean up gcs files print("clean up gcs files") From f41cc596c9c1f60013e808e9d44436f0f3b6da2e Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 30 Nov 2023 10:58:42 +0800 Subject: [PATCH 33/47] resolve some comments --- src/connector/src/source/filesystem/file_common.rs | 2 +- .../src/source/filesystem/opendal_source/opendal_reader.rs | 4 ++-- src/connector/src/source/filesystem/s3/source/reader.rs | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index bbda0a8c14857..9698dcb9e6c77 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -92,7 +92,7 @@ where Self { name: value.key().unwrap().to_owned(), offset: 0, - size: value.size() as usize, + size: value.size().unwrap_or_default() as usize, marker: PhantomData, } } 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 438191589d268..0f61b5cae97f9 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -155,9 +155,9 @@ where .partition_input_bytes .with_label_values(&[&actor_id, &source_id, &split_id]) .inc_by(batch_size as u64); + let yield_batch = std::mem::take(&mut batch); batch_size = 0; - yield batch.clone(); - batch.clear(); + yield yield_batch; } } if !batch.is_empty() { diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index e4132ee615d3f..4d51dbc4d2b44 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -124,9 +124,9 @@ impl S3FileReader { .partition_input_bytes .with_label_values(&[&actor_id, &source_id, &split_id]) .inc_by(batch_size as u64); + let yield_batch = std::mem::take(&mut batch); batch_size = 0; - yield batch.clone(); - batch.clear(); + yield yield_batch; } } if !batch.is_empty() { From 52a95f1d31b677dde9f1215d1f736edd462a669f Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 30 Nov 2023 14:57:05 +0800 Subject: [PATCH 34/47] gcs source supports csv, add assume role for s3 --- src/connector/src/source/base.rs | 2 ++ .../src/source/filesystem/opendal_source/gcs_source.rs | 4 ++++ .../src/source/filesystem/opendal_source/mod.rs | 6 +++++- .../src/source/filesystem/opendal_source/s3_source.rs | 9 ++++++++- src/frontend/src/handler/create_source.rs | 2 +- src/source/src/connector_source.rs | 2 +- 6 files changed, 21 insertions(+), 4 deletions(-) diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index be0a386267a88..758b9a59cee39 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -393,9 +393,11 @@ impl ConnectorProperties { .ok_or_else(|| anyhow!("Must specify 'connector' in WITH clause"))?; match connector.as_str() { "s3_v2" => { + let assume_role = props.get("s3.assume_role").cloned(); return Ok(ConnectorProperties::OpenDalS3(Box::new( OpendalS3Properties { s3_properties: S3Properties::try_from_hashmap(props)?, + assume_role, }, ))); } diff --git a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs index 5a58e05c406c0..3db0e6a59e083 100644 --- a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs @@ -39,6 +39,10 @@ where if let Some(cred) = cred { builder.credential(&cred); } + + if let Some(service_account) = gcs_properties.service_account { + builder.service_account(&service_account); + } let op: Operator = Operator::new(builder)? .layer(LoggingLayer::default()) .layer(RetryLayer::default()) diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index 5c3cae8eff039..e51bf443017df 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -36,6 +36,8 @@ pub struct GcsProperties { pub bucket_name: String, #[serde(rename = "gcs.credential")] pub credential: Option, + #[serde(rename = "gcs.service_account", default)] + pub service_account: Option, #[serde(rename = "match_pattern", default)] pub match_pattern: Option, } @@ -53,6 +55,8 @@ impl SourceProperties for GcsProperties { #[derive(Clone, Debug, Deserialize, PartialEq)] pub struct OpendalS3Properties { pub s3_properties: S3Properties, + #[serde(rename = "s3.assume_role", default)] + pub assume_role: Option, } impl SourceProperties for OpendalS3Properties { @@ -77,6 +81,6 @@ impl OpenDalSourceProperties for GcsProperties { impl OpenDalSourceProperties for OpendalS3Properties { fn new_enumerator(properties: Self) -> anyhow::Result> { - OpendalEnumerator::new_s3_source(properties.s3_properties) + OpendalEnumerator::new_s3_source(properties.s3_properties, properties.assume_role) } } diff --git a/src/connector/src/source/filesystem/opendal_source/s3_source.rs b/src/connector/src/source/filesystem/opendal_source/s3_source.rs index d6978eb681ea6..f70644db74acf 100644 --- a/src/connector/src/source/filesystem/opendal_source/s3_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/s3_source.rs @@ -29,7 +29,10 @@ where C: Sized + Send + Clone + PartialEq + 'static + Sync, { /// create opendal s3 source. - pub fn new_s3_source(s3_properties: S3Properties) -> anyhow::Result { + pub fn new_s3_source( + s3_properties: S3Properties, + assume_role: Option, + ) -> anyhow::Result { // Create s3 builder. let mut builder = S3::default(); builder.bucket(&s3_properties.bucket_name); @@ -47,6 +50,10 @@ where builder.secret_access_key(&secret); } + if let Some(assume_role) = assume_role { + builder.role_arn(&assume_role); + } + builder.disable_config_load(); let (prefix, matcher) = if let Some(pattern) = s3_properties.match_pattern.as_ref() { let prefix = get_prefix(pattern); diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index e7d42c8f935b8..3e15d08fc6ae5 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -909,7 +909,7 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock vec![Encode::Csv, Encode::Json], ), GCS_CONNECTOR => hashmap!( - Format::Plain => vec![Encode::Json], + Format::Plain => vec![Encode::Csv, Encode::Json], ), MYSQL_CDC_CONNECTOR => hashmap!( Format::Plain => vec![Encode::Bytes], diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 33cf7529dfcb3..2a54563ac45d5 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -101,7 +101,7 @@ impl ConnectorSource { } ConnectorProperties::OpenDalS3(prop) => { let lister: OpendalEnumerator = - OpendalEnumerator::new_s3_source(prop.s3_properties)?; + OpendalEnumerator::new_s3_source(prop.s3_properties, prop.assume_role)?; Ok(build_opendal_fs_list_stream(lister)) } other => Err(internal_error(format!("Unsupported source: {:?}", other))), From 1a08aa7a15c52b187a626b28d6c74b330c52ae87 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 4 Dec 2023 16:15:28 +0800 Subject: [PATCH 35/47] resolve some comments --- .../source/filesystem/opendal_source/s3_source.rs | 12 ++++++++++++ src/source/src/connector_source.rs | 2 +- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/src/connector/src/source/filesystem/opendal_source/s3_source.rs b/src/connector/src/source/filesystem/opendal_source/s3_source.rs index f70644db74acf..4d048c3357779 100644 --- a/src/connector/src/source/filesystem/opendal_source/s3_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/s3_source.rs @@ -44,12 +44,24 @@ where if let Some(access) = s3_properties.access { builder.access_key_id(&access); + } else { + tracing::error!( + "access key id of aws s3 is not set, bucket {}", + s3_properties.bucket_name + ); } if let Some(secret) = s3_properties.secret { builder.secret_access_key(&secret); + } else { + tracing::error!( + "secret access key of aws s3 is not set, bucket {}", + s3_properties.bucket_name + ); } + builder.enable_virtual_host_style(); + if let Some(assume_role) = assume_role { builder.role_arn(&assume_role); } diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 2a54563ac45d5..6ff6f649e5b37 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -199,7 +199,7 @@ async fn build_opendal_fs_list_stream(lister: Openda } Err(err) => { tracing::error!("list object fail, err {}", err); - break; + return Err(err.into()); } } } From 84538360250f23cb404060e66190f7c903a7fe59 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 5 Dec 2023 16:03:39 +0800 Subject: [PATCH 36/47] resolve some comments --- .../src/executor/source/fetch_executor.rs | 103 ++++++------------ src/stream/src/from_proto/source/fs_fetch.rs | 42 ++++--- 2 files changed, 62 insertions(+), 83 deletions(-) diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 884eefdf0bddb..e441ea90c6165 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::fmt::{Debug, Formatter}; +use std::marker::PhantomData; use std::ops::Bound; use std::sync::Arc; @@ -24,7 +25,9 @@ use risingwave_common::catalog::{ColumnId, Schema, TableId}; use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{ScalarRef, ScalarRefImpl}; -use risingwave_connector::source::filesystem::opendal_source::OpendalS3Properties; +use risingwave_connector::source::filesystem::opendal_source::{ + OpenDalSourceProperties, OpendalS3Properties, +}; use risingwave_connector::source::filesystem::{GcsProperties, OpendalFsSplit}; use risingwave_connector::source::{ BoxSourceWithStateStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData, @@ -46,7 +49,7 @@ const SPLIT_BATCH_SIZE: usize = 1000; type SplitBatch = Option>; -pub struct FsFetchExecutor { +pub struct FsFetchExecutor { actor_ctx: ActorContextRef, info: ExecutorInfo, @@ -61,9 +64,11 @@ pub struct FsFetchExecutor { // config for the connector node connector_params: ConnectorParams, + + pub(crate) marker: PhantomData, } -impl FsFetchExecutor { +impl FsFetchExecutor { #[allow(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, @@ -80,6 +85,7 @@ impl FsFetchExecutor { upstream: Some(upstream), source_ctrl_opts, connector_params, + marker: PhantomData, } } @@ -266,36 +272,20 @@ impl FsFetchExecutor { // Receiving file assignments from upstream list executor, // store into state table and try building a new reader. Message::Chunk(chunk) => { - let properties = source_desc.source.config.clone(); - match properties{ - risingwave_connector::source::ConnectorProperties::Gcs(_) => { - let file_assignment = chunk - .data_chunk() - .rows() - .map(|row| { - let filename = row.datum_at(0).unwrap().into_utf8(); - let size = row.datum_at(2).unwrap().into_int64(); - let split: OpendalFsSplit= OpendalFsSplit::new(filename.to_owned(), 0, size as usize); - split - }) - .collect(); - state_store_handler.take_snapshot(file_assignment).await?; - }, - risingwave_connector::source::ConnectorProperties::OpenDalS3(_) => { - let file_assignment = chunk - .data_chunk() - .rows() - .map(|row| { - let filename = row.datum_at(0).unwrap().into_utf8(); - let size = row.datum_at(2).unwrap().into_int64(); - let split: OpendalFsSplit= OpendalFsSplit::new(filename.to_owned(), 0, size as usize); - split - }) - .collect(); - state_store_handler.take_snapshot(file_assignment).await?; - }, - _ => unreachable!() - } + let file_assignment = chunk + .data_chunk() + .rows() + .map(|row| { + let filename = row.datum_at(0).unwrap().into_utf8(); + let size = row.datum_at(2).unwrap().into_int64(); + OpendalFsSplit::::new( + filename.to_owned(), + 0, + size as usize, + ) + }) + .collect(); + state_store_handler.take_snapshot(file_assignment).await?; } _ => unreachable!(), } @@ -312,37 +302,14 @@ impl FsFetchExecutor { .get(split_id.clone()) .await? .expect("The fs_split should be in the state table."); - let properties = source_desc.source.config.clone(); - if let risingwave_connector::source::ConnectorProperties::OpenDalS3(_)=properties{ - let fs_split: OpendalFsSplit = - 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?; - } - } - else if let risingwave_connector::source::ConnectorProperties::Gcs(_)=properties{ - let fs_split: OpendalFsSplit = - match row.datum_at(1) { - Some(ScalarRefImpl::Jsonb(jsonb_ref)) => { - OpendalFsSplit::restore_from_json( - jsonb_ref.to_owned_scalar(), - )? - } - _ => unreachable!(), - }; + 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; @@ -352,10 +319,6 @@ impl FsFetchExecutor { .set(split_id, fs_split.encode_to_json()) .await?; } - } - else{ - unreachable!() - } } yield Message::Chunk(chunk); @@ -367,7 +330,7 @@ impl FsFetchExecutor { } } -impl Executor for FsFetchExecutor { +impl Executor for FsFetchExecutor { fn execute(self: Box) -> BoxedMessageStream { self.into_stream().boxed() } @@ -385,7 +348,7 @@ impl Executor for FsFetchExecutor { } } -impl Debug for FsFetchExecutor { +impl Debug for FsFetchExecutor { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { if let Some(core) = &self.stream_source_core { f.debug_struct("FsFetchExecutor") diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs index 7266006fc69de..d5edcb9737219 100644 --- a/src/stream/src/from_proto/source/fs_fetch.rs +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -15,7 +15,9 @@ use std::sync::Arc; use risingwave_common::catalog::{ColumnId, TableId}; -use risingwave_connector::source::SourceCtrlOpts; +use risingwave_connector::source::filesystem::opendal_source::OpendalS3Properties; +use risingwave_connector::source::filesystem::GcsProperties; +use risingwave_connector::source::{ConnectorProperties, SourceCtrlOpts}; use risingwave_pb::stream_plan::StreamFsFetchNode; use risingwave_source::source_desc::SourceDescBuilder; use risingwave_storage::StateStore; @@ -46,7 +48,7 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { let source_id = TableId::new(source.source_id); let source_name = source.source_name.clone(); let source_info = source.get_info()?; - + let properties = ConnectorProperties::extract(source.properties.clone())?; let source_desc_builder = SourceDescBuilder::new( source.columns.clone(), params.env.source_metrics(), @@ -57,7 +59,6 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { params.env.config().developer.connector_message_buffer_size, params.info.pk_indices.clone(), ); - let source_ctrl_opts = SourceCtrlOpts { chunk_size: params.env.config().developer.chunk_size, }; @@ -87,16 +88,31 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { state_table_handler, ); - let executor = FsFetchExecutor::new( - params.actor_context.clone(), - params.info, - stream_source_core, - upstream, - source_ctrl_opts, - params.env.connector_params(), - ) - .boxed(); - + let executor = match properties { + risingwave_connector::source::ConnectorProperties::Gcs(_) => { + FsFetchExecutor::<_, GcsProperties>::new( + params.actor_context.clone(), + params.info, + stream_source_core, + upstream, + source_ctrl_opts, + params.env.connector_params(), + ) + .boxed() + } + risingwave_connector::source::ConnectorProperties::OpenDalS3(_) => { + FsFetchExecutor::<_, OpendalS3Properties>::new( + params.actor_context.clone(), + params.info, + stream_source_core, + upstream, + source_ctrl_opts, + params.env.connector_params(), + ) + .boxed() + } + _ => unreachable!(), + }; let rate_limit = source.rate_limit.map(|x| x as _); Ok(FlowControlExecutor::new(executor, params.actor_context, rate_limit).boxed()) } From 5081ee7d025e57a57a2d6db5280f547f733c2284 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Wed, 6 Dec 2023 19:03:17 +0800 Subject: [PATCH 37/47] refactor: refine OpendalSource types Signed-off-by: Richard Chien --- src/connector/src/macros.rs | 4 +- src/connector/src/source/base.rs | 2 +- .../src/source/filesystem/file_common.rs | 27 +++------ .../filesystem/opendal_source/gcs_source.rs | 7 +-- .../source/filesystem/opendal_source/mod.rs | 56 +++++++++++-------- .../opendal_source/opendal_enumerator.rs | 31 ++++------ .../opendal_source/opendal_reader.rs | 30 ++++------ .../filesystem/opendal_source/s3_source.rs | 7 +-- .../opendal_engine/opendal_object_store.rs | 2 +- src/source/src/connector_source.rs | 12 ++-- .../src/executor/source/fetch_executor.rs | 26 ++++----- src/stream/src/from_proto/source/fs_fetch.rs | 9 ++- 12 files changed, 96 insertions(+), 117 deletions(-) diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index 81780c43bf28d..bea334e44ee77 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -32,8 +32,8 @@ macro_rules! for_all_classified_sources { { GooglePubsub, $crate::source::google_pubsub::PubsubProperties, $crate::source::google_pubsub::PubsubSplit }, { Nats, $crate::source::nats::NatsProperties, $crate::source::nats::split::NatsSplit }, { S3, $crate::source::filesystem::S3Properties, $crate::source::filesystem::FsSplit }, - { Gcs, $crate::source::filesystem::opendal_source::GcsProperties , $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::GcsProperties> }, - { OpenDalS3, $crate::source::filesystem::opendal_source::OpendalS3Properties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalS3Properties> }, + { Gcs, $crate::source::filesystem::opendal_source::GcsProperties , $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalGcs> }, + { OpendalS3, $crate::source::filesystem::opendal_source::OpendalS3Properties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalS3> }, { Test, $crate::source::test_source::TestSourceProperties, $crate::source::test_source::TestSourceSplit} } $( diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 758b9a59cee39..4d954e6cea96a 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -394,7 +394,7 @@ impl ConnectorProperties { match connector.as_str() { "s3_v2" => { let assume_role = props.get("s3.assume_role").cloned(); - return Ok(ConnectorProperties::OpenDalS3(Box::new( + return Ok(ConnectorProperties::OpendalS3(Box::new( OpendalS3Properties { s3_properties: S3Properties::try_from_hashmap(props)?, assume_role, diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index 9698dcb9e6c77..5d2b034f41129 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -20,6 +20,7 @@ use aws_sdk_s3::types::Object; use risingwave_common::types::{JsonbVal, Timestamp}; use serde::{Deserialize, Serialize}; +use super::opendal_source::OpendalSource; use crate::source::{SplitId, SplitMetaData}; /// [`FsSplit`] Describes a file or a split of a file. A file is a generic concept, @@ -74,34 +75,25 @@ impl FsSplit { /// [`OpendalFsSplit`] Describes a file or a split of a file. A file is a generic concept, /// and can be a local file, a distributed file system, or am object in S3 bucket. #[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)] -pub struct OpendalFsSplit -where - C: Send + Clone + 'static, -{ +pub struct OpendalFsSplit { pub name: String, pub offset: usize, pub size: usize, - marker: PhantomData, + _marker: PhantomData, } -impl From<&Object> for OpendalFsSplit -where - C: Send + Clone + 'static, -{ +impl From<&Object> for OpendalFsSplit { fn from(value: &Object) -> Self { Self { name: value.key().unwrap().to_owned(), offset: 0, size: value.size().unwrap_or_default() as usize, - marker: PhantomData, + _marker: PhantomData, } } } -impl SplitMetaData for OpendalFsSplit -where - C: Sized + Send + Clone + 'static, -{ +impl SplitMetaData for OpendalFsSplit { fn id(&self) -> SplitId { self.name.as_str().into() } @@ -121,16 +113,13 @@ where } } -impl OpendalFsSplit -where - C: Send + Clone + 'static, -{ +impl OpendalFsSplit { pub fn new(name: String, start: usize, size: usize) -> Self { Self { name, offset: start, size, - marker: PhantomData, + _marker: PhantomData, } } } diff --git a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs index 3db0e6a59e083..d5f303fbe1b5f 100644 --- a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs @@ -20,13 +20,10 @@ use opendal::services::Gcs; use opendal::Operator; use super::opendal_enumerator::OpendalEnumerator; -use super::{GcsProperties, OpenDalSourceProperties}; +use super::{GcsProperties, OpendalSource}; use crate::source::filesystem::s3::enumerator::get_prefix; -impl OpendalEnumerator -where - C: Send + Clone + PartialEq + 'static + Sync, -{ +impl OpendalEnumerator { /// create opendal gcs source. pub fn new_gcs_source(gcs_properties: GcsProperties) -> anyhow::Result { // Create gcs builder. diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index e51bf443017df..a3f60c2eb2417 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -43,15 +43,43 @@ pub struct GcsProperties { } impl SourceProperties for GcsProperties { - type Split = OpendalFsSplit; - type SplitEnumerator = OpendalEnumerator; - type SplitReader = OpendalReader; + type Split = OpendalFsSplit; + type SplitEnumerator = OpendalEnumerator; + type SplitReader = OpendalReader; const SOURCE_NAME: &'static str = GCS_CONNECTOR; fn init_from_pb_source(&mut self, _source: &risingwave_pb::catalog::PbSource) {} } +pub trait OpendalSource: Send + Sync + 'static + Clone + PartialEq { + type Properties: SourceProperties + Send + Sync; + + fn new_enumerator(properties: Self::Properties) -> anyhow::Result>; +} + +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct OpendalS3; + +impl OpendalSource for OpendalS3 { + type Properties = OpendalS3Properties; + + fn new_enumerator(properties: Self::Properties) -> anyhow::Result> { + OpendalEnumerator::new_s3_source(properties.s3_properties, properties.assume_role) + } +} + +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct OpendalGcs; + +impl OpendalSource for OpendalGcs { + type Properties = GcsProperties; + + fn new_enumerator(properties: Self::Properties) -> anyhow::Result> { + OpendalEnumerator::new_gcs_source(properties) + } +} + #[derive(Clone, Debug, Deserialize, PartialEq)] pub struct OpendalS3Properties { pub s3_properties: S3Properties, @@ -60,27 +88,11 @@ pub struct OpendalS3Properties { } impl SourceProperties for OpendalS3Properties { - type Split = OpendalFsSplit; - type SplitEnumerator = OpendalEnumerator; - type SplitReader = OpendalReader; + type Split = OpendalFsSplit; + type SplitEnumerator = OpendalEnumerator; + type SplitReader = OpendalReader; const SOURCE_NAME: &'static str = OPENDAL_S3_CONNECTOR; fn init_from_pb_source(&mut self, _source: &risingwave_pb::catalog::PbSource) {} } - -pub trait OpenDalSourceProperties: Sized + Send + Clone + PartialEq + 'static + Sync { - fn new_enumerator(properties: Self) -> anyhow::Result>; -} - -impl OpenDalSourceProperties for GcsProperties { - fn new_enumerator(properties: Self) -> anyhow::Result> { - OpendalEnumerator::new_gcs_source(properties) - } -} - -impl OpenDalSourceProperties for OpendalS3Properties { - fn new_enumerator(properties: Self) -> anyhow::Result> { - OpendalEnumerator::new_s3_source(properties.s3_properties, properties.assume_role) - } -} diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index 0c44c77dcdb15..cc05a09aa350a 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -21,46 +21,37 @@ use futures::StreamExt; use opendal::{Metakey, Operator}; use risingwave_common::types::Timestamp; -use super::OpenDalSourceProperties; +use super::OpendalSource; use crate::source::filesystem::{FsPageItem, OpendalFsSplit}; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; #[derive(Debug, Clone)] -pub struct OpendalEnumerator -where - C: Send + Clone + Sized + PartialEq + 'static + Sync, -{ +pub struct OpendalEnumerator { pub(crate) op: Operator, // prefix is used to reduce the number of objects to be listed pub(crate) prefix: Option, pub(crate) matcher: Option, - pub(crate) marker: PhantomData, + pub(crate) marker: PhantomData, } #[async_trait] -impl SplitEnumerator for OpendalEnumerator -where - C: Sized + Send + Clone + PartialEq + 'static + Sync, -{ - type Properties = C; - type Split = OpendalFsSplit; +impl SplitEnumerator for OpendalEnumerator { + type Properties = Src::Properties; + type Split = OpendalFsSplit; async fn new( - properties: Self::Properties, + properties: Src::Properties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result> { - Self::Properties::new_enumerator(properties) + ) -> anyhow::Result { + Src::new_enumerator(properties) } - async fn list_splits(&mut self) -> anyhow::Result>> { + async fn list_splits(&mut self) -> anyhow::Result>> { Ok(vec![]) } } -impl OpendalEnumerator -where - C: Send + Clone + Sized + PartialEq + 'static + Sync, -{ +impl OpendalEnumerator { pub async fn list(&self) -> anyhow::Result { // Currently, we need to do full list and then filter the prefix and matcher, // After OpenDAL implementing the list prefix, we can use the user-specified prefix. 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 0f61b5cae97f9..d2758ba9bb0ef 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -22,7 +22,7 @@ use tokio::io::BufReader; use tokio_util::io::{ReaderStream, StreamReader}; use super::opendal_enumerator::OpendalEnumerator; -use super::OpenDalSourceProperties; +use super::OpendalSource; use crate::parser::{ByteStreamSourceParserImpl, ParserConfig}; use crate::source::filesystem::{nd_streaming, OpendalFsSplit}; use crate::source::{ @@ -33,28 +33,25 @@ use crate::source::{ const MAX_CHANNEL_BUFFER_SIZE: usize = 2048; const STREAM_READER_CAPACITY: usize = 4096; #[derive(Debug, Clone)] -pub struct OpendalReader { - connector: OpendalEnumerator, - splits: Vec>, +pub struct OpendalReader { + connector: OpendalEnumerator, + splits: Vec>, parser_config: ParserConfig, source_ctx: SourceContextRef, } #[async_trait] -impl SplitReader for OpendalReader -where - C: Send + Clone + PartialEq + 'static + Sync, -{ - type Properties = C; - type Split = OpendalFsSplit; +impl SplitReader for OpendalReader { + type Properties = Src::Properties; + type Split = OpendalFsSplit; async fn new( - properties: C, - splits: Vec>, + properties: Src::Properties, + splits: Vec>, parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, ) -> Result { - let connector = Self::Properties::new_enumerator(properties)?; + let connector = Src::new_enumerator(properties)?; let opendal_reader = OpendalReader { connector, splits, @@ -69,10 +66,7 @@ where } } -impl OpendalReader -where - C: Send + Clone + Sized + PartialEq + 'static, -{ +impl OpendalReader { #[try_stream(boxed, ok = StreamChunkWithState, error = RwError)] async fn into_chunk_stream(self) { for split in self.splits { @@ -111,7 +105,7 @@ where #[try_stream(boxed, ok = Vec, error = anyhow::Error)] pub async fn stream_read_object( op: Operator, - split: OpendalFsSplit, + split: OpendalFsSplit, source_ctx: SourceContextRef, ) { let actor_id = source_ctx.source_info.actor_id.to_string(); diff --git a/src/connector/src/source/filesystem/opendal_source/s3_source.rs b/src/connector/src/source/filesystem/opendal_source/s3_source.rs index 4d048c3357779..27d7a639f7016 100644 --- a/src/connector/src/source/filesystem/opendal_source/s3_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/s3_source.rs @@ -20,14 +20,11 @@ use opendal::services::S3; use opendal::Operator; use super::opendal_enumerator::OpendalEnumerator; -use super::OpenDalSourceProperties; +use super::OpendalSource; // use crate::source::filesystem::opendal_source::get_prefix; use crate::source::filesystem::{s3::enumerator::get_prefix, S3Properties}; -impl OpendalEnumerator -where - C: Sized + Send + Clone + PartialEq + 'static + Sync, -{ +impl OpendalEnumerator { /// create opendal s3 source. pub fn new_s3_source( s3_properties: S3Properties, diff --git a/src/object_store/src/object/opendal_engine/opendal_object_store.rs b/src/object_store/src/object/opendal_engine/opendal_object_store.rs index 552ec6381fc54..37200eea81c8a 100644 --- a/src/object_store/src/object/opendal_engine/opendal_object_store.rs +++ b/src/object_store/src/object/opendal_engine/opendal_object_store.rs @@ -117,7 +117,7 @@ impl ObjectStore for OpendalObjectStore { let reader = self.op.reader_with(path).range(range).await?; let stream = reader .into_stream() - .map(|item| item.map_err(|e| ObjectError::internal(format!("OpenDalError: {:?}", e)))); + .map(|item| item.map_err(|e| ObjectError::internal(format!("OpendalError: {:?}", e)))); Ok(Box::pin(stream)) } diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 6ff6f649e5b37..0575b477fde2e 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -28,9 +28,9 @@ use risingwave_connector::dispatch_source_prop; use risingwave_connector::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; use risingwave_connector::source::filesystem::opendal_source::opendal_enumerator::OpendalEnumerator; use risingwave_connector::source::filesystem::opendal_source::{ - OpenDalSourceProperties, OpendalS3Properties, + OpendalGcs, OpendalS3, OpendalSource, }; -use risingwave_connector::source::filesystem::{FsPageItem, GcsProperties}; +use risingwave_connector::source::filesystem::FsPageItem; use risingwave_connector::source::{ create_split_reader, BoxSourceWithStateStream, BoxTryStream, Column, ConnectorProperties, ConnectorState, FsFilterCtrlCtx, SourceColumnDesc, SourceContext, SplitReader, @@ -95,12 +95,12 @@ impl ConnectorSource { let config = self.config.clone(); match config { ConnectorProperties::Gcs(prop) => { - let lister: OpendalEnumerator = + let lister: OpendalEnumerator = OpendalEnumerator::new_gcs_source(*prop)?; Ok(build_opendal_fs_list_stream(lister)) } - ConnectorProperties::OpenDalS3(prop) => { - let lister: OpendalEnumerator = + ConnectorProperties::OpendalS3(prop) => { + let lister: OpendalEnumerator = OpendalEnumerator::new_s3_source(prop.s3_properties, prop.assume_role)?; Ok(build_opendal_fs_list_stream(lister)) } @@ -172,7 +172,7 @@ impl ConnectorSource { } #[try_stream(boxed, ok = FsPageItem, error = RwError)] -async fn build_opendal_fs_list_stream(lister: OpendalEnumerator) { +async fn build_opendal_fs_list_stream(lister: OpendalEnumerator) { let prefix = lister .get_prefix() .as_ref() diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index e441ea90c6165..8d725a313e528 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -26,9 +26,9 @@ use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{ScalarRef, ScalarRefImpl}; use risingwave_connector::source::filesystem::opendal_source::{ - OpenDalSourceProperties, OpendalS3Properties, + OpendalGcs, OpendalS3, OpendalSource, }; -use risingwave_connector::source::filesystem::{GcsProperties, OpendalFsSplit}; +use risingwave_connector::source::filesystem::OpendalFsSplit; use risingwave_connector::source::{ BoxSourceWithStateStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData, StreamChunkWithState, @@ -49,7 +49,7 @@ const SPLIT_BATCH_SIZE: usize = 1000; type SplitBatch = Option>; -pub struct FsFetchExecutor { +pub struct FsFetchExecutor { actor_ctx: ActorContextRef, info: ExecutorInfo, @@ -65,10 +65,10 @@ pub struct FsFetchExecutor { // config for the connector node connector_params: ConnectorParams, - pub(crate) marker: PhantomData, + _marker: PhantomData, } -impl FsFetchExecutor { +impl FsFetchExecutor { #[allow(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, @@ -85,7 +85,7 @@ impl FsFetchExecutor { upstream: Some(upstream), source_ctrl_opts, connector_params, - marker: PhantomData, + _marker: PhantomData, } } @@ -114,12 +114,12 @@ impl FsFetchExecutor { let split = match row.datum_at(1) { Some(ScalarRefImpl::Jsonb(jsonb_ref)) => match properties { risingwave_connector::source::ConnectorProperties::Gcs(_) => { - let split: OpendalFsSplit = + let split: OpendalFsSplit = OpendalFsSplit::restore_from_json(jsonb_ref.to_owned_scalar())?; SplitImpl::from(split) } - risingwave_connector::source::ConnectorProperties::OpenDalS3(_) => { - let split: OpendalFsSplit = + risingwave_connector::source::ConnectorProperties::OpendalS3(_) => { + let split: OpendalFsSplit = OpendalFsSplit::restore_from_json(jsonb_ref.to_owned_scalar())?; SplitImpl::from(split) } @@ -278,7 +278,7 @@ impl FsFetchExecutor { .map(|row| { let filename = row.datum_at(0).unwrap().into_utf8(); let size = row.datum_at(2).unwrap().into_int64(); - OpendalFsSplit::::new( + OpendalFsSplit::::new( filename.to_owned(), 0, size as usize, @@ -304,7 +304,7 @@ impl FsFetchExecutor { .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( + OpendalFsSplit::::restore_from_json( jsonb_ref.to_owned_scalar(), )? } @@ -330,7 +330,7 @@ impl FsFetchExecutor { } } -impl Executor for FsFetchExecutor { +impl Executor for FsFetchExecutor { fn execute(self: Box) -> BoxedMessageStream { self.into_stream().boxed() } @@ -348,7 +348,7 @@ impl Executor for FsFetchExecutor Debug for FsFetchExecutor { +impl Debug for FsFetchExecutor { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { if let Some(core) = &self.stream_source_core { f.debug_struct("FsFetchExecutor") diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs index d5edcb9737219..227fa7abf4ba3 100644 --- a/src/stream/src/from_proto/source/fs_fetch.rs +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -15,8 +15,7 @@ use std::sync::Arc; use risingwave_common::catalog::{ColumnId, TableId}; -use risingwave_connector::source::filesystem::opendal_source::OpendalS3Properties; -use risingwave_connector::source::filesystem::GcsProperties; +use risingwave_connector::source::filesystem::opendal_source::{OpendalGcs, OpendalS3}; use risingwave_connector::source::{ConnectorProperties, SourceCtrlOpts}; use risingwave_pb::stream_plan::StreamFsFetchNode; use risingwave_source::source_desc::SourceDescBuilder; @@ -90,7 +89,7 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { let executor = match properties { risingwave_connector::source::ConnectorProperties::Gcs(_) => { - FsFetchExecutor::<_, GcsProperties>::new( + FsFetchExecutor::<_, OpendalGcs>::new( params.actor_context.clone(), params.info, stream_source_core, @@ -100,8 +99,8 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { ) .boxed() } - risingwave_connector::source::ConnectorProperties::OpenDalS3(_) => { - FsFetchExecutor::<_, OpendalS3Properties>::new( + risingwave_connector::source::ConnectorProperties::OpendalS3(_) => { + FsFetchExecutor::<_, OpendalS3>::new( params.actor_context.clone(), params.info, stream_source_core, From b7c26da95a46d7f4b15f20966de0b70f305fd7fc Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 6 Dec 2023 20:40:07 +0800 Subject: [PATCH 38/47] resolve some comments --- src/connector/src/macros.rs | 1 + src/connector/src/source/filesystem/opendal_source/mod.rs | 4 ---- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index bea334e44ee77..d777faf9bb2cc 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -23,6 +23,7 @@ macro_rules! for_all_classified_sources { { Citus } }, // other sources + // todo: file source do not nest with mq source. { { Kafka, $crate::source::kafka::KafkaProperties, $crate::source::kafka::KafkaSplit }, { Pulsar, $crate::source::pulsar::PulsarProperties, $crate::source::pulsar::PulsarSplit }, diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index a3f60c2eb2417..d72f94badb85c 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -48,8 +48,6 @@ impl SourceProperties for GcsProperties { type SplitReader = OpendalReader; const SOURCE_NAME: &'static str = GCS_CONNECTOR; - - fn init_from_pb_source(&mut self, _source: &risingwave_pb::catalog::PbSource) {} } pub trait OpendalSource: Send + Sync + 'static + Clone + PartialEq { @@ -93,6 +91,4 @@ impl SourceProperties for OpendalS3Properties { type SplitReader = OpendalReader; const SOURCE_NAME: &'static str = OPENDAL_S3_CONNECTOR; - - fn init_from_pb_source(&mut self, _source: &risingwave_pb::catalog::PbSource) {} } From 1bab9fcf7b7b56accc7beb09ef4977e3315d1d36 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 7 Dec 2023 11:29:37 +0800 Subject: [PATCH 39/47] minor --- src/connector/src/source/filesystem/file_common.rs | 9 +++++++++ .../filesystem/opendal_source/opendal_enumerator.rs | 4 +++- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index 5d2b034f41129..3ad1929cadb52 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -122,6 +122,15 @@ impl OpendalFsSplit { _marker: PhantomData, } } + + pub fn empty_split() -> Self { + Self { + name: "empty_split".to_string(), + offset: 0, + size: 0, + _marker: PhantomData, + } + } } #[derive(Clone, Debug)] diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index cc05a09aa350a..fd041630a597e 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -47,7 +47,9 @@ impl SplitEnumerator for OpendalEnumerator { } async fn list_splits(&mut self) -> anyhow::Result>> { - Ok(vec![]) + let empty_split: OpendalFsSplit = OpendalFsSplit::empty_split(); + + Ok(vec![empty_split]) } } From a352b0c210e17b0cd3236d5532f19bf01b1151fe Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 7 Dec 2023 15:39:16 +0800 Subject: [PATCH 40/47] remove Timestamp --- .../src/source/filesystem/file_common.rs | 15 ++------------- .../opendal_source/opendal_enumerator.rs | 13 +++++-------- src/stream/src/executor/source/list_executor.rs | 4 ++-- 3 files changed, 9 insertions(+), 23 deletions(-) diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index 3ad1929cadb52..a865ee7e4b3ed 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -17,7 +17,7 @@ use std::marker::PhantomData; use anyhow::anyhow; use aws_sdk_s3::types::Object; -use risingwave_common::types::{JsonbVal, Timestamp}; +use risingwave_common::types::{JsonbVal, Timestamptz}; use serde::{Deserialize, Serialize}; use super::opendal_source::OpendalSource; @@ -137,18 +137,7 @@ impl OpendalFsSplit { pub struct FsPageItem { pub name: String, pub size: i64, - pub timestamp: Timestamp, + pub timestamp: Timestamptz, } pub type FsPage = Vec; - -impl From<&Object> for FsPageItem { - fn from(value: &Object) -> Self { - let aws_ts = value.last_modified().unwrap(); - Self { - name: value.key().unwrap().to_owned(), - size: value.size().unwrap_or_default(), - timestamp: Timestamp::from_timestamp_uncheck(aws_ts.secs(), aws_ts.subsec_nanos()), - } - } -} diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index fd041630a597e..fbd0d5ee350e6 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -15,11 +15,11 @@ use std::marker::PhantomData; use async_trait::async_trait; -use chrono::NaiveDateTime; +use chrono::{DateTime, Utc}; use futures::stream::{self, BoxStream}; use futures::StreamExt; use opendal::{Metakey, Operator}; -use risingwave_common::types::Timestamp; +use risingwave_common::types::Timestamptz; use super::OpendalSource; use crate::source::filesystem::{FsPageItem, OpendalFsSplit}; @@ -78,13 +78,10 @@ impl OpendalEnumerator { let om = object.metadata(); let t = match om.last_modified() { - Some(t) => t.naive_utc(), - None => { - let timestamp = 0; - NaiveDateTime::from_timestamp_opt(timestamp, 0).unwrap() - } + Some(t) => t, + None => DateTime::::from_timestamp(0, 0).unwrap_or_default(), }; - let timestamp = Timestamp::new(t); + let timestamp = Timestamptz::from(t); let size = om.content_length() as i64; let metadata = FsPageItem { name, diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/list_executor.rs index d73a103394a48..841435e1f8786 100644 --- a/src/stream/src/executor/source/list_executor.rs +++ b/src/stream/src/executor/source/list_executor.rs @@ -106,7 +106,7 @@ impl FsListExecutor { Op::Insert, OwnedRow::new(vec![ Some(ScalarImpl::Utf8(page_item.name.into_boxed_str())), - Some(ScalarImpl::Timestamp(page_item.timestamp)), + Some(ScalarImpl::Timestamptz(page_item.timestamp)), Some(ScalarImpl::Int64(page_item.size)), ]), ) @@ -115,7 +115,7 @@ impl FsListExecutor { Ok(StreamChunk::from_rows( &rows, - &[DataType::Varchar, DataType::Timestamp, DataType::Int64], + &[DataType::Varchar, DataType::Timestamptz, DataType::Int64], )) }); From 7beee7c7d58f283c86bf3d3491d80e162c82ad39 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 7 Dec 2023 16:38:19 +0800 Subject: [PATCH 41/47] minor --- src/frontend/src/optimizer/plan_node/logical_source.rs | 2 +- src/stream/src/executor/source/fetch_executor.rs | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 74d0a50ffcf77..640e7d010ce6a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -182,7 +182,7 @@ impl LogicalSource { column_desc: ColumnDesc::from_field_with_column_id( &Field { name: "last_edit_time".to_string(), - data_type: DataType::Timestamp, + data_type: DataType::Timestamptz, sub_fields: vec![], type_name: "".to_string(), }, diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 8d725a313e528..584d1d76aaa74 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -297,7 +297,8 @@ impl FsFetchExecutor { split_offset_mapping, }) => { let mapping = split_offset_mapping.unwrap(); - for (split_id, offset) in mapping { + 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? From e04e46fc43da2c68fd814e9a9289cc4cb786f37d Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 8 Dec 2023 15:05:25 +0800 Subject: [PATCH 42/47] add github label --- ci/scripts/notify.py | 1 + ci/workflows/main-cron.yml | 5 +++++ 2 files changed, 6 insertions(+) diff --git a/ci/scripts/notify.py b/ci/scripts/notify.py index 0bff5b18552bd..b21a2c697224e 100755 --- a/ci/scripts/notify.py +++ b/ci/scripts/notify.py @@ -20,6 +20,7 @@ "e2e-clickhouse-sink-tests": ["bohan"], "e2e-pulsar-sink-tests": ["renjie"], "s3-source-test-for-opendal-fs-engine": ["congyi"], + "gcs-source-test": ["congyi"], "pulsar-source-tests": ["renjie"], "connector-node-integration-test": ["siyuan"], } diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 8da57bc5934ad..25725b10fd58b 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -469,7 +469,12 @@ steps: retry: *auto-retry - label: "GCS source check on AWS (json parser)" + key: "gcs-source-tests" command: "ci/scripts/s3-source-test.sh -p ci-release -s 'gcs_source.py json'" + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-gcs-source-tests" + || build.env("CI_STEPS") =~ /(^|,)gcs-source-tests?(,|$$)/ depends_on: build plugins: - seek-oss/aws-sm#v2.3.1: From 40ac24bf01e8db6bb51d0adb133d345c6551f49f Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 11 Dec 2023 16:10:16 +0800 Subject: [PATCH 43/47] use prefix list --- Cargo.lock | 35 +++++++++++++++++-- src/connector/Cargo.toml | 2 +- .../opendal_source/opendal_enumerator.rs | 21 ++++------- src/source/src/connector_source.rs | 15 +++----- 4 files changed, 44 insertions(+), 29 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 827f6fff73681..327b71995922b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5595,6 +5595,37 @@ dependencies = [ "uuid", ] +[[package]] +name = "opendal" +version = "0.43.0" +source = "git+https://github.com/apache/incubator-opendal?rev=9a222e4d72b328a24d5775b1565292f4636bbe69#9a222e4d72b328a24d5775b1565292f4636bbe69" +dependencies = [ + "anyhow", + "async-compat", + "async-trait", + "backon", + "base64 0.21.4", + "bytes", + "chrono", + "flagset", + "futures", + "http 0.2.9", + "log", + "md-5", + "once_cell", + "parking_lot 0.12.1", + "percent-encoding", + "pin-project", + "quick-xml 0.30.0", + "reqsign", + "reqwest", + "serde", + "serde_json", + "sha2", + "tokio", + "uuid", +] + [[package]] name = "openidconnect" version = "3.4.0" @@ -7752,7 +7783,7 @@ dependencies = [ "mysql_common", "nexmark", "num-bigint", - "opendal 0.41.0", + "opendal 0.43.0 (git+https://github.com/apache/incubator-opendal?rev=9a222e4d72b328a24d5775b1565292f4636bbe69)", "parking_lot 0.12.1", "paste", "pretty_assertions", @@ -8307,7 +8338,7 @@ dependencies = [ "itertools 0.12.0", "madsim-aws-sdk-s3", "madsim-tokio", - "opendal 0.43.0", + "opendal 0.43.0 (registry+https://github.com/rust-lang/crates.io-index)", "prometheus", "risingwave_common", "rustls", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 9470b115945af..1aa4ccd997b5d 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -78,7 +78,7 @@ mysql_common = { version = "0.31", default-features = false, features = [ ] } nexmark = { version = "0.2", features = ["serde"] } num-bigint = "0.4" -opendal = "0.41" +opendal = { git = "https://github.com/apache/incubator-opendal", rev = "9a222e4d72b328a24d5775b1565292f4636bbe69" } parking_lot = "0.12" paste = "1" prometheus = { version = "0.13", features = ["process"] } diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index fbd0d5ee350e6..1a10cdef1309b 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -55,20 +55,15 @@ impl SplitEnumerator for OpendalEnumerator { impl OpendalEnumerator { pub async fn list(&self) -> anyhow::Result { - // Currently, we need to do full list and then filter the prefix and matcher, - // After OpenDAL implementing the list prefix, we can use the user-specified prefix. - // https://github.com/apache/incubator-opendal/issues/3247 - // todo(wcy-fdu): manual filtering prefix - - // let _prefix = match &self.prefix { - // Some(prefix) => prefix, - // None => "", - // }; + let prefix = match &self.prefix { + Some(prefix) => prefix, + None => "", + }; let object_lister = self .op - .lister_with("/") - .delimiter("") + .lister_with(prefix) + .recursive(true) .metakey(Metakey::ContentLength | Metakey::LastModified) .await?; let stream = stream::unfold(object_lister, |mut object_lister| async move { @@ -101,10 +96,6 @@ impl OpendalEnumerator { Ok(stream.boxed()) } - pub fn get_prefix(&self) -> &Option { - &self.prefix - } - pub fn get_matcher(&self) -> &Option { &self.matcher } diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 0fd6f09880c76..00c7e33196abf 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -172,23 +172,16 @@ impl ConnectorSource { #[try_stream(boxed, ok = FsPageItem, error = RwError)] async fn build_opendal_fs_list_stream(lister: OpendalEnumerator) { - let prefix = lister - .get_prefix() - .as_ref() - .map(|s| s.as_str()) - .unwrap_or(""); - let matcher = lister.get_matcher(); let mut object_metadata_iter = lister.list().await?; while let Some(list_res) = object_metadata_iter.next().await { match list_res { Ok(res) => { - if res.name.starts_with(prefix) - && matcher - .as_ref() - .map(|m| m.matches(&res.name)) - .unwrap_or(true) + if matcher + .as_ref() + .map(|m| m.matches(&res.name)) + .unwrap_or(true) { yield res } else { From c05c22e905efce7fd0a1e7f0e430a9b95c2cd0cd Mon Sep 17 00:00:00 2001 From: wcy-fdu Date: Mon, 11 Dec 2023 08:45:47 +0000 Subject: [PATCH 44/47] Fix "cargo-hakari" --- Cargo.lock | 1 + src/workspace-hack/Cargo.toml | 4 +++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/Cargo.lock b/Cargo.lock index 327b71995922b..b3a6bb5d33c0e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -11619,6 +11619,7 @@ dependencies = [ "futures-task", "futures-util", "generic-array", + "getrandom", "hashbrown 0.12.3", "hashbrown 0.14.0", "hmac", diff --git a/src/workspace-hack/Cargo.toml b/src/workspace-hack/Cargo.toml index 3582d27f0135f..4be5bc7b4eeaf 100644 --- a/src/workspace-hack/Cargo.toml +++ b/src/workspace-hack/Cargo.toml @@ -55,6 +55,7 @@ futures-sink = { version = "0.3" } futures-task = { version = "0.3" } futures-util = { version = "0.3", features = ["channel", "io", "sink"] } generic-array = { version = "0.14", default-features = false, features = ["more_lengths", "zeroize"] } +getrandom = { git = "https://github.com/madsim-rs/getrandom.git", rev = "e79a7ae", default-features = false, features = ["std"] } hashbrown-582f2526e08bb6a0 = { package = "hashbrown", version = "0.14", features = ["nightly", "raw"] } hashbrown-5ef9efb8ec2df382 = { package = "hashbrown", version = "0.12", features = ["nightly", "raw"] } hmac = { version = "0.12", default-features = false, features = ["reset"] } @@ -104,7 +105,7 @@ redis = { version = "0.24", features = ["async-std-comp", "tokio-comp"] } regex = { version = "1" } regex-automata = { version = "0.4", default-features = false, features = ["dfa-onepass", "hybrid", "meta", "nfa-backtrack", "perf-inline", "perf-literal", "unicode"] } regex-syntax = { version = "0.8" } -reqwest = { version = "0.11", features = ["blocking", "json", "rustls-tls"] } +reqwest = { version = "0.11", features = ["blocking", "json", "rustls-tls", "rustls-tls-native-roots", "stream"] } ring = { version = "0.16", features = ["std"] } rust_decimal = { version = "1", features = ["db-postgres", "maths"] } rustls = { version = "0.21" } @@ -158,6 +159,7 @@ deranged = { version = "0.3", default-features = false, features = ["powerfmt", either = { version = "1", features = ["serde"] } fixedbitset = { version = "0.4" } frunk_core = { version = "0.4", default-features = false, features = ["std"] } +getrandom = { git = "https://github.com/madsim-rs/getrandom.git", rev = "e79a7ae", default-features = false, features = ["std"] } hashbrown-582f2526e08bb6a0 = { package = "hashbrown", version = "0.14", features = ["nightly", "raw"] } itertools-93f6ce9d446188ac = { package = "itertools", version = "0.10" } itertools-a6292c17cd707f01 = { package = "itertools", version = "0.11" } From cdce5db4ce0f39a59b7b2176d1d56438d6052996 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 14 Dec 2023 16:33:33 +0800 Subject: [PATCH 45/47] rerun --- .../src/source/filesystem/opendal_source/s3_source.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/connector/src/source/filesystem/opendal_source/s3_source.rs b/src/connector/src/source/filesystem/opendal_source/s3_source.rs index 27d7a639f7016..22bac20be8dc0 100644 --- a/src/connector/src/source/filesystem/opendal_source/s3_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/s3_source.rs @@ -21,8 +21,8 @@ use opendal::Operator; use super::opendal_enumerator::OpendalEnumerator; use super::OpendalSource; -// use crate::source::filesystem::opendal_source::get_prefix; -use crate::source::filesystem::{s3::enumerator::get_prefix, S3Properties}; +use crate::source::filesystem::s3::enumerator::get_prefix; +use crate::source::filesystem::S3Properties; impl OpendalEnumerator { /// create opendal s3 source. From b320bac1753fa789939abe4380e5c9ff77587f7d Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 15 Dec 2023 17:03:36 +0800 Subject: [PATCH 46/47] minor --- ci/scripts/notify.py | 1 - 1 file changed, 1 deletion(-) diff --git a/ci/scripts/notify.py b/ci/scripts/notify.py index aea4f501610d8..54332df0ca1dc 100755 --- a/ci/scripts/notify.py +++ b/ci/scripts/notify.py @@ -20,7 +20,6 @@ "e2e-clickhouse-sink-tests": ["bohan"], "e2e-pulsar-sink-tests": ["renjie"], "s3-source-test-for-opendal-fs-engine": ["congyi"], - "gcs-source-test": ["congyi"], "pulsar-source-tests": ["renjie"], "connector-node-integration-test": ["siyuan"], } From 6e3b49235a75ec81bf627d3179ed7e1c6afc89df Mon Sep 17 00:00:00 2001 From: wcy-fdu Date: Mon, 18 Dec 2023 06:20:13 +0000 Subject: [PATCH 47/47] Fix "cargo-hakari" --- Cargo.lock | 1 + src/workspace-hack/Cargo.toml | 1 + 2 files changed, 2 insertions(+) diff --git a/Cargo.lock b/Cargo.lock index a280a2c947007..ebaceffb0dac5 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -12408,6 +12408,7 @@ dependencies = [ "futures-util", "generic-array", "getrandom", + "governor", "hashbrown 0.12.3", "hashbrown 0.14.0", "hmac", diff --git a/src/workspace-hack/Cargo.toml b/src/workspace-hack/Cargo.toml index 4be5bc7b4eeaf..1fd062e702b66 100644 --- a/src/workspace-hack/Cargo.toml +++ b/src/workspace-hack/Cargo.toml @@ -56,6 +56,7 @@ futures-task = { version = "0.3" } futures-util = { version = "0.3", features = ["channel", "io", "sink"] } generic-array = { version = "0.14", default-features = false, features = ["more_lengths", "zeroize"] } getrandom = { git = "https://github.com/madsim-rs/getrandom.git", rev = "e79a7ae", default-features = false, features = ["std"] } +governor = { version = "0.6", default-features = false, features = ["dashmap", "jitter", "std"] } hashbrown-582f2526e08bb6a0 = { package = "hashbrown", version = "0.14", features = ["nightly", "raw"] } hashbrown-5ef9efb8ec2df382 = { package = "hashbrown", version = "0.12", features = ["nightly", "raw"] } hmac = { version = "0.12", default-features = false, features = ["reset"] }