From d8349c9025d7a4693e3525be2cd7a18e7659cc6a Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 11 Mar 2024 17:24:29 +0800 Subject: [PATCH 01/76] save work --- Cargo.lock | 102 ++++++++++++++- src/connector/src/sink/mod.rs | 4 + src/connector/src/sink/opendal/gcs.rs | 23 ++++ src/connector/src/sink/opendal/mod.rs | 46 +++++++ src/connector/src/sink/opendal/s3.rs | 174 ++++++++++++++++++++++++++ 5 files changed, 346 insertions(+), 3 deletions(-) create mode 100644 src/connector/src/sink/opendal/gcs.rs create mode 100644 src/connector/src/sink/opendal/mod.rs create mode 100644 src/connector/src/sink/opendal/s3.rs diff --git a/Cargo.lock b/Cargo.lock index bf8d3c00df2d5..928de9d3f645e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -23,6 +23,17 @@ version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "aae1277d39aeec15cb388266ecc24b11c80469deae6067e17a1a7aa9e5c1f234" +[[package]] +name = "aes" +version = "0.8.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b169f7a6d4742236a0a00c541b845991d0ac43e546831af1249753ab4c3aa3a0" +dependencies = [ + "cfg-if", + "cipher", + "cpufeatures", +] + [[package]] name = "ahash" version = "0.7.7" @@ -1647,6 +1658,15 @@ dependencies = [ "generic-array", ] +[[package]] +name = "block-padding" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a8894febbff9f758034a5b8e12d87918f56dfc64a8e1fe757d65e29041538d93" +dependencies = [ + "generic-array", +] + [[package]] name = "blocking" version = "1.3.1" @@ -1932,6 +1952,15 @@ version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" +[[package]] +name = "cbc" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26b52a9543ae338f279b96b0b9fed9c8093744685043739079ce85cd58f289a6" +dependencies = [ + "cipher", +] + [[package]] name = "cc" version = "1.0.83" @@ -2046,6 +2075,16 @@ dependencies = [ "half 1.8.2", ] +[[package]] +name = "cipher" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "773f3b9af64447d2ce9850330c473515014aa235e6a783b02db81ff39e4a3dad" +dependencies = [ + "crypto-common", + "inout", +] + [[package]] name = "clang-sys" version = "1.6.1" @@ -5232,6 +5271,16 @@ dependencies = [ "syn 2.0.48", ] +[[package]] +name = "inout" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a0c10553d664a4d0bcff9f4215d0aac67a639cc68ef660840afe309b807bc9f5" +dependencies = [ + "block-padding", + "generic-array", +] + [[package]] name = "inquire" version = "0.7.0" @@ -7265,6 +7314,16 @@ dependencies = [ "prost-types 0.11.9", ] +[[package]] +name = "pbkdf2" +version = "0.12.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f8ed6a7761f76e3b9f92dfb0a60a6a6477c61024b775147ff0973a02653abaf2" +dependencies = [ + "digest", + "hmac", +] + [[package]] name = "peeking_take_while" version = "0.1.2" @@ -7437,6 +7496,21 @@ dependencies = [ "spki 0.7.2", ] +[[package]] +name = "pkcs5" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e847e2c91a18bfa887dd028ec33f2fe6f25db77db3619024764914affe8b69a6" +dependencies = [ + "aes", + "cbc", + "der 0.7.8", + "pbkdf2", + "scrypt", + "sha2", + "spki 0.7.2", +] + [[package]] name = "pkcs8" version = "0.9.0" @@ -7454,6 +7528,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f950b2377845cebe5cf8b5165cb3cc1a5e0fa5cfa3e1f7f55707d8fd82e0a7b7" dependencies = [ "der 0.7.8", + "pkcs5", + "rand_core", "spki 0.7.2", ] @@ -8469,9 +8545,9 @@ dependencies = [ [[package]] name = "reqsign" -version = "0.14.6" +version = "0.14.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dce87f66ba6c6acef277a729f989a0eca946cb9ce6a15bcc036bda0f72d4b9fd" +checksum = "43e319d9de9ff4d941abf4ac718897118b0fe04577ea3f8e0f5788971784eef5" dependencies = [ "anyhow", "async-trait", @@ -8496,7 +8572,6 @@ dependencies = [ "serde_json", "sha1", "sha2", - "tokio", ] [[package]] @@ -10313,6 +10388,7 @@ dependencies = [ "pkcs1", "pkcs8 0.10.2", "rand_core", + "sha2", "signature 2.0.0", "spki 0.7.2", "subtle", @@ -10545,6 +10621,15 @@ version = "1.0.15" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1ad4cc8da4ef723ed60bced201181d83791ad433213d8c24efffda1eec85d741" +[[package]] +name = "salsa20" +version = "0.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97a22f5af31f73a954c10289c93e8a50cc23d971e80ee446f1f6f7137a088213" +dependencies = [ + "cipher", +] + [[package]] name = "same-file" version = "1.0.6" @@ -10611,6 +10696,17 @@ version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a3cf7c11c38cb994f3d40e8a8cde3bbd1f72a435e4c49e85d6553d8312306152" +[[package]] +name = "scrypt" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0516a385866c09368f0b5bcd1caff3366aace790fcd46e2bb032697bb172fd1f" +dependencies = [ + "pbkdf2", + "salsa20", + "sha2", +] + [[package]] name = "sct" version = "0.7.0" diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 7daf0883ac7e8..858492b83b82a 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -29,6 +29,7 @@ pub mod kinesis; pub mod log_store; pub mod mock_coordination_client; pub mod nats; +pub mod opendal; pub mod pulsar; pub mod redis; pub mod remote; @@ -88,6 +89,7 @@ macro_rules! for_all_sinks { { HttpJava, $crate::sink::remote::HttpJavaSink }, { Doris, $crate::sink::doris::DorisSink }, { Starrocks, $crate::sink::starrocks::StarrocksSink }, + { S3, $crate::sink::opendal::s3::S3Sink }, { DeltaLake, $crate::sink::deltalake::DeltaLakeSink }, { BigQuery, $crate::sink::big_query::BigQuerySink }, { Test, $crate::sink::test_sink::TestSink }, @@ -525,6 +527,8 @@ pub enum SinkError { ), #[error("Starrocks error: {0}")] Starrocks(String), + #[error("S3 error: {0}")] + S3(String), #[error("Pulsar error: {0}")] Pulsar( #[source] diff --git a/src/connector/src/sink/opendal/gcs.rs b/src/connector/src/sink/opendal/gcs.rs new file mode 100644 index 0000000000000..09c0891ee8434 --- /dev/null +++ b/src/connector/src/sink/opendal/gcs.rs @@ -0,0 +1,23 @@ +// Copyright 2024 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. + +// impl OpendalSinkWriter { +// /// create opendal gcs sink. +// pub fn new_gcs_sink( +// gcs_properties: gcsPropertiesCommon, +// assume_role: Option, +// ) -> Result { +// todo!(); +// } +// } diff --git a/src/connector/src/sink/opendal/mod.rs b/src/connector/src/sink/opendal/mod.rs new file mode 100644 index 0000000000000..50b3f05fc18f2 --- /dev/null +++ b/src/connector/src/sink/opendal/mod.rs @@ -0,0 +1,46 @@ +// Copyright 2024 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 s3; +// pub mod gcs; + +pub const GCS_SINK: &str = "gcs"; + +// pub trait OpendalSink: Send + Sync + 'static + Clone + PartialEq { +// type Properties: SourceProperties + Send + Sync; + +// fn new_sink_engine(properties: Self::Properties) -> ConnectorResult>; +// } + +// #[derive(Debug, Clone, Copy, PartialEq, Eq)] +// pub struct OpendalS3; + +// impl OpendalSink for OpendalS3 { +// type Properties = OpendalS3Properties; + +// fn new_sink_engine(properties: Self::Properties) -> ConnectorResult> { +// todo!() +// } +// } + +// #[derive(Debug, Clone, Copy, PartialEq, Eq)] +// pub struct OpendalGcs; + +// impl OpendalSink for OpendalGcs { +// type Properties = GcsProperties; + +// fn new_sink_engine(properties: Self::Properties) -> ConnectorResult> { +// OpendalEnumerator::new_gcs_source(properties) +// } +// } diff --git a/src/connector/src/sink/opendal/s3.rs b/src/connector/src/sink/opendal/s3.rs new file mode 100644 index 0000000000000..4f1492dbbefa8 --- /dev/null +++ b/src/connector/src/sink/opendal/s3.rs @@ -0,0 +1,174 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +use std::collections::HashMap; +use std::sync::Arc; +use opendal::{Metakey, Operator}; +use anyhow::anyhow; +use crate::sink::encoder::{JsonEncoder, RowEncoder}; +use crate::sink::writer::LogSinkerOf; +use crate::sink::{SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; +use async_trait::async_trait; +use bytes::Bytes; +use itertools::Itertools; +use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::buffer::Bitmap; +use risingwave_common::catalog::Schema; +use risingwave_common::types::DataType; +use serde::Deserialize; +use serde_derive::Serialize; +use serde_json::Value; +use serde_with::serde_as; +use thiserror_ext::AsReport; +use with_options::WithOptions; + +use crate::sink::writer::SinkWriterExt; +use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkWriter, SinkWriterParam}; + +#[derive(Deserialize, Debug, Clone, WithOptions)] +pub struct S3Common { + #[serde(rename = "s3.region_name")] + pub region_name: String, + #[serde(rename = "s3.bucket_name")] + pub bucket_name: String, + #[serde(rename = "match_pattern", default)] + pub match_pattern: Option, + #[serde(rename = "s3.credentials.access", default)] + pub access: Option, + #[serde(rename = "s3.credentials.secret", default)] + pub secret: Option, + #[serde(rename = "s3.endpoint_url")] + pub endpoint_url: Option, + #[serde(rename = "s3.assume_role", default)] + pub assume_role: Option, +} + +#[serde_as] +#[derive(Clone, Debug, Deserialize, WithOptions)] +pub struct S3Config { + #[serde(flatten)] + pub common: S3Common, + + pub r#type: String, // accept "append-only" or "upsert" +} + +pub const S3_SINK: &str = "s3"; + +impl S3Config { + pub fn from_hashmap(properties: HashMap) -> Result { + let config = + serde_json::from_value::(serde_json::to_value(properties).unwrap()) + .map_err(|e| SinkError::Config(anyhow!(e)))?; + if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { + return Err(SinkError::Config(anyhow!( + "`{}` must be {}, or {}", + SINK_TYPE_OPTION, + SINK_TYPE_APPEND_ONLY, + SINK_TYPE_UPSERT + ))); + } + Ok(config) + } +} + +#[derive(Debug)] +pub struct S3Sink { + pub config: S3Config, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, +} + +impl S3Sink { + pub fn new( + config: S3Config, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + ) -> Result { + Ok(Self { + config, + schema, + pk_indices, + is_append_only, + }) + } +} + + +impl Sink for S3Sink { + type Coordinator = DummySinkCommitCoordinator; + type LogSinker = LogSinkerOf; + + + const SINK_NAME: &'static str = S3_SINK; + + async fn validate(&self) -> Result<()> { + todo!() + } + + async fn new_log_sinker( + &self, + writer_param: crate::sink::SinkWriterParam, + ) -> Result { + todo!() + } +} + +impl TryFrom for S3Sink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let schema = param.schema(); + let config = S3Config::from_hashmap(param.properties)?; + S3Sink::new( + config, + schema, + param.downstream_pk, + param.sink_type.is_append_only(), + ) + } +} + + +pub struct S3SinkWriter { + pub config: S3Config, + schema: Schema, + op: Operator, + pk_indices: Vec, + is_append_only: bool, + row_encoder: JsonEncoder, +} + +#[async_trait] +impl SinkWriter for S3SinkWriter { + async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { + todo!() + } + + async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { + todo!() + } + + async fn abort(&mut self) -> Result<()> { + todo!() + } + + async fn barrier(&mut self, _is_checkpoint: bool) -> Result<()> { + todo!() + } + + async fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc) -> Result<()> { + todo!() + } +} From dea340aa9aef61548f2ba7add0e3937a212d79ab Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 12 Mar 2024 14:35:44 +0800 Subject: [PATCH 02/76] save work, add gcs --- src/connector/src/sink/encoder/json.rs | 32 ++++- src/connector/src/sink/encoder/mod.rs | 1 + src/connector/src/sink/mod.rs | 3 + src/connector/src/sink/opendal/gcs.rs | 170 +++++++++++++++++++++++-- src/connector/src/sink/opendal/mod.rs | 113 +++++++++++++--- src/connector/src/sink/opendal/s3.rs | 88 +++++++------ 6 files changed, 338 insertions(+), 69 deletions(-) diff --git a/src/connector/src/sink/encoder/json.rs b/src/connector/src/sink/encoder/json.rs index 64a06ff70770f..f3cae9f75fc0e 100644 --- a/src/connector/src/sink/encoder/json.rs +++ b/src/connector/src/sink/encoder/json.rs @@ -114,6 +114,23 @@ impl JsonEncoder { } } + pub fn new_with_s3( + schema: Schema, + col_indices: Option>, + map: HashMap, + ) -> Self { + Self { + schema, + col_indices, + time_handling_mode: TimeHandlingMode::Milli, + date_handling_mode: DateHandlingMode::String, + timestamp_handling_mode: TimestampHandlingMode::String, + timestamptz_handling_mode: TimestamptzHandlingMode::UtcWithoutSuffix, + custom_json_type: CustomJsonType::S3, + kafka_connect: None, + } + } + pub fn new_with_bigquery(schema: Schema, col_indices: Option>) -> Self { Self { schema, @@ -259,7 +276,10 @@ fn datum_to_json_object( } json!(v_string) } - CustomJsonType::Es | CustomJsonType::None | CustomJsonType::BigQuery => { + CustomJsonType::Es + | CustomJsonType::None + | CustomJsonType::BigQuery + | CustomJsonType::S3 => { json!(v.to_text()) } }, @@ -311,7 +331,10 @@ fn datum_to_json_object( } (DataType::Jsonb, ScalarRefImpl::Jsonb(jsonb_ref)) => match custom_json_type { CustomJsonType::Es | CustomJsonType::StarRocks(_) => JsonbVal::from(jsonb_ref).take(), - CustomJsonType::Doris(_) | CustomJsonType::None | CustomJsonType::BigQuery => { + CustomJsonType::Doris(_) + | CustomJsonType::None + | CustomJsonType::BigQuery + | CustomJsonType::S3 => { json!(jsonb_ref.to_string()) } }, @@ -362,7 +385,10 @@ fn datum_to_json_object( "starrocks can't support struct".to_string(), )); } - CustomJsonType::Es | CustomJsonType::None | CustomJsonType::BigQuery => { + CustomJsonType::Es + | CustomJsonType::None + | CustomJsonType::BigQuery + | CustomJsonType::S3 => { let mut map = Map::with_capacity(st.len()); for (sub_datum_ref, sub_field) in struct_ref.iter_fields_ref().zip_eq_debug( st.iter() diff --git a/src/connector/src/sink/encoder/mod.rs b/src/connector/src/sink/encoder/mod.rs index 3254447e27077..97b2cb1f8eb99 100644 --- a/src/connector/src/sink/encoder/mod.rs +++ b/src/connector/src/sink/encoder/mod.rs @@ -144,6 +144,7 @@ pub enum CustomJsonType { Es, // starrocks' need jsonb is struct StarRocks(HashMap), + S3, // bigquery need null array -> [] BigQuery, None, diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 858492b83b82a..1d6f5c9b0e660 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -90,6 +90,7 @@ macro_rules! for_all_sinks { { Doris, $crate::sink::doris::DorisSink }, { Starrocks, $crate::sink::starrocks::StarrocksSink }, { S3, $crate::sink::opendal::s3::S3Sink }, + { GCS, $crate::sink::opendal::gcs::GcsSink }, { DeltaLake, $crate::sink::deltalake::DeltaLakeSink }, { BigQuery, $crate::sink::big_query::BigQuerySink }, { Test, $crate::sink::test_sink::TestSink }, @@ -529,6 +530,8 @@ pub enum SinkError { Starrocks(String), #[error("S3 error: {0}")] S3(String), + #[error("Gcs error: {0}")] + GCS(String), #[error("Pulsar error: {0}")] Pulsar( #[source] diff --git a/src/connector/src/sink/opendal/gcs.rs b/src/connector/src/sink/opendal/gcs.rs index 09c0891ee8434..3c7bdfc56ba67 100644 --- a/src/connector/src/sink/opendal/gcs.rs +++ b/src/connector/src/sink/opendal/gcs.rs @@ -11,13 +11,165 @@ // 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::collections::HashMap; +use std::sync::Arc; -// impl OpendalSinkWriter { -// /// create opendal gcs sink. -// pub fn new_gcs_sink( -// gcs_properties: gcsPropertiesCommon, -// assume_role: Option, -// ) -> Result { -// todo!(); -// } -// } +use anyhow::anyhow; +use async_trait::async_trait; +use bytes::Bytes; +use itertools::Itertools; +use opendal::layers::{LoggingLayer, RetryLayer}; +use opendal::services::Gcs; +use crate::sink::opendal::OpenDalSinkWriter; +use opendal::{Metakey, Operator}; +use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::buffer::Bitmap; +use risingwave_common::catalog::Schema; +use risingwave_common::types::DataType; +use serde::Deserialize; +use serde_derive::Serialize; +use serde_json::Value; +use serde_with::serde_as; +use thiserror_ext::AsReport; +use with_options::WithOptions; + +use crate::error::ConnectorError; +use crate::sink::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; +use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; +use crate::sink::{ + DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SinkWriter, SinkWriterParam, + SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, +}; + +#[derive(Deserialize, Debug, Clone, WithOptions)] +pub struct GcsCommon { + #[serde(rename = "gcs.bucket_name")] + pub bucket_name: String, + + /// The base64 encoded credential key. If not set, ADC will be used. + #[serde(rename = "gcs.credential")] + pub credential: Option, + + /// If credential/ADC is not set. The service account can be used to provide the credential info. + #[serde(rename = "gcs.service_account", default)] + pub service_account: Option, + + #[serde(rename = "match_pattern", default)] + pub match_pattern: Option, + + #[serde(flatten)] + pub unknown_fields: HashMap, +} + +#[serde_as] +#[derive(Clone, Debug, Deserialize, WithOptions)] +pub struct GcsConfig { + #[serde(flatten)] + pub common: GcsCommon, + + pub r#type: String, // accept "append-only" or "upsert" +} + +pub const GCS_SINK: &str = "gcs"; + +impl GcsConfig { + pub fn from_hashmap(properties: HashMap) -> Result { + let config = serde_json::from_value::(serde_json::to_value(properties).unwrap()) + .map_err(|e| SinkError::Config(anyhow!(e)))?; + if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { + return Err(SinkError::Config(anyhow!( + "`{}` must be {}, or {}", + SINK_TYPE_OPTION, + SINK_TYPE_APPEND_ONLY, + SINK_TYPE_UPSERT + ))); + } + Ok(config) + } +} + +#[derive(Debug)] +pub struct GcsSink { + pub config: GcsConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, +} + +impl GcsSink { + pub fn new( + config: GcsConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + ) -> Result { + Ok(Self { + config, + schema, + pk_indices, + is_append_only, + }) + } +} + +impl Sink for GcsSink { + type Coordinator = DummySinkCommitCoordinator; + type LogSinker = LogSinkerOf; + + const SINK_NAME: &'static str = GCS_SINK; + + async fn validate(&self) -> Result<()> { + todo!() + } + + async fn new_log_sinker( + &self, + writer_param: crate::sink::SinkWriterParam, + ) -> Result { + todo!() + } +} + +impl TryFrom for GcsSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let schema = param.schema(); + let config = GcsConfig::from_hashmap(param.properties)?; + GcsSink::new( + config, + schema, + param.downstream_pk, + param.sink_type.is_append_only(), + ) + } +} + +impl OpenDalSinkWriter { + pub async fn new_gcs_sink(config: GcsConfig) -> Result { + // Create gcs builder. + let mut builder = Gcs::default(); + + builder.bucket(&config.common.bucket_name); + + // if credential env is set, use it. Otherwise, ADC will be used. + if let Some(cred) = config.common.credential { + builder.credential(&cred); + } else { + let cred = std::env::var("GOOGLE_APPLICATION_CREDENTIALS"); + if let Ok(cred) = cred { + builder.credential(&cred); + } + } + + if let Some(service_account) = config.common.service_account { + builder.service_account(&service_account); + } + let operator: Operator = Operator::new(builder) + .map_err(|e| SinkError::Connector(e.into()))? + .layer(LoggingLayer::default()) + .layer(RetryLayer::default()) + .finish(); + Ok(operator) + } +} diff --git a/src/connector/src/sink/opendal/mod.rs b/src/connector/src/sink/opendal/mod.rs index 50b3f05fc18f2..495c1673c0406 100644 --- a/src/connector/src/sink/opendal/mod.rs +++ b/src/connector/src/sink/opendal/mod.rs @@ -12,35 +12,106 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod gcs; pub mod s3; -// pub mod gcs; + +use std::collections::HashMap; +use std::sync::Arc; + +use anyhow::anyhow; +use async_trait::async_trait; +use bytes::Bytes; +use itertools::Itertools; +use opendal::{Metakey, Operator}; +use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::buffer::Bitmap; +use risingwave_common::catalog::Schema; +use risingwave_common::types::DataType; +use serde::Deserialize; +use serde_derive::Serialize; +use serde_json::Value; +use serde_with::serde_as; +use thiserror_ext::AsReport; +use with_options::WithOptions; + +use crate::error::ConnectorError; +use crate::sink::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; +use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; +use crate::sink::{ + DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SinkWriter, SinkWriterParam, + SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, +}; pub const GCS_SINK: &str = "gcs"; -// pub trait OpendalSink: Send + Sync + 'static + Clone + PartialEq { -// type Properties: SourceProperties + Send + Sync; -// fn new_sink_engine(properties: Self::Properties) -> ConnectorResult>; -// } +pub struct OpenDalSinkWriter { + schema: Schema, + op: Operator, + pk_indices: Vec, + is_append_only: bool, + row_encoder: JsonEncoder, + path: String, +} + +#[async_trait] +impl SinkWriter for OpenDalSinkWriter { + async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { + let path = &self.path.clone(); + if self.is_append_only { + self.append_only(chunk, path).await + } else { + unimplemented!() + } + } -// #[derive(Debug, Clone, Copy, PartialEq, Eq)] -// pub struct OpendalS3; + async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { + Ok(()) + } -// impl OpendalSink for OpendalS3 { -// type Properties = OpendalS3Properties; + async fn abort(&mut self) -> Result<()> { + Ok(()) + } -// fn new_sink_engine(properties: Self::Properties) -> ConnectorResult> { -// todo!() -// } -// } + async fn barrier(&mut self, _is_checkpoint: bool) -> Result<()> { + todo!() + } -// #[derive(Debug, Clone, Copy, PartialEq, Eq)] -// pub struct OpendalGcs; + async fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc) -> Result<()> { + Ok(()) + } +} -// impl OpendalSink for OpendalGcs { -// type Properties = GcsProperties; +impl OpenDalSinkWriter { + pub async fn new( + op: Operator, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + path: &str, + ) -> Result { + let mut decimal_map = HashMap::default(); + Ok(Self { + schema: schema.clone(), + pk_indices, + op, + is_append_only, + row_encoder: JsonEncoder::new_with_s3(schema, None, decimal_map), + path: path.to_string(), + }) + } -// fn new_sink_engine(properties: Self::Properties) -> ConnectorResult> { -// OpendalEnumerator::new_gcs_source(properties) -// } -// } + async fn append_only(&mut self, chunk: StreamChunk, path: &str) -> Result<()> { + for (op, row) in chunk.rows() { + if op != Op::Insert { + continue; + } + let row_json_string = Value::Object(self.row_encoder.encode(row)?).to_string(); + self.op + .write(path, row_json_string) + .await + .map_err(|e| SinkError::Connector(e.into()))?; + } + Ok(()) + } +} diff --git a/src/connector/src/sink/opendal/s3.rs b/src/connector/src/sink/opendal/s3.rs index 4f1492dbbefa8..5dc9f546e12c7 100644 --- a/src/connector/src/sink/opendal/s3.rs +++ b/src/connector/src/sink/opendal/s3.rs @@ -13,14 +13,16 @@ // limitations under the License. use std::collections::HashMap; use std::sync::Arc; -use opendal::{Metakey, Operator}; + use anyhow::anyhow; -use crate::sink::encoder::{JsonEncoder, RowEncoder}; -use crate::sink::writer::LogSinkerOf; -use crate::sink::{SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; use async_trait::async_trait; use bytes::Bytes; +use deltalake::storage::s3; +use crate::sink::opendal::OpenDalSinkWriter; use itertools::Itertools; +use opendal::layers::{LoggingLayer, RetryLayer}; +use opendal::services::S3; +use opendal::{Metakey, Operator}; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::Schema; @@ -32,8 +34,13 @@ use serde_with::serde_as; use thiserror_ext::AsReport; use with_options::WithOptions; -use crate::sink::writer::SinkWriterExt; -use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkWriter, SinkWriterParam}; +use crate::error::ConnectorError; +use crate::sink::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; +use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; +use crate::sink::{ + DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SinkWriter, SinkWriterParam, + SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, +}; #[derive(Deserialize, Debug, Clone, WithOptions)] pub struct S3Common { @@ -66,9 +73,8 @@ pub const S3_SINK: &str = "s3"; impl S3Config { pub fn from_hashmap(properties: HashMap) -> Result { - let config = - serde_json::from_value::(serde_json::to_value(properties).unwrap()) - .map_err(|e| SinkError::Config(anyhow!(e)))?; + let config = serde_json::from_value::(serde_json::to_value(properties).unwrap()) + .map_err(|e| SinkError::Config(anyhow!(e)))?; if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { return Err(SinkError::Config(anyhow!( "`{}` must be {}, or {}", @@ -105,12 +111,10 @@ impl S3Sink { } } - impl Sink for S3Sink { type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf; + type LogSinker = LogSinkerOf; - const SINK_NAME: &'static str = S3_SINK; async fn validate(&self) -> Result<()> { @@ -140,35 +144,47 @@ impl TryFrom for S3Sink { } } +impl OpenDalSinkWriter { + pub async fn new_s3_sink(config: S3Config) -> Result { + // Create s3 builder. + let mut builder = S3::default(); + builder.bucket(&config.common.bucket_name); + builder.region(&config.common.region_name); -pub struct S3SinkWriter { - pub config: S3Config, - schema: Schema, - op: Operator, - pk_indices: Vec, - is_append_only: bool, - row_encoder: JsonEncoder, -} + if let Some(endpoint_url) = config.common.endpoint_url { + builder.endpoint(&endpoint_url); + } -#[async_trait] -impl SinkWriter for S3SinkWriter { - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { - todo!() - } + if let Some(access) = config.common.access { + builder.access_key_id(&access); + } else { + tracing::error!( + "access key id of aws s3 is not set, bucket {}", + config.common.bucket_name + ); + } - async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { - todo!() - } + if let Some(secret) = config.common.secret { + builder.secret_access_key(&secret); + } else { + tracing::error!( + "secret access key of aws s3 is not set, bucket {}", + config.common.bucket_name + ); + } - async fn abort(&mut self) -> Result<()> { - todo!() - } + builder.enable_virtual_host_style(); - async fn barrier(&mut self, _is_checkpoint: bool) -> Result<()> { - todo!() - } + if let Some(assume_role) = config.common.assume_role { + builder.role_arn(&assume_role); + } - async fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc) -> Result<()> { - todo!() + let operator: Operator = Operator::new(builder) + .map_err(|e| SinkError::Connector(e.into()))? + .layer(LoggingLayer::default()) + .layer(RetryLayer::default()) + .finish(); + + Ok(operator) } } From 22fe512724fa018379d1f9915cc491e830e3573b Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 12 Mar 2024 15:51:45 +0800 Subject: [PATCH 03/76] implement sink writer --- src/connector/Cargo.toml | 2 +- src/connector/src/sink/encoder/json.rs | 2 +- src/connector/src/sink/mod.rs | 19 +-- .../src/sink/{opendal => opendal_sink}/gcs.rs | 101 ++++++++-------- .../src/sink/{opendal => opendal_sink}/mod.rs | 59 ++++----- .../src/sink/{opendal => opendal_sink}/s3.rs | 114 +++++++++--------- 6 files changed, 149 insertions(+), 148 deletions(-) rename src/connector/src/sink/{opendal => opendal_sink}/gcs.rs (82%) rename src/connector/src/sink/{opendal => opendal_sink}/mod.rs (62%) rename src/connector/src/sink/{opendal => opendal_sink}/s3.rs (83%) diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 77f2c1374dc92..65ead4ce3230a 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -82,7 +82,7 @@ mysql_common = { version = "0.31", default-features = false, features = [ ] } nexmark = { version = "0.2", features = ["serde"] } num-bigint = "0.4" -opendal = "0.44" +opendal = "0.44.2" parking_lot = "0.12" paste = "1" prometheus = { version = "0.13", features = ["process"] } diff --git a/src/connector/src/sink/encoder/json.rs b/src/connector/src/sink/encoder/json.rs index f3cae9f75fc0e..13fce6f5cfe75 100644 --- a/src/connector/src/sink/encoder/json.rs +++ b/src/connector/src/sink/encoder/json.rs @@ -117,7 +117,7 @@ impl JsonEncoder { pub fn new_with_s3( schema: Schema, col_indices: Option>, - map: HashMap, + _map: HashMap, ) -> Self { Self { schema, diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 1d6f5c9b0e660..0137bf5dc7b82 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -29,7 +29,7 @@ pub mod kinesis; pub mod log_store; pub mod mock_coordination_client; pub mod nats; -pub mod opendal; +pub mod opendal_sink; pub mod pulsar; pub mod redis; pub mod remote; @@ -47,6 +47,7 @@ use ::deltalake::DeltaTableError; use ::redis::RedisError; use anyhow::anyhow; use async_trait::async_trait; +use opendal::Error as OpendalError; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::{ColumnDesc, Field, Schema}; use risingwave_common::metrics::{ @@ -89,8 +90,8 @@ macro_rules! for_all_sinks { { HttpJava, $crate::sink::remote::HttpJavaSink }, { Doris, $crate::sink::doris::DorisSink }, { Starrocks, $crate::sink::starrocks::StarrocksSink }, - { S3, $crate::sink::opendal::s3::S3Sink }, - { GCS, $crate::sink::opendal::gcs::GcsSink }, + { S3, $crate::sink::opendal_sink::s3::S3Sink }, + { GCS, $crate::sink::opendal_sink::gcs::GcsSink }, { DeltaLake, $crate::sink::deltalake::DeltaLakeSink }, { BigQuery, $crate::sink::big_query::BigQuerySink }, { Test, $crate::sink::test_sink::TestSink }, @@ -528,10 +529,8 @@ pub enum SinkError { ), #[error("Starrocks error: {0}")] Starrocks(String), - #[error("S3 error: {0}")] - S3(String), - #[error("Gcs error: {0}")] - GCS(String), + #[error("Opendal error: {0}")] + Opendal(String), #[error("Pulsar error: {0}")] Pulsar( #[source] @@ -564,6 +563,12 @@ impl From for SinkError { } } +impl From for SinkError { + fn from(error: OpendalError) -> Self { + SinkError::Opendal(error.to_string()) + } +} + impl From for SinkError { fn from(value: RpcError) -> Self { SinkError::Remote(anyhow!(value)) diff --git a/src/connector/src/sink/opendal/gcs.rs b/src/connector/src/sink/opendal_sink/gcs.rs similarity index 82% rename from src/connector/src/sink/opendal/gcs.rs rename to src/connector/src/sink/opendal_sink/gcs.rs index 3c7bdfc56ba67..03b89d1fcfdf3 100644 --- a/src/connector/src/sink/opendal/gcs.rs +++ b/src/connector/src/sink/opendal_sink/gcs.rs @@ -12,35 +12,25 @@ // See the License for the specific language governing permissions and // limitations under the License. use std::collections::HashMap; -use std::sync::Arc; use anyhow::anyhow; -use async_trait::async_trait; -use bytes::Bytes; -use itertools::Itertools; use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::Gcs; -use crate::sink::opendal::OpenDalSinkWriter; -use opendal::{Metakey, Operator}; -use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::buffer::Bitmap; +use opendal::Operator; use risingwave_common::catalog::Schema; -use risingwave_common::types::DataType; use serde::Deserialize; -use serde_derive::Serialize; -use serde_json::Value; use serde_with::serde_as; -use thiserror_ext::AsReport; use with_options::WithOptions; -use crate::error::ConnectorError; -use crate::sink::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; +use crate::sink::encoder::RowEncoder; +use crate::sink::opendal_sink::OpenDalSinkWriter; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{ - DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SinkWriter, SinkWriterParam, - SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, + DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, + SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; +const GCS_WRITE_BUFFER_SIZE: usize = 16 * 1024 * 1024; #[derive(Deserialize, Debug, Clone, WithOptions)] pub struct GcsCommon { #[serde(rename = "gcs.bucket_name")] @@ -54,8 +44,8 @@ pub struct GcsCommon { #[serde(rename = "gcs.service_account", default)] pub service_account: Option, - #[serde(rename = "match_pattern", default)] - pub match_pattern: Option, + #[serde(rename = "gcs.path", default)] + pub path: String, #[serde(flatten)] pub unknown_fields: HashMap, @@ -112,6 +102,34 @@ impl GcsSink { } } +impl GcsSink { + pub async fn new_gcs_sink(config: GcsConfig) -> Result { + // Create gcs builder. + let mut builder = Gcs::default(); + + builder.bucket(&config.common.bucket_name); + + // if credential env is set, use it. Otherwise, ADC will be used. + if let Some(cred) = config.common.credential { + builder.credential(&cred); + } else { + let cred = std::env::var("GOOGLE_APPLICATION_CREDENTIALS"); + if let Ok(cred) = cred { + builder.credential(&cred); + } + } + + if let Some(service_account) = config.common.service_account { + builder.service_account(&service_account); + } + let operator: Operator = Operator::new(builder)? + .layer(LoggingLayer::default()) + .layer(RetryLayer::default()) + .finish(); + Ok(operator) + } +} + impl Sink for GcsSink { type Coordinator = DummySinkCommitCoordinator; type LogSinker = LogSinkerOf; @@ -119,14 +137,28 @@ impl Sink for GcsSink { const SINK_NAME: &'static str = GCS_SINK; async fn validate(&self) -> Result<()> { - todo!() + let op = Self::new_gcs_sink(self.config.clone()).await?; + Ok(()) } async fn new_log_sinker( &self, writer_param: crate::sink::SinkWriterParam, ) -> Result { - todo!() + let op = Self::new_gcs_sink(self.config.clone()).await?; + let path = self.config.common.path.as_ref(); + let writer = op + .writer_with(&path) + .concurrent(8) + .buffer(GCS_WRITE_BUFFER_SIZE) + .await?; + Ok(OpenDalSinkWriter::new( + writer, + self.schema.clone(), + self.pk_indices.clone(), + self.is_append_only, + )? + .into_log_sinker(writer_param.sink_metrics)) } } @@ -144,32 +176,3 @@ impl TryFrom for GcsSink { ) } } - -impl OpenDalSinkWriter { - pub async fn new_gcs_sink(config: GcsConfig) -> Result { - // Create gcs builder. - let mut builder = Gcs::default(); - - builder.bucket(&config.common.bucket_name); - - // if credential env is set, use it. Otherwise, ADC will be used. - if let Some(cred) = config.common.credential { - builder.credential(&cred); - } else { - let cred = std::env::var("GOOGLE_APPLICATION_CREDENTIALS"); - if let Ok(cred) = cred { - builder.credential(&cred); - } - } - - if let Some(service_account) = config.common.service_account { - builder.service_account(&service_account); - } - let operator: Operator = Operator::new(builder) - .map_err(|e| SinkError::Connector(e.into()))? - .layer(LoggingLayer::default()) - .layer(RetryLayer::default()) - .finish(); - Ok(operator) - } -} diff --git a/src/connector/src/sink/opendal/mod.rs b/src/connector/src/sink/opendal_sink/mod.rs similarity index 62% rename from src/connector/src/sink/opendal/mod.rs rename to src/connector/src/sink/opendal_sink/mod.rs index 495c1673c0406..2c20d1fe53da9 100644 --- a/src/connector/src/sink/opendal/mod.rs +++ b/src/connector/src/sink/opendal_sink/mod.rs @@ -18,48 +18,31 @@ pub mod s3; use std::collections::HashMap; use std::sync::Arc; -use anyhow::anyhow; use async_trait::async_trait; -use bytes::Bytes; -use itertools::Itertools; -use opendal::{Metakey, Operator}; +use opendal::{Operator, Writer}; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::Schema; -use risingwave_common::types::DataType; -use serde::Deserialize; -use serde_derive::Serialize; use serde_json::Value; -use serde_with::serde_as; -use thiserror_ext::AsReport; -use with_options::WithOptions; -use crate::error::ConnectorError; -use crate::sink::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; -use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; -use crate::sink::{ - DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SinkWriter, SinkWriterParam, - SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, -}; +use crate::sink::encoder::{JsonEncoder, RowEncoder}; +use crate::sink::{Result, SinkError, SinkWriter}; pub const GCS_SINK: &str = "gcs"; - pub struct OpenDalSinkWriter { schema: Schema, - op: Operator, + writer: Writer, pk_indices: Vec, is_append_only: bool, row_encoder: JsonEncoder, - path: String, } #[async_trait] impl SinkWriter for OpenDalSinkWriter { async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { - let path = &self.path.clone(); if self.is_append_only { - self.append_only(chunk, path).await + self.append_only(chunk).await } else { unimplemented!() } @@ -70,11 +53,22 @@ impl SinkWriter for OpenDalSinkWriter { } async fn abort(&mut self) -> Result<()> { + self.writer.abort().await?; Ok(()) } - async fn barrier(&mut self, _is_checkpoint: bool) -> Result<()> { - todo!() + async fn barrier(&mut self, is_checkpoint: bool) -> Result<()> { + if is_checkpoint { + match self.writer.close().await { + Ok(_) => (), + Err(err) => { + self.writer.abort().await?; + return Err(err.into()); + } + }; + } + + Ok(()) } async fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc) -> Result<()> { @@ -83,34 +77,29 @@ impl SinkWriter for OpenDalSinkWriter { } impl OpenDalSinkWriter { - pub async fn new( - op: Operator, + pub fn new( + writer: Writer, schema: Schema, pk_indices: Vec, is_append_only: bool, - path: &str, ) -> Result { - let mut decimal_map = HashMap::default(); + let decimal_map = HashMap::default(); Ok(Self { schema: schema.clone(), pk_indices, - op, + writer, is_append_only, row_encoder: JsonEncoder::new_with_s3(schema, None, decimal_map), - path: path.to_string(), }) } - async fn append_only(&mut self, chunk: StreamChunk, path: &str) -> Result<()> { + async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { for (op, row) in chunk.rows() { if op != Op::Insert { continue; } let row_json_string = Value::Object(self.row_encoder.encode(row)?).to_string(); - self.op - .write(path, row_json_string) - .await - .map_err(|e| SinkError::Connector(e.into()))?; + self.writer.write(row_json_string).await?; } Ok(()) } diff --git a/src/connector/src/sink/opendal/s3.rs b/src/connector/src/sink/opendal_sink/s3.rs similarity index 83% rename from src/connector/src/sink/opendal/s3.rs rename to src/connector/src/sink/opendal_sink/s3.rs index 5dc9f546e12c7..b69cb484b6e07 100644 --- a/src/connector/src/sink/opendal/s3.rs +++ b/src/connector/src/sink/opendal_sink/s3.rs @@ -12,44 +12,34 @@ // See the License for the specific language governing permissions and // limitations under the License. use std::collections::HashMap; -use std::sync::Arc; use anyhow::anyhow; -use async_trait::async_trait; -use bytes::Bytes; -use deltalake::storage::s3; -use crate::sink::opendal::OpenDalSinkWriter; -use itertools::Itertools; use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::S3; -use opendal::{Metakey, Operator}; -use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::buffer::Bitmap; +use opendal::Operator; use risingwave_common::catalog::Schema; -use risingwave_common::types::DataType; use serde::Deserialize; -use serde_derive::Serialize; -use serde_json::Value; use serde_with::serde_as; -use thiserror_ext::AsReport; use with_options::WithOptions; -use crate::error::ConnectorError; -use crate::sink::encoder::{JsonEncoder, RowEncoder, TimestampHandlingMode}; +use crate::sink::encoder::RowEncoder; +use crate::sink::opendal_sink::OpenDalSinkWriter; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{ - DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SinkWriter, SinkWriterParam, - SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, + DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, + SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; +const S3_WRITE_BUFFER_SIZE: usize = 16 * 1024 * 1024; + #[derive(Deserialize, Debug, Clone, WithOptions)] pub struct S3Common { #[serde(rename = "s3.region_name")] pub region_name: String, #[serde(rename = "s3.bucket_name")] pub bucket_name: String, - #[serde(rename = "match_pattern", default)] - pub match_pattern: Option, + #[serde(rename = "s3.path", default)] + pub path: String, #[serde(rename = "s3.credentials.access", default)] pub access: Option, #[serde(rename = "s3.credentials.secret", default)] @@ -58,6 +48,7 @@ pub struct S3Common { pub endpoint_url: Option, #[serde(rename = "s3.assume_role", default)] pub assume_role: Option, + } #[serde_as] @@ -111,40 +102,7 @@ impl S3Sink { } } -impl Sink for S3Sink { - type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf; - - const SINK_NAME: &'static str = S3_SINK; - - async fn validate(&self) -> Result<()> { - todo!() - } - - async fn new_log_sinker( - &self, - writer_param: crate::sink::SinkWriterParam, - ) -> Result { - todo!() - } -} - -impl TryFrom for S3Sink { - type Error = SinkError; - - fn try_from(param: SinkParam) -> std::result::Result { - let schema = param.schema(); - let config = S3Config::from_hashmap(param.properties)?; - S3Sink::new( - config, - schema, - param.downstream_pk, - param.sink_type.is_append_only(), - ) - } -} - -impl OpenDalSinkWriter { +impl S3Sink { pub async fn new_s3_sink(config: S3Config) -> Result { // Create s3 builder. let mut builder = S3::default(); @@ -179,8 +137,7 @@ impl OpenDalSinkWriter { builder.role_arn(&assume_role); } - let operator: Operator = Operator::new(builder) - .map_err(|e| SinkError::Connector(e.into()))? + let operator: Operator = Operator::new(builder)? .layer(LoggingLayer::default()) .layer(RetryLayer::default()) .finish(); @@ -188,3 +145,50 @@ impl OpenDalSinkWriter { Ok(operator) } } + +impl Sink for S3Sink { + type Coordinator = DummySinkCommitCoordinator; + type LogSinker = LogSinkerOf; + + const SINK_NAME: &'static str = S3_SINK; + + async fn validate(&self) -> Result<()> { + let op = Self::new_s3_sink(self.config.clone()).await?; + Ok(()) + } + + async fn new_log_sinker( + &self, + writer_param: crate::sink::SinkWriterParam, + ) -> Result { + let op = Self::new_s3_sink(self.config.clone()).await?; + let path = self.config.common.path.as_ref(); + let writer = op + .writer_with(&path) + .concurrent(8) + .buffer(S3_WRITE_BUFFER_SIZE) + .await?; + Ok(OpenDalSinkWriter::new( + writer, + self.schema.clone(), + self.pk_indices.clone(), + self.is_append_only, + )? + .into_log_sinker(writer_param.sink_metrics)) + } +} + +impl TryFrom for S3Sink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let schema = param.schema(); + let config = S3Config::from_hashmap(param.properties)?; + S3Sink::new( + config, + schema, + param.downstream_pk, + param.sink_type.is_append_only(), + ) + } +} From 0b471174de2d2934de4c1dd77e5d6edd358f9137 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 12 Mar 2024 16:01:35 +0800 Subject: [PATCH 04/76] make clippy happy --- src/connector/src/sink/mod.rs | 2 +- src/connector/src/sink/opendal_sink/gcs.rs | 9 ++++----- src/connector/src/sink/opendal_sink/mod.rs | 4 ++-- src/connector/src/sink/opendal_sink/s3.rs | 10 ++++------ 4 files changed, 11 insertions(+), 14 deletions(-) diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 0137bf5dc7b82..5fd3d57d07423 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -91,7 +91,7 @@ macro_rules! for_all_sinks { { Doris, $crate::sink::doris::DorisSink }, { Starrocks, $crate::sink::starrocks::StarrocksSink }, { S3, $crate::sink::opendal_sink::s3::S3Sink }, - { GCS, $crate::sink::opendal_sink::gcs::GcsSink }, + { Gcs, $crate::sink::opendal_sink::gcs::GcsSink }, { DeltaLake, $crate::sink::deltalake::DeltaLakeSink }, { BigQuery, $crate::sink::big_query::BigQuerySink }, { Test, $crate::sink::test_sink::TestSink }, diff --git a/src/connector/src/sink/opendal_sink/gcs.rs b/src/connector/src/sink/opendal_sink/gcs.rs index 03b89d1fcfdf3..360e9bdce8483 100644 --- a/src/connector/src/sink/opendal_sink/gcs.rs +++ b/src/connector/src/sink/opendal_sink/gcs.rs @@ -22,7 +22,6 @@ use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; -use crate::sink::encoder::RowEncoder; use crate::sink::opendal_sink::OpenDalSinkWriter; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{ @@ -103,7 +102,7 @@ impl GcsSink { } impl GcsSink { - pub async fn new_gcs_sink(config: GcsConfig) -> Result { + pub fn new_gcs_sink(config: GcsConfig) -> Result { // Create gcs builder. let mut builder = Gcs::default(); @@ -137,7 +136,7 @@ impl Sink for GcsSink { const SINK_NAME: &'static str = GCS_SINK; async fn validate(&self) -> Result<()> { - let op = Self::new_gcs_sink(self.config.clone()).await?; + let _op = Self::new_gcs_sink(self.config.clone())?; Ok(()) } @@ -145,10 +144,10 @@ impl Sink for GcsSink { &self, writer_param: crate::sink::SinkWriterParam, ) -> Result { - let op = Self::new_gcs_sink(self.config.clone()).await?; + let op = Self::new_gcs_sink(self.config.clone())?; let path = self.config.common.path.as_ref(); let writer = op - .writer_with(&path) + .writer_with(path) .concurrent(8) .buffer(GCS_WRITE_BUFFER_SIZE) .await?; diff --git a/src/connector/src/sink/opendal_sink/mod.rs b/src/connector/src/sink/opendal_sink/mod.rs index 2c20d1fe53da9..0b68b0956a780 100644 --- a/src/connector/src/sink/opendal_sink/mod.rs +++ b/src/connector/src/sink/opendal_sink/mod.rs @@ -19,14 +19,14 @@ use std::collections::HashMap; use std::sync::Arc; use async_trait::async_trait; -use opendal::{Operator, Writer}; +use opendal::Writer; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::Schema; use serde_json::Value; use crate::sink::encoder::{JsonEncoder, RowEncoder}; -use crate::sink::{Result, SinkError, SinkWriter}; +use crate::sink::{Result, SinkWriter}; pub const GCS_SINK: &str = "gcs"; diff --git a/src/connector/src/sink/opendal_sink/s3.rs b/src/connector/src/sink/opendal_sink/s3.rs index b69cb484b6e07..7b0f2422f14c7 100644 --- a/src/connector/src/sink/opendal_sink/s3.rs +++ b/src/connector/src/sink/opendal_sink/s3.rs @@ -22,7 +22,6 @@ use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; -use crate::sink::encoder::RowEncoder; use crate::sink::opendal_sink::OpenDalSinkWriter; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{ @@ -48,7 +47,6 @@ pub struct S3Common { pub endpoint_url: Option, #[serde(rename = "s3.assume_role", default)] pub assume_role: Option, - } #[serde_as] @@ -103,7 +101,7 @@ impl S3Sink { } impl S3Sink { - pub async fn new_s3_sink(config: S3Config) -> Result { + pub fn new_s3_sink(config: S3Config) -> Result { // Create s3 builder. let mut builder = S3::default(); builder.bucket(&config.common.bucket_name); @@ -153,7 +151,7 @@ impl Sink for S3Sink { const SINK_NAME: &'static str = S3_SINK; async fn validate(&self) -> Result<()> { - let op = Self::new_s3_sink(self.config.clone()).await?; + let _op = Self::new_s3_sink(self.config.clone())?; Ok(()) } @@ -161,10 +159,10 @@ impl Sink for S3Sink { &self, writer_param: crate::sink::SinkWriterParam, ) -> Result { - let op = Self::new_s3_sink(self.config.clone()).await?; + let op = Self::new_s3_sink(self.config.clone())?; let path = self.config.common.path.as_ref(); let writer = op - .writer_with(&path) + .writer_with(path) .concurrent(8) .buffer(S3_WRITE_BUFFER_SIZE) .await?; From ad6f3a34986d301b4da32075501a768d4e03a65f Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 13 Mar 2024 17:31:52 +0800 Subject: [PATCH 05/76] save work, add parquet writer --- Cargo.lock | 1 + src/connector/Cargo.toml | 1 + src/connector/src/sink/mod.rs | 13 ++++++ src/connector/src/sink/opendal_sink/gcs.rs | 18 +++++++-- src/connector/src/sink/opendal_sink/mod.rs | 46 +++++++++++----------- src/connector/src/sink/opendal_sink/s3.rs | 19 +++++++-- 6 files changed, 67 insertions(+), 31 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 928de9d3f645e..859cc20c87c7f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -9228,6 +9228,7 @@ dependencies = [ "num-bigint", "opendal", "parking_lot 0.12.1", + "parquet 50.0.0", "paste", "pretty_assertions", "prometheus", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 65ead4ce3230a..b1febda7078f5 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -84,6 +84,7 @@ nexmark = { version = "0.2", features = ["serde"] } num-bigint = "0.4" opendal = "0.44.2" parking_lot = "0.12" +parquet = "50.0.0" paste = "1" prometheus = { version = "0.13", features = ["process"] } prost = { version = "0.12", features = ["no-recursion-limit"] } diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 5fd3d57d07423..0d8975683d00f 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -48,6 +48,7 @@ use ::redis::RedisError; use anyhow::anyhow; use async_trait::async_trait; use opendal::Error as OpendalError; +use risingwave_common::array::ArrayError; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::{ColumnDesc, Field, Schema}; use risingwave_common::metrics::{ @@ -569,6 +570,18 @@ impl From for SinkError { } } +impl From for SinkError { + fn from(error: parquet::errors::ParquetError) -> Self { + SinkError::Opendal(error.to_string()) + } +} + +impl From for SinkError { + fn from(error: ArrayError) -> Self { + SinkError::Opendal(error.to_string()) + } +} + impl From for SinkError { fn from(value: RpcError) -> Self { SinkError::Remote(anyhow!(value)) diff --git a/src/connector/src/sink/opendal_sink/gcs.rs b/src/connector/src/sink/opendal_sink/gcs.rs index 360e9bdce8483..0dd2d0479f2e9 100644 --- a/src/connector/src/sink/opendal_sink/gcs.rs +++ b/src/connector/src/sink/opendal_sink/gcs.rs @@ -12,17 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. use std::collections::HashMap; +use std::sync::Arc; use anyhow::anyhow; use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::Gcs; -use opendal::Operator; +use opendal::{Operator, Writer as OpendalWriter}; +use parquet::arrow::async_writer::AsyncArrowWriter; use risingwave_common::catalog::Schema; use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; -use crate::sink::opendal_sink::OpenDalSinkWriter; +use crate::sink::opendal_sink::{change_schema_to_arrow_schema, OpenDalSinkWriter}; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{ DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, @@ -146,13 +148,21 @@ impl Sink for GcsSink { ) -> Result { let op = Self::new_gcs_sink(self.config.clone())?; let path = self.config.common.path.as_ref(); - let writer = op + let gcs_writer = op .writer_with(path) .concurrent(8) .buffer(GCS_WRITE_BUFFER_SIZE) .await?; + + let arrow_schema = change_schema_to_arrow_schema(self.schema.clone()); + let sink_writer: AsyncArrowWriter = AsyncArrowWriter::try_new( + gcs_writer, + Arc::new(arrow_schema), + GCS_WRITE_BUFFER_SIZE, + None, + )?; Ok(OpenDalSinkWriter::new( - writer, + sink_writer, self.schema.clone(), self.pk_indices.clone(), self.is_append_only, diff --git a/src/connector/src/sink/opendal_sink/mod.rs b/src/connector/src/sink/opendal_sink/mod.rs index 0b68b0956a780..f06b5bbca0a3e 100644 --- a/src/connector/src/sink/opendal_sink/mod.rs +++ b/src/connector/src/sink/opendal_sink/mod.rs @@ -14,13 +14,16 @@ pub mod gcs; pub mod s3; - use std::collections::HashMap; use std::sync::Arc; +use arrow_schema::{ + DataType as ArrowDataType, Field as ArrowField, Fields, Schema as ArrowSchema, SchemaRef, +}; use async_trait::async_trait; -use opendal::Writer; -use risingwave_common::array::{Op, StreamChunk}; +use opendal::Writer as OpendalWriter; +use parquet::arrow::async_writer::AsyncArrowWriter; +use risingwave_common::array::{to_record_batch_with_schema, Op, StreamChunk}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::Schema; use serde_json::Value; @@ -32,10 +35,9 @@ pub const GCS_SINK: &str = "gcs"; pub struct OpenDalSinkWriter { schema: Schema, - writer: Writer, + writer: AsyncArrowWriter, pk_indices: Vec, is_append_only: bool, - row_encoder: JsonEncoder, } #[async_trait] @@ -53,19 +55,12 @@ impl SinkWriter for OpenDalSinkWriter { } async fn abort(&mut self) -> Result<()> { - self.writer.abort().await?; Ok(()) } async fn barrier(&mut self, is_checkpoint: bool) -> Result<()> { if is_checkpoint { - match self.writer.close().await { - Ok(_) => (), - Err(err) => { - self.writer.abort().await?; - return Err(err.into()); - } - }; + todo!() } Ok(()) @@ -78,29 +73,34 @@ impl SinkWriter for OpenDalSinkWriter { impl OpenDalSinkWriter { pub fn new( - writer: Writer, + writer: AsyncArrowWriter, schema: Schema, pk_indices: Vec, is_append_only: bool, ) -> Result { - let decimal_map = HashMap::default(); Ok(Self { schema: schema.clone(), pk_indices, writer, is_append_only, - row_encoder: JsonEncoder::new_with_s3(schema, None, decimal_map), }) } async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { - for (op, row) in chunk.rows() { - if op != Op::Insert { - continue; - } - let row_json_string = Value::Object(self.row_encoder.encode(row)?).to_string(); - self.writer.write(row_json_string).await?; - } + let (mut chunk, ops) = chunk.compact().into_parts(); + let filters = + chunk.visibility() & ops.iter().map(|op| *op == Op::Insert).collect::(); + chunk.set_visibility(filters); + let arrow_schema = change_schema_to_arrow_schema(self.schema.clone()); + let batch = to_record_batch_with_schema(Arc::new(arrow_schema), &chunk.compact())?; + self.writer.write(&batch).await?; + Ok(()) } } + +fn change_schema_to_arrow_schema( + schema: risingwave_common::catalog::Schema, +) -> arrow_schema::Schema { + todo!() +} diff --git a/src/connector/src/sink/opendal_sink/s3.rs b/src/connector/src/sink/opendal_sink/s3.rs index 7b0f2422f14c7..ec60ef2ab16c8 100644 --- a/src/connector/src/sink/opendal_sink/s3.rs +++ b/src/connector/src/sink/opendal_sink/s3.rs @@ -12,17 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. use std::collections::HashMap; +use std::sync::Arc; use anyhow::anyhow; use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::S3; -use opendal::Operator; +use opendal::{Operator, Writer as OpendalWriter}; +use parquet::arrow::async_writer::AsyncArrowWriter; use risingwave_common::catalog::Schema; use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; -use crate::sink::opendal_sink::OpenDalSinkWriter; +use crate::sink::opendal_sink::{change_schema_to_arrow_schema, OpenDalSinkWriter}; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{ DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, @@ -161,13 +163,22 @@ impl Sink for S3Sink { ) -> Result { let op = Self::new_s3_sink(self.config.clone())?; let path = self.config.common.path.as_ref(); - let writer = op + let s3_writer = op .writer_with(path) .concurrent(8) .buffer(S3_WRITE_BUFFER_SIZE) .await?; + + let arrow_schema = change_schema_to_arrow_schema(self.schema.clone()); + let sink_writer: AsyncArrowWriter = AsyncArrowWriter::try_new( + s3_writer, + Arc::new(arrow_schema), + S3_WRITE_BUFFER_SIZE, + None, + )?; + Ok(OpenDalSinkWriter::new( - writer, + sink_writer, self.schema.clone(), self.pk_indices.clone(), self.is_append_only, From 08d05f071b0665cc2b2984b89f5d39ffd87c59a4 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 13 Mar 2024 17:33:14 +0800 Subject: [PATCH 06/76] minor --- src/connector/src/sink/opendal_sink/mod.rs | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/src/connector/src/sink/opendal_sink/mod.rs b/src/connector/src/sink/opendal_sink/mod.rs index f06b5bbca0a3e..5a9099ce33ccb 100644 --- a/src/connector/src/sink/opendal_sink/mod.rs +++ b/src/connector/src/sink/opendal_sink/mod.rs @@ -14,21 +14,19 @@ pub mod gcs; pub mod s3; -use std::collections::HashMap; + use std::sync::Arc; -use arrow_schema::{ - DataType as ArrowDataType, Field as ArrowField, Fields, Schema as ArrowSchema, SchemaRef, -}; + use async_trait::async_trait; use opendal::Writer as OpendalWriter; use parquet::arrow::async_writer::AsyncArrowWriter; use risingwave_common::array::{to_record_batch_with_schema, Op, StreamChunk}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::Schema; -use serde_json::Value; -use crate::sink::encoder::{JsonEncoder, RowEncoder}; + +use crate::sink::encoder::{RowEncoder}; use crate::sink::{Result, SinkWriter}; pub const GCS_SINK: &str = "gcs"; @@ -100,7 +98,7 @@ impl OpenDalSinkWriter { } fn change_schema_to_arrow_schema( - schema: risingwave_common::catalog::Schema, + _schema: risingwave_common::catalog::Schema, ) -> arrow_schema::Schema { todo!() } From f4618c10d2a2abaa6ad18e1e8d7635e6d950b2ff Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 14 Mar 2024 16:14:16 +0800 Subject: [PATCH 07/76] add parquet writer, todo: add e2e test and comments --- src/connector/src/sink/opendal_sink/gcs.rs | 23 +---- src/connector/src/sink/opendal_sink/mod.rs | 107 ++++++++++++++++----- src/connector/src/sink/opendal_sink/s3.rs | 25 +---- 3 files changed, 94 insertions(+), 61 deletions(-) diff --git a/src/connector/src/sink/opendal_sink/gcs.rs b/src/connector/src/sink/opendal_sink/gcs.rs index 0dd2d0479f2e9..2715fde48a0ae 100644 --- a/src/connector/src/sink/opendal_sink/gcs.rs +++ b/src/connector/src/sink/opendal_sink/gcs.rs @@ -12,19 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. use std::collections::HashMap; -use std::sync::Arc; use anyhow::anyhow; use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::Gcs; -use opendal::{Operator, Writer as OpendalWriter}; -use parquet::arrow::async_writer::AsyncArrowWriter; +use opendal::Operator; use risingwave_common::catalog::Schema; use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; -use crate::sink::opendal_sink::{change_schema_to_arrow_schema, OpenDalSinkWriter}; +use crate::sink::opendal_sink::OpenDalSinkWriter; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{ DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, @@ -148,21 +146,10 @@ impl Sink for GcsSink { ) -> Result { let op = Self::new_gcs_sink(self.config.clone())?; let path = self.config.common.path.as_ref(); - let gcs_writer = op - .writer_with(path) - .concurrent(8) - .buffer(GCS_WRITE_BUFFER_SIZE) - .await?; - - let arrow_schema = change_schema_to_arrow_schema(self.schema.clone()); - let sink_writer: AsyncArrowWriter = AsyncArrowWriter::try_new( - gcs_writer, - Arc::new(arrow_schema), - GCS_WRITE_BUFFER_SIZE, - None, - )?; + Ok(OpenDalSinkWriter::new( - sink_writer, + op, + path, self.schema.clone(), self.pk_indices.clone(), self.is_append_only, diff --git a/src/connector/src/sink/opendal_sink/mod.rs b/src/connector/src/sink/opendal_sink/mod.rs index 5a9099ce33ccb..8415d9de84c70 100644 --- a/src/connector/src/sink/opendal_sink/mod.rs +++ b/src/connector/src/sink/opendal_sink/mod.rs @@ -14,33 +14,39 @@ pub mod gcs; pub mod s3; - +use std::collections::HashMap; use std::sync::Arc; - +use anyhow::anyhow; +use arrow_schema::{DataType as ArrowDataType, Field as ArrowField, SchemaRef}; use async_trait::async_trait; -use opendal::Writer as OpendalWriter; -use parquet::arrow::async_writer::AsyncArrowWriter; +use icelake::config::ParquetWriterConfig; +use opendal::{Operator, Writer as OpendalWriter}; +use parquet::arrow::AsyncArrowWriter; +use parquet::file::properties::{WriterProperties, WriterVersion}; use risingwave_common::array::{to_record_batch_with_schema, Op, StreamChunk}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::Schema; +use crate::sink::{Result, SinkError, SinkWriter}; -use crate::sink::encoder::{RowEncoder}; -use crate::sink::{Result, SinkWriter}; - -pub const GCS_SINK: &str = "gcs"; +const SINK_WRITE_BUFFER_SIZE: usize = 16 * 1024 * 1024; pub struct OpenDalSinkWriter { - schema: Schema, - writer: AsyncArrowWriter, + schema: SchemaRef, + operator: Operator, + sink_writer: Option>, pk_indices: Vec, is_append_only: bool, + write_path: String, } #[async_trait] impl SinkWriter for OpenDalSinkWriter { async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { + if self.sink_writer.is_none() { + self.create_sink_writer().await?; + } if self.is_append_only { self.append_only(chunk).await } else { @@ -58,7 +64,11 @@ impl SinkWriter for OpenDalSinkWriter { async fn barrier(&mut self, is_checkpoint: bool) -> Result<()> { if is_checkpoint { - todo!() + let sink_writer = self + .sink_writer + .take() + .ok_or_else(|| SinkError::Opendal("Can't get sink writer".to_string()))?; + sink_writer.close().await?; } Ok(()) @@ -71,34 +81,87 @@ impl SinkWriter for OpenDalSinkWriter { impl OpenDalSinkWriter { pub fn new( - writer: AsyncArrowWriter, - schema: Schema, + operator: Operator, + write_path: &str, + rw_schema: Schema, pk_indices: Vec, is_append_only: bool, ) -> Result { + let arrow_schema = convert_rw_schema_to_arrow_schema(rw_schema)?; Ok(Self { - schema: schema.clone(), + schema: Arc::new(arrow_schema), + write_path: write_path.to_string(), pk_indices, - writer, + operator, + sink_writer: None, is_append_only, }) } + async fn create_sink_writer(&mut self) -> Result<()> { + let object_store_writer = self + .operator + .writer_with(&self.write_path) + .concurrent(8) + .buffer(SINK_WRITE_BUFFER_SIZE) + .await?; + let parquet_config = ParquetWriterConfig::default(); + let mut props = WriterProperties::builder() + .set_writer_version(WriterVersion::PARQUET_1_0) + .set_bloom_filter_enabled(parquet_config.enable_bloom_filter) + .set_compression(parquet_config.compression) + .set_max_row_group_size(parquet_config.max_row_group_size) + .set_write_batch_size(parquet_config.write_batch_size) + .set_data_page_size_limit(parquet_config.data_page_size); + if let Some(created_by) = parquet_config.created_by.as_ref() { + props = props.set_created_by(created_by.to_string()); + } + self.sink_writer = Some(AsyncArrowWriter::try_new( + object_store_writer, + self.schema.clone(), + SINK_WRITE_BUFFER_SIZE, + Some(props.build()), + )?); + Ok(()) + } + async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { let (mut chunk, ops) = chunk.compact().into_parts(); let filters = chunk.visibility() & ops.iter().map(|op| *op == Op::Insert).collect::(); chunk.set_visibility(filters); - let arrow_schema = change_schema_to_arrow_schema(self.schema.clone()); - let batch = to_record_batch_with_schema(Arc::new(arrow_schema), &chunk.compact())?; - self.writer.write(&batch).await?; + + let batch = to_record_batch_with_schema(self.schema.clone(), &chunk.compact())?; + + self.sink_writer + .as_mut() + .ok_or_else(|| SinkError::Opendal("Sink writer is not created.".to_string()))? + .write(&batch) + .await?; Ok(()) } } -fn change_schema_to_arrow_schema( - _schema: risingwave_common::catalog::Schema, -) -> arrow_schema::Schema { - todo!() +fn convert_rw_schema_to_arrow_schema( + rw_schema: risingwave_common::catalog::Schema, +) -> anyhow::Result { + let mut schema_fields = HashMap::new(); + rw_schema.fields.iter().for_each(|field| { + let res = schema_fields.insert(&field.name, &field.data_type); + // This assert is to make sure there is no duplicate field name in the schema. + assert!(res.is_none()) + }); + let mut arrow_fileds = vec![]; + for rw_field in &rw_schema.fields { + let converted_arrow_data_type = + ArrowDataType::try_from(rw_field.data_type.clone()).map_err(|e| anyhow!(e))?; + arrow_fileds.push(ArrowField::new( + rw_field.name.clone(), + converted_arrow_data_type, + false, + )); + } + + Ok(arrow_schema::Schema::new(arrow_fileds)) } diff --git a/src/connector/src/sink/opendal_sink/s3.rs b/src/connector/src/sink/opendal_sink/s3.rs index ec60ef2ab16c8..dc0a19fe92731 100644 --- a/src/connector/src/sink/opendal_sink/s3.rs +++ b/src/connector/src/sink/opendal_sink/s3.rs @@ -12,27 +12,23 @@ // See the License for the specific language governing permissions and // limitations under the License. use std::collections::HashMap; -use std::sync::Arc; use anyhow::anyhow; use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::S3; -use opendal::{Operator, Writer as OpendalWriter}; -use parquet::arrow::async_writer::AsyncArrowWriter; +use opendal::Operator; use risingwave_common::catalog::Schema; use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; -use crate::sink::opendal_sink::{change_schema_to_arrow_schema, OpenDalSinkWriter}; +use crate::sink::opendal_sink::OpenDalSinkWriter; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{ DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; -const S3_WRITE_BUFFER_SIZE: usize = 16 * 1024 * 1024; - #[derive(Deserialize, Debug, Clone, WithOptions)] pub struct S3Common { #[serde(rename = "s3.region_name")] @@ -163,22 +159,9 @@ impl Sink for S3Sink { ) -> Result { let op = Self::new_s3_sink(self.config.clone())?; let path = self.config.common.path.as_ref(); - let s3_writer = op - .writer_with(path) - .concurrent(8) - .buffer(S3_WRITE_BUFFER_SIZE) - .await?; - - let arrow_schema = change_schema_to_arrow_schema(self.schema.clone()); - let sink_writer: AsyncArrowWriter = AsyncArrowWriter::try_new( - s3_writer, - Arc::new(arrow_schema), - S3_WRITE_BUFFER_SIZE, - None, - )?; - Ok(OpenDalSinkWriter::new( - sink_writer, + op, + path, self.schema.clone(), self.pk_indices.clone(), self.is_append_only, From 31a80520a977c08c76218fdc2c6d41a09501973a Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 14 Mar 2024 16:20:06 +0800 Subject: [PATCH 08/76] minor --- src/connector/src/sink/encoder/json.rs | 32 +++----------------------- src/connector/src/sink/encoder/mod.rs | 1 - 2 files changed, 3 insertions(+), 30 deletions(-) diff --git a/src/connector/src/sink/encoder/json.rs b/src/connector/src/sink/encoder/json.rs index 13fce6f5cfe75..64a06ff70770f 100644 --- a/src/connector/src/sink/encoder/json.rs +++ b/src/connector/src/sink/encoder/json.rs @@ -114,23 +114,6 @@ impl JsonEncoder { } } - pub fn new_with_s3( - schema: Schema, - col_indices: Option>, - _map: HashMap, - ) -> Self { - Self { - schema, - col_indices, - time_handling_mode: TimeHandlingMode::Milli, - date_handling_mode: DateHandlingMode::String, - timestamp_handling_mode: TimestampHandlingMode::String, - timestamptz_handling_mode: TimestamptzHandlingMode::UtcWithoutSuffix, - custom_json_type: CustomJsonType::S3, - kafka_connect: None, - } - } - pub fn new_with_bigquery(schema: Schema, col_indices: Option>) -> Self { Self { schema, @@ -276,10 +259,7 @@ fn datum_to_json_object( } json!(v_string) } - CustomJsonType::Es - | CustomJsonType::None - | CustomJsonType::BigQuery - | CustomJsonType::S3 => { + CustomJsonType::Es | CustomJsonType::None | CustomJsonType::BigQuery => { json!(v.to_text()) } }, @@ -331,10 +311,7 @@ fn datum_to_json_object( } (DataType::Jsonb, ScalarRefImpl::Jsonb(jsonb_ref)) => match custom_json_type { CustomJsonType::Es | CustomJsonType::StarRocks(_) => JsonbVal::from(jsonb_ref).take(), - CustomJsonType::Doris(_) - | CustomJsonType::None - | CustomJsonType::BigQuery - | CustomJsonType::S3 => { + CustomJsonType::Doris(_) | CustomJsonType::None | CustomJsonType::BigQuery => { json!(jsonb_ref.to_string()) } }, @@ -385,10 +362,7 @@ fn datum_to_json_object( "starrocks can't support struct".to_string(), )); } - CustomJsonType::Es - | CustomJsonType::None - | CustomJsonType::BigQuery - | CustomJsonType::S3 => { + CustomJsonType::Es | CustomJsonType::None | CustomJsonType::BigQuery => { let mut map = Map::with_capacity(st.len()); for (sub_datum_ref, sub_field) in struct_ref.iter_fields_ref().zip_eq_debug( st.iter() diff --git a/src/connector/src/sink/encoder/mod.rs b/src/connector/src/sink/encoder/mod.rs index 97b2cb1f8eb99..3254447e27077 100644 --- a/src/connector/src/sink/encoder/mod.rs +++ b/src/connector/src/sink/encoder/mod.rs @@ -144,7 +144,6 @@ pub enum CustomJsonType { Es, // starrocks' need jsonb is struct StarRocks(HashMap), - S3, // bigquery need null array -> [] BigQuery, None, From d1b61a9d5e9d10e8efd65bc46265be37888b138a Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 14 Mar 2024 17:11:07 +0800 Subject: [PATCH 09/76] fix typo --- src/connector/src/sink/opendal_sink/mod.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/connector/src/sink/opendal_sink/mod.rs b/src/connector/src/sink/opendal_sink/mod.rs index 8415d9de84c70..723450cce1645 100644 --- a/src/connector/src/sink/opendal_sink/mod.rs +++ b/src/connector/src/sink/opendal_sink/mod.rs @@ -152,16 +152,16 @@ fn convert_rw_schema_to_arrow_schema( // This assert is to make sure there is no duplicate field name in the schema. assert!(res.is_none()) }); - let mut arrow_fileds = vec![]; + let mut arrow_fields = vec![]; for rw_field in &rw_schema.fields { let converted_arrow_data_type = ArrowDataType::try_from(rw_field.data_type.clone()).map_err(|e| anyhow!(e))?; - arrow_fileds.push(ArrowField::new( + arrow_fields.push(ArrowField::new( rw_field.name.clone(), converted_arrow_data_type, false, )); } - Ok(arrow_schema::Schema::new(arrow_fileds)) + Ok(arrow_schema::Schema::new(arrow_fields)) } From f2163792f876ca6ac7d58ff0f1ba9dde613829ac Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 18 Mar 2024 15:46:23 +0800 Subject: [PATCH 10/76] add fs sink for test --- src/connector/src/sink/opendal_sink/fs.rs | 145 ++++++++++++++++++++++ 1 file changed, 145 insertions(+) create mode 100644 src/connector/src/sink/opendal_sink/fs.rs diff --git a/src/connector/src/sink/opendal_sink/fs.rs b/src/connector/src/sink/opendal_sink/fs.rs new file mode 100644 index 0000000000000..44b6ffe75fde4 --- /dev/null +++ b/src/connector/src/sink/opendal_sink/fs.rs @@ -0,0 +1,145 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +use std::collections::HashMap; + +use anyhow::anyhow; +use opendal::layers::{LoggingLayer, RetryLayer}; +use opendal::services::Fs; +use opendal::Operator; +use risingwave_common::catalog::Schema; +use serde::Deserialize; +use serde_with::serde_as; +use with_options::WithOptions; + +use crate::sink::opendal_sink::OpenDalSinkWriter; +use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; +use crate::sink::{ + DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, + SINK_TYPE_OPTION, SINK_TYPE_UPSERT, +}; + +#[derive(Deserialize, Debug, Clone, WithOptions)] +pub struct FsCommon { + #[serde(rename = "fs.path", default)] + pub path: String, +} + +#[serde_as] +#[derive(Clone, Debug, Deserialize, WithOptions)] +pub struct FsConfig { + #[serde(flatten)] + pub common: FsCommon, + + pub r#type: String, // accept "append-only" or "upsert" +} + +pub const FS_SINK: &str = "fs"; + +impl FsConfig { + pub fn from_hashmap(properties: HashMap) -> Result { + let config = serde_json::from_value::(serde_json::to_value(properties).unwrap()) + .map_err(|e| SinkError::Config(anyhow!(e)))?; + if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { + return Err(SinkError::Config(anyhow!( + "`{}` must be {}, or {}", + SINK_TYPE_OPTION, + SINK_TYPE_APPEND_ONLY, + SINK_TYPE_UPSERT + ))); + } + Ok(config) + } +} + +#[derive(Debug)] +pub struct FsSink { + pub config: FsConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, +} + +impl FsSink { + pub fn new( + config: FsConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + ) -> Result { + Ok(Self { + config, + schema, + pk_indices, + is_append_only, + }) + } +} + +impl FsSink { + pub fn new_fs_sink(config: FsConfig) -> Result { + // Create fs builder. + let mut builder = Fs::default(); + // Create fs backend builder. + builder.root(&config.common.path); + let operator: Operator = Operator::new(builder)? + .layer(LoggingLayer::default()) + .layer(RetryLayer::default()) + .finish(); + Ok(operator) + } +} + +impl Sink for FsSink { + type Coordinator = DummySinkCommitCoordinator; + type LogSinker = LogSinkerOf; + + const SINK_NAME: &'static str = FS_SINK; + + async fn validate(&self) -> Result<()> { + let _op = Self::new_fs_sink(self.config.clone())?; + Ok(()) + } + + async fn new_log_sinker( + &self, + writer_param: crate::sink::SinkWriterParam, + ) -> Result { + let op = Self::new_fs_sink(self.config.clone())?; + let path = self.config.common.path.as_ref(); + println!("path = {}", path); + Ok(OpenDalSinkWriter::new( + op, + path, + self.schema.clone(), + self.pk_indices.clone(), + self.is_append_only, + )? + .into_log_sinker(writer_param.sink_metrics)) + } +} + +impl TryFrom for FsSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let schema = param.schema(); + let config = FsConfig::from_hashmap(param.properties)?; + FsSink::new( + config, + schema, + param.downstream_pk, + param.sink_type.is_append_only(), + ) + } +} From 9b593c7a28871678bf3f45819e53d83e042d02ff Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 17 Jun 2024 16:57:09 +0800 Subject: [PATCH 11/76] save work --- risedev.yml | 8 ++++---- src/connector/src/sink/opendal_sink/mod.rs | 10 +++++----- src/connector/src/sink/opendal_sink/s3.rs | 3 +-- 3 files changed, 10 insertions(+), 11 deletions(-) diff --git a/risedev.yml b/risedev.yml index 9308fd9a63e03..e7f738dcbd50d 100644 --- a/risedev.yml +++ b/risedev.yml @@ -102,10 +102,10 @@ profile: - use: compactor - use: prometheus - use: grafana - - use: zookeeper - persist-data: true - - use: kafka - persist-data: true + # - use: zookeeper + # persist-data: true + # - use: kafka + # persist-data: true standalone-full-peripherals: steps: diff --git a/src/connector/src/sink/opendal_sink/mod.rs b/src/connector/src/sink/opendal_sink/mod.rs index 723450cce1645..cab072e2a3b8f 100644 --- a/src/connector/src/sink/opendal_sink/mod.rs +++ b/src/connector/src/sink/opendal_sink/mod.rs @@ -14,6 +14,7 @@ pub mod gcs; pub mod s3; +pub mod fs; use std::collections::HashMap; use std::sync::Arc; @@ -63,11 +64,10 @@ impl SinkWriter for OpenDalSinkWriter { } async fn barrier(&mut self, is_checkpoint: bool) -> Result<()> { - if is_checkpoint { - let sink_writer = self - .sink_writer - .take() - .ok_or_else(|| SinkError::Opendal("Can't get sink writer".to_string()))?; + if is_checkpoint && let Some(sink_writer) =self + .sink_writer + .take() { + sink_writer.close().await?; } diff --git a/src/connector/src/sink/opendal_sink/s3.rs b/src/connector/src/sink/opendal_sink/s3.rs index dc0a19fe92731..30cf8c8b218c2 100644 --- a/src/connector/src/sink/opendal_sink/s3.rs +++ b/src/connector/src/sink/opendal_sink/s3.rs @@ -127,12 +127,11 @@ impl S3Sink { ); } - builder.enable_virtual_host_style(); if let Some(assume_role) = config.common.assume_role { builder.role_arn(&assume_role); } - + builder.disable_config_load(); let operator: Operator = Operator::new(builder)? .layer(LoggingLayer::default()) .layer(RetryLayer::default()) From 69d5052393593485a9aaae71bc6482b821a4d6ed Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 17 Jun 2024 18:46:15 +0800 Subject: [PATCH 12/76] save work --- proto/connector_service.proto | 1 + src/connector/src/sink/opendal_sink/fs.rs | 1 + src/connector/src/sink/opendal_sink/gcs.rs | 1 + src/connector/src/sink/opendal_sink/mod.rs | 20 ++++++++++++++++---- src/connector/src/sink/opendal_sink/s3.rs | 1 + 5 files changed, 20 insertions(+), 4 deletions(-) diff --git a/proto/connector_service.proto b/proto/connector_service.proto index 4deb0d6fb6096..c2b4ff13842a7 100644 --- a/proto/connector_service.proto +++ b/proto/connector_service.proto @@ -34,6 +34,7 @@ enum SinkPayloadFormat { FORMAT_UNSPECIFIED = 0; JSON = 1; STREAM_CHUNK = 2; + PARQUET = 3; } message SinkWriterStreamRequest { diff --git a/src/connector/src/sink/opendal_sink/fs.rs b/src/connector/src/sink/opendal_sink/fs.rs index 44b6ffe75fde4..a696c5f1c6cd3 100644 --- a/src/connector/src/sink/opendal_sink/fs.rs +++ b/src/connector/src/sink/opendal_sink/fs.rs @@ -124,6 +124,7 @@ impl Sink for FsSink { self.schema.clone(), self.pk_indices.clone(), self.is_append_only, + writer_param.connector_params.sink_payload_format, )? .into_log_sinker(writer_param.sink_metrics)) } diff --git a/src/connector/src/sink/opendal_sink/gcs.rs b/src/connector/src/sink/opendal_sink/gcs.rs index 2715fde48a0ae..9a3ff6b63a16a 100644 --- a/src/connector/src/sink/opendal_sink/gcs.rs +++ b/src/connector/src/sink/opendal_sink/gcs.rs @@ -153,6 +153,7 @@ impl Sink for GcsSink { self.schema.clone(), self.pk_indices.clone(), self.is_append_only, + writer_param.connector_params.sink_payload_format, )? .into_log_sinker(writer_param.sink_metrics)) } diff --git a/src/connector/src/sink/opendal_sink/mod.rs b/src/connector/src/sink/opendal_sink/mod.rs index cab072e2a3b8f..cd82140aca299 100644 --- a/src/connector/src/sink/opendal_sink/mod.rs +++ b/src/connector/src/sink/opendal_sink/mod.rs @@ -17,7 +17,7 @@ pub mod s3; pub mod fs; use std::collections::HashMap; use std::sync::Arc; - +use crate::SinkPayloadFormat; use anyhow::anyhow; use arrow_schema::{DataType as ArrowDataType, Field as ArrowField, SchemaRef}; use async_trait::async_trait; @@ -40,13 +40,20 @@ pub struct OpenDalSinkWriter { pk_indices: Vec, is_append_only: bool, write_path: String, + epoch: Option, + sink_payload_format: SinkPayloadFormat, } + + #[async_trait] impl SinkWriter for OpenDalSinkWriter { async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { + let epoch = self.epoch.ok_or_else(|| { + SinkError::Opendal("epoch has not been initialize, call `begin_epoch`".to_string()) + })?; if self.sink_writer.is_none() { - self.create_sink_writer().await?; + self.create_sink_writer(epoch).await?; } if self.is_append_only { self.append_only(chunk).await @@ -67,6 +74,7 @@ impl SinkWriter for OpenDalSinkWriter { if is_checkpoint && let Some(sink_writer) =self .sink_writer .take() { + sink_writer.close().await?; } @@ -86,6 +94,7 @@ impl OpenDalSinkWriter { rw_schema: Schema, pk_indices: Vec, is_append_only: bool, + sink_payload_format: SinkPayloadFormat, ) -> Result { let arrow_schema = convert_rw_schema_to_arrow_schema(rw_schema)?; Ok(Self { @@ -95,13 +104,16 @@ impl OpenDalSinkWriter { operator, sink_writer: None, is_append_only, + sink_payload_format, + epoch: None, }) } - async fn create_sink_writer(&mut self) -> Result<()> { + async fn create_sink_writer(&mut self, epoch: u64) -> Result<()> { + let object_name = format!("{}/{}.parquet", self.write_path, epoch.to_string()); let object_store_writer = self .operator - .writer_with(&self.write_path) + .writer_with(&object_name) .concurrent(8) .buffer(SINK_WRITE_BUFFER_SIZE) .await?; diff --git a/src/connector/src/sink/opendal_sink/s3.rs b/src/connector/src/sink/opendal_sink/s3.rs index 30cf8c8b218c2..14004a0addd65 100644 --- a/src/connector/src/sink/opendal_sink/s3.rs +++ b/src/connector/src/sink/opendal_sink/s3.rs @@ -164,6 +164,7 @@ impl Sink for S3Sink { self.schema.clone(), self.pk_indices.clone(), self.is_append_only, + writer_param.connector_params.sink_payload_format, )? .into_log_sinker(writer_param.sink_metrics)) } From 0a72bdbe9839cba3b88f076aede7e40b95d3ec63 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 18 Jun 2024 09:24:39 +0800 Subject: [PATCH 13/76] save work --- src/connector/src/sink/opendal_sink/mod.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/connector/src/sink/opendal_sink/mod.rs b/src/connector/src/sink/opendal_sink/mod.rs index cd82140aca299..a6306681e7449 100644 --- a/src/connector/src/sink/opendal_sink/mod.rs +++ b/src/connector/src/sink/opendal_sink/mod.rs @@ -62,7 +62,8 @@ impl SinkWriter for OpenDalSinkWriter { } } - async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { + async fn begin_epoch(&mut self, epoch: u64) -> Result<()> { + self.epoch = Some(epoch); Ok(()) } @@ -110,7 +111,8 @@ impl OpenDalSinkWriter { } async fn create_sink_writer(&mut self, epoch: u64) -> Result<()> { - let object_name = format!("{}/{}.parquet", self.write_path, epoch.to_string()); + println!("这里epoch = {:?}", epoch.to_string()); + let object_name = format!("{}/epoch_{}.parquet", self.write_path, epoch.to_string()); let object_store_writer = self .operator .writer_with(&object_name) @@ -142,7 +144,7 @@ impl OpenDalSinkWriter { let filters = chunk.visibility() & ops.iter().map(|op| *op == Op::Insert).collect::(); chunk.set_visibility(filters); - + println!("schema是{:?}", self.schema.clone()); let batch = to_record_batch_with_schema(self.schema.clone(), &chunk.compact())?; self.sink_writer From b200a3c910f285574c87551fa4eef69156cab05f Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 18 Jun 2024 16:31:00 +0800 Subject: [PATCH 14/76] introduce file sink with parquet type --- proto/plan_common.proto | 1 + src/connector/src/sink/catalog/mod.rs | 3 + src/connector/src/sink/encoder/mod.rs | 2 + src/connector/src/sink/formatter/mod.rs | 5 +- src/connector/src/sink/opendal_sink/fs.rs | 39 +++--- src/connector/src/sink/opendal_sink/gcs.rs | 36 ++---- src/connector/src/sink/opendal_sink/mod.rs | 113 ++++++++++++------ src/connector/src/sink/opendal_sink/s3.rs | 38 ++---- .../src/handler/alter_source_with_sr.rs | 1 + src/frontend/src/handler/create_sink.rs | 5 + src/frontend/src/handler/create_source.rs | 1 + src/sqlparser/src/ast/statement.rs | 5 +- 12 files changed, 134 insertions(+), 115 deletions(-) diff --git a/proto/plan_common.proto b/proto/plan_common.proto index 79a1b1622704e..7cfc5fda48345 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -147,6 +147,7 @@ enum EncodeType { ENCODE_TYPE_BYTES = 6; ENCODE_TYPE_TEMPLATE = 7; ENCODE_TYPE_NONE = 8; + ENCODE_TYPE_PARQUET= 10; } enum RowFormatType { diff --git a/src/connector/src/sink/catalog/mod.rs b/src/connector/src/sink/catalog/mod.rs index e6a654f75a5fd..4b817853d4605 100644 --- a/src/connector/src/sink/catalog/mod.rs +++ b/src/connector/src/sink/catalog/mod.rs @@ -134,6 +134,7 @@ pub enum SinkEncode { Protobuf, Avro, Template, + Parquet, } impl SinkFormatDesc { @@ -180,6 +181,7 @@ impl SinkFormatDesc { SinkEncode::Protobuf => E::Protobuf, SinkEncode::Avro => E::Avro, SinkEncode::Template => E::Template, + SinkEncode::Parquet => E::Parquet, }; let options = self .options @@ -222,6 +224,7 @@ impl TryFrom for SinkFormatDesc { E::Protobuf => SinkEncode::Protobuf, E::Template => SinkEncode::Template, E::Avro => SinkEncode::Avro, + E::Parquet => SinkEncode::Parquet, e @ (E::Unspecified | E::Native | E::Csv | E::Bytes | E::None) => { return Err(SinkError::Config(anyhow!( "sink encode unsupported: {}", diff --git a/src/connector/src/sink/encoder/mod.rs b/src/connector/src/sink/encoder/mod.rs index 3254447e27077..1ba0288cc517e 100644 --- a/src/connector/src/sink/encoder/mod.rs +++ b/src/connector/src/sink/encoder/mod.rs @@ -22,11 +22,13 @@ use crate::sink::Result; mod avro; mod json; +// mod parquet; mod proto; pub mod template; pub use avro::{AvroEncoder, AvroHeader}; pub use json::JsonEncoder; +// pub use parquet::ParquetEncoder; pub use proto::ProtoEncoder; /// Encode a row of a relation into diff --git a/src/connector/src/sink/formatter/mod.rs b/src/connector/src/sink/formatter/mod.rs index cb36bda793709..7a4e521907bc1 100644 --- a/src/connector/src/sink/formatter/mod.rs +++ b/src/connector/src/sink/formatter/mod.rs @@ -109,6 +109,9 @@ impl SinkFormatterImpl { }); match format_desc.encode { + SinkEncode::Parquet => { + unreachable!() + } SinkEncode::Json => { let val_encoder = JsonEncoder::new( schema, @@ -256,7 +259,7 @@ impl SinkFormatterImpl { let formatter = UpsertFormatter::new(key_encoder, val_encoder); Ok(SinkFormatterImpl::UpsertAvro(formatter)) } - SinkEncode::Protobuf => err_unsupported(), + SinkEncode::Protobuf | SinkEncode::Parquet => err_unsupported(), } } } diff --git a/src/connector/src/sink/opendal_sink/fs.rs b/src/connector/src/sink/opendal_sink/fs.rs index a696c5f1c6cd3..b8157b39e54f7 100644 --- a/src/connector/src/sink/opendal_sink/fs.rs +++ b/src/connector/src/sink/opendal_sink/fs.rs @@ -25,8 +25,8 @@ use with_options::WithOptions; use crate::sink::opendal_sink::OpenDalSinkWriter; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{ - DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, - SINK_TYPE_OPTION, SINK_TYPE_UPSERT, + DummySinkCommitCoordinator, Result, Sink, SinkError, SinkFormatDesc, SinkParam, + SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; #[derive(Deserialize, Debug, Clone, WithOptions)] @@ -66,24 +66,9 @@ impl FsConfig { pub struct FsSink { pub config: FsConfig, schema: Schema, - pk_indices: Vec, is_append_only: bool, -} - -impl FsSink { - pub fn new( - config: FsConfig, - schema: Schema, - pk_indices: Vec, - is_append_only: bool, - ) -> Result { - Ok(Self { - config, - schema, - pk_indices, - is_append_only, - }) - } + pk_indices: Vec, + format_desc: SinkFormatDesc, } impl FsSink { @@ -117,14 +102,15 @@ impl Sink for FsSink { ) -> Result { let op = Self::new_fs_sink(self.config.clone())?; let path = self.config.common.path.as_ref(); - println!("path = {}", path); + Ok(OpenDalSinkWriter::new( op, path, self.schema.clone(), self.pk_indices.clone(), self.is_append_only, - writer_param.connector_params.sink_payload_format, + writer_param.executor_id, + self.format_desc.encode.clone(), )? .into_log_sinker(writer_param.sink_metrics)) } @@ -136,11 +122,14 @@ impl TryFrom for FsSink { fn try_from(param: SinkParam) -> std::result::Result { let schema = param.schema(); let config = FsConfig::from_hashmap(param.properties)?; - FsSink::new( + Ok(Self { config, schema, - param.downstream_pk, - param.sink_type.is_append_only(), - ) + is_append_only: param.sink_type.is_append_only(), + pk_indices: param.downstream_pk, + format_desc: param + .format_desc + .ok_or_else(|| SinkError::Config(anyhow!("missing FORMAT ... ENCODE ...")))?, + }) } } diff --git a/src/connector/src/sink/opendal_sink/gcs.rs b/src/connector/src/sink/opendal_sink/gcs.rs index 9a3ff6b63a16a..cc04f3fc4c248 100644 --- a/src/connector/src/sink/opendal_sink/gcs.rs +++ b/src/connector/src/sink/opendal_sink/gcs.rs @@ -25,8 +25,8 @@ use with_options::WithOptions; use crate::sink::opendal_sink::OpenDalSinkWriter; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{ - DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, - SINK_TYPE_OPTION, SINK_TYPE_UPSERT, + DummySinkCommitCoordinator, Result, Sink, SinkError, SinkFormatDesc, SinkParam, + SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; const GCS_WRITE_BUFFER_SIZE: usize = 16 * 1024 * 1024; @@ -83,22 +83,7 @@ pub struct GcsSink { schema: Schema, pk_indices: Vec, is_append_only: bool, -} - -impl GcsSink { - pub fn new( - config: GcsConfig, - schema: Schema, - pk_indices: Vec, - is_append_only: bool, - ) -> Result { - Ok(Self { - config, - schema, - pk_indices, - is_append_only, - }) - } + format_desc: SinkFormatDesc, } impl GcsSink { @@ -146,14 +131,14 @@ impl Sink for GcsSink { ) -> Result { let op = Self::new_gcs_sink(self.config.clone())?; let path = self.config.common.path.as_ref(); - Ok(OpenDalSinkWriter::new( op, path, self.schema.clone(), self.pk_indices.clone(), self.is_append_only, - writer_param.connector_params.sink_payload_format, + writer_param.executor_id, + self.format_desc.encode.clone(), )? .into_log_sinker(writer_param.sink_metrics)) } @@ -165,11 +150,14 @@ impl TryFrom for GcsSink { fn try_from(param: SinkParam) -> std::result::Result { let schema = param.schema(); let config = GcsConfig::from_hashmap(param.properties)?; - GcsSink::new( + Ok(Self { config, schema, - param.downstream_pk, - param.sink_type.is_append_only(), - ) + is_append_only: param.sink_type.is_append_only(), + pk_indices: param.downstream_pk, + format_desc: param + .format_desc + .ok_or_else(|| SinkError::Config(anyhow!("missing FORMAT ... ENCODE ...")))?, + }) } } diff --git a/src/connector/src/sink/opendal_sink/mod.rs b/src/connector/src/sink/opendal_sink/mod.rs index a6306681e7449..f5c30cf0278c2 100644 --- a/src/connector/src/sink/opendal_sink/mod.rs +++ b/src/connector/src/sink/opendal_sink/mod.rs @@ -12,12 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod fs; pub mod gcs; pub mod s3; -pub mod fs; use std::collections::HashMap; use std::sync::Arc; -use crate::SinkPayloadFormat; + use anyhow::anyhow; use arrow_schema::{DataType as ArrowDataType, Field as ArrowField, SchemaRef}; use async_trait::async_trait; @@ -29,6 +29,7 @@ use risingwave_common::array::{to_record_batch_with_schema, Op, StreamChunk}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::Schema; +use crate::sink::catalog::SinkEncode; use crate::sink::{Result, SinkError, SinkWriter}; const SINK_WRITE_BUFFER_SIZE: usize = 16 * 1024 * 1024; @@ -36,15 +37,19 @@ const SINK_WRITE_BUFFER_SIZE: usize = 16 * 1024 * 1024; pub struct OpenDalSinkWriter { schema: SchemaRef, operator: Operator, - sink_writer: Option>, + sink_writer: Option, pk_indices: Vec, is_append_only: bool, write_path: String, epoch: Option, - sink_payload_format: SinkPayloadFormat, + executor_id: u64, + encode_type: SinkEncode, } - +enum FileWriterEnum { + ParquetWriter(AsyncArrowWriter), + FileWriter(OpendalWriter), +} #[async_trait] impl SinkWriter for OpenDalSinkWriter { @@ -72,12 +77,13 @@ impl SinkWriter for OpenDalSinkWriter { } async fn barrier(&mut self, is_checkpoint: bool) -> Result<()> { - if is_checkpoint && let Some(sink_writer) =self - .sink_writer - .take() { - - - sink_writer.close().await?; + if is_checkpoint && let Some(sink_writer) = self.sink_writer.take() { + match sink_writer { + FileWriterEnum::ParquetWriter(mut w) => { + let _ = w.close().await?; + } + FileWriterEnum::FileWriter(mut w) => w.close().await?, + }; } Ok(()) @@ -95,7 +101,8 @@ impl OpenDalSinkWriter { rw_schema: Schema, pk_indices: Vec, is_append_only: bool, - sink_payload_format: SinkPayloadFormat, + executor_id: u64, + encode_type: SinkEncode, ) -> Result { let arrow_schema = convert_rw_schema_to_arrow_schema(rw_schema)?; Ok(Self { @@ -105,37 +112,61 @@ impl OpenDalSinkWriter { operator, sink_writer: None, is_append_only, - sink_payload_format, epoch: None, + executor_id, + encode_type, }) } - async fn create_sink_writer(&mut self, epoch: u64) -> Result<()> { - println!("这里epoch = {:?}", epoch.to_string()); - let object_name = format!("{}/epoch_{}.parquet", self.write_path, epoch.to_string()); - let object_store_writer = self + async fn create_object_writer(&mut self, epoch: u64) -> Result<(OpendalWriter)> { + let suffix = match self.encode_type { + SinkEncode::Json => "json", + SinkEncode::Parquet => "parquet", + _ => unimplemented!(), + }; + let object_name = format!( + "{}/epoch_{}_executor_{}.{}", + self.write_path, + epoch.to_string(), + self.executor_id.to_string(), + suffix.to_string(), + ); + Ok(self .operator .writer_with(&object_name) .concurrent(8) .buffer(SINK_WRITE_BUFFER_SIZE) - .await?; - let parquet_config = ParquetWriterConfig::default(); - let mut props = WriterProperties::builder() - .set_writer_version(WriterVersion::PARQUET_1_0) - .set_bloom_filter_enabled(parquet_config.enable_bloom_filter) - .set_compression(parquet_config.compression) - .set_max_row_group_size(parquet_config.max_row_group_size) - .set_write_batch_size(parquet_config.write_batch_size) - .set_data_page_size_limit(parquet_config.data_page_size); - if let Some(created_by) = parquet_config.created_by.as_ref() { - props = props.set_created_by(created_by.to_string()); + .await?) + } + + async fn create_sink_writer(&mut self, epoch: u64) -> Result<()> { + let object_writer = self.create_object_writer(epoch).await?; + match self.encode_type { + SinkEncode::Parquet => { + let parquet_config = ParquetWriterConfig::default(); + let mut props = WriterProperties::builder() + .set_writer_version(WriterVersion::PARQUET_1_0) + .set_bloom_filter_enabled(parquet_config.enable_bloom_filter) + .set_compression(parquet_config.compression) + .set_max_row_group_size(parquet_config.max_row_group_size) + .set_write_batch_size(parquet_config.write_batch_size) + .set_data_page_size_limit(parquet_config.data_page_size); + if let Some(created_by) = parquet_config.created_by.as_ref() { + props = props.set_created_by(created_by.to_string()); + } + self.sink_writer = Some(FileWriterEnum::ParquetWriter(AsyncArrowWriter::try_new( + object_writer, + self.schema.clone(), + SINK_WRITE_BUFFER_SIZE, + Some(props.build()), + )?)); + } + SinkEncode::Json => { + self.sink_writer = Some(FileWriterEnum::FileWriter(object_writer)); + } + _ => unimplemented!(), } - self.sink_writer = Some(AsyncArrowWriter::try_new( - object_store_writer, - self.schema.clone(), - SINK_WRITE_BUFFER_SIZE, - Some(props.build()), - )?); + Ok(()) } @@ -144,14 +175,18 @@ impl OpenDalSinkWriter { let filters = chunk.visibility() & ops.iter().map(|op| *op == Op::Insert).collect::(); chunk.set_visibility(filters); - println!("schema是{:?}", self.schema.clone()); - let batch = to_record_batch_with_schema(self.schema.clone(), &chunk.compact())?; - self.sink_writer + match self + .sink_writer .as_mut() .ok_or_else(|| SinkError::Opendal("Sink writer is not created.".to_string()))? - .write(&batch) - .await?; + { + FileWriterEnum::ParquetWriter(w) => { + let batch = to_record_batch_with_schema(self.schema.clone(), &chunk.compact())?; + w.write(&batch).await?; + } + FileWriterEnum::FileWriter(_w) => unimplemented!(), + } Ok(()) } diff --git a/src/connector/src/sink/opendal_sink/s3.rs b/src/connector/src/sink/opendal_sink/s3.rs index 14004a0addd65..eace046e90c4f 100644 --- a/src/connector/src/sink/opendal_sink/s3.rs +++ b/src/connector/src/sink/opendal_sink/s3.rs @@ -25,8 +25,8 @@ use with_options::WithOptions; use crate::sink::opendal_sink::OpenDalSinkWriter; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{ - DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, - SINK_TYPE_OPTION, SINK_TYPE_UPSERT, + DummySinkCommitCoordinator, Result, Sink, SinkError, SinkFormatDesc, SinkParam, + SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; #[derive(Deserialize, Debug, Clone, WithOptions)] @@ -78,24 +78,9 @@ impl S3Config { pub struct S3Sink { pub config: S3Config, schema: Schema, - pk_indices: Vec, is_append_only: bool, -} - -impl S3Sink { - pub fn new( - config: S3Config, - schema: Schema, - pk_indices: Vec, - is_append_only: bool, - ) -> Result { - Ok(Self { - config, - schema, - pk_indices, - is_append_only, - }) - } + pk_indices: Vec, + format_desc: SinkFormatDesc, } impl S3Sink { @@ -127,7 +112,6 @@ impl S3Sink { ); } - if let Some(assume_role) = config.common.assume_role { builder.role_arn(&assume_role); } @@ -164,7 +148,8 @@ impl Sink for S3Sink { self.schema.clone(), self.pk_indices.clone(), self.is_append_only, - writer_param.connector_params.sink_payload_format, + writer_param.executor_id, + self.format_desc.encode.clone(), )? .into_log_sinker(writer_param.sink_metrics)) } @@ -176,11 +161,14 @@ impl TryFrom for S3Sink { fn try_from(param: SinkParam) -> std::result::Result { let schema = param.schema(); let config = S3Config::from_hashmap(param.properties)?; - S3Sink::new( + Ok(Self { config, schema, - param.downstream_pk, - param.sink_type.is_append_only(), - ) + is_append_only: param.sink_type.is_append_only(), + pk_indices: param.downstream_pk, + format_desc: param + .format_desc + .ok_or_else(|| SinkError::Config(anyhow!("missing FORMAT ... ENCODE ...")))?, + }) } } diff --git a/src/frontend/src/handler/alter_source_with_sr.rs b/src/frontend/src/handler/alter_source_with_sr.rs index fc35552270a2e..60c30a6a1d16a 100644 --- a/src/frontend/src/handler/alter_source_with_sr.rs +++ b/src/frontend/src/handler/alter_source_with_sr.rs @@ -61,6 +61,7 @@ fn encode_type_to_encode(from: EncodeType) -> Option { EncodeType::Json => Encode::Json, EncodeType::Bytes => Encode::Bytes, EncodeType::Template => Encode::Template, + EncodeType::Parquet => Encode::Parquet, EncodeType::None => Encode::None, }) } diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index 9de145dc47801..9cce3b2716c65 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -738,6 +738,7 @@ fn bind_sink_format_desc(value: ConnectorSchema) -> Result { E::Protobuf => SinkEncode::Protobuf, E::Avro => SinkEncode::Avro, E::Template => SinkEncode::Template, + E::Parquet => SinkEncode::Parquet, e @ (E::Native | E::Csv | E::Bytes | E::None) => { return Err(ErrorCode::BindError(format!("sink encode unsupported: {e}")).into()); } @@ -759,6 +760,7 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock vec![Encode::Json, Encode::Avro], Format::Debezium => vec![Encode::Json], ), + S3Sink::SINK_NAME => hashmap!( + Format::Plain => vec![Encode::Json, Encode::Parquet], + ), KinesisSink::SINK_NAME => hashmap!( Format::Plain => vec![Encode::Json], Format::Upsert => vec![Encode::Json], diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 566727a9085f4..b2fefbd03b7d9 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -1466,6 +1466,7 @@ fn row_encode_to_prost(row_encode: &Encode) -> EncodeType { Encode::Csv => EncodeType::Csv, Encode::Bytes => EncodeType::Bytes, Encode::Template => EncodeType::Template, + Encode::Parquet => EncodeType::Parquet, Encode::None => EncodeType::None, } } diff --git a/src/sqlparser/src/ast/statement.rs b/src/sqlparser/src/ast/statement.rs index dac019fe5e126..58f3353975bdf 100644 --- a/src/sqlparser/src/ast/statement.rs +++ b/src/sqlparser/src/ast/statement.rs @@ -155,6 +155,7 @@ pub enum Encode { None, // Keyword::None Native, Template, + Parquet, } // TODO: unify with `from_keyword` @@ -172,6 +173,7 @@ impl fmt::Display for Encode { Encode::Native => "NATIVE", Encode::Template => "TEMPLATE", Encode::None => "NONE", + Encode::Parquet => "PARQUET", } ) } @@ -188,8 +190,9 @@ impl Encode { "TEMPLATE" => Encode::Template, "NATIVE" => Encode::Native, // used internally for schema change "NONE" => Encode::None, // used by iceberg + "PARQUET" => Encode::Parquet, _ => return Err(ParserError::ParserError( - "expected AVRO | BYTES | CSV | PROTOBUF | JSON | NATIVE | TEMPLATE | NONE after Encode" + "expected AVRO | BYTES | CSV | PROTOBUF | JSON | NATIVE | TEMPLATE | PARQUET | NONE after Encode" .to_string(), )), }) From 7e18fe1e73e1bd1752161bb61034a16369f0f0bf Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 19 Jun 2024 17:48:39 +0800 Subject: [PATCH 15/76] refactor --- .../sink/{opendal_sink => file_sink}/fs.rs | 95 +++++------------- .../sink/{opendal_sink => file_sink}/gcs.rs | 94 +++++------------- .../sink/{opendal_sink => file_sink}/mod.rs | 3 +- .../src/sink/file_sink/opendal_sink.rs | 95 ++++++++++++++++++ .../sink/{opendal_sink => file_sink}/s3.rs | 97 +++++-------------- src/connector/src/sink/mod.rs | 10 +- src/frontend/src/handler/create_sink.rs | 9 +- .../src/optimizer/plan_node/stream_sink.rs | 2 + src/stream/src/from_proto/sink.rs | 2 + 9 files changed, 189 insertions(+), 218 deletions(-) rename src/connector/src/sink/{opendal_sink => file_sink}/fs.rs (53%) rename src/connector/src/sink/{opendal_sink => file_sink}/gcs.rs (62%) rename src/connector/src/sink/{opendal_sink => file_sink}/mod.rs (99%) create mode 100644 src/connector/src/sink/file_sink/opendal_sink.rs rename src/connector/src/sink/{opendal_sink => file_sink}/s3.rs (63%) diff --git a/src/connector/src/sink/opendal_sink/fs.rs b/src/connector/src/sink/file_sink/fs.rs similarity index 53% rename from src/connector/src/sink/opendal_sink/fs.rs rename to src/connector/src/sink/file_sink/fs.rs index b8157b39e54f7..f0c5ef01726d5 100644 --- a/src/connector/src/sink/opendal_sink/fs.rs +++ b/src/connector/src/sink/file_sink/fs.rs @@ -22,12 +22,10 @@ use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; -use crate::sink::opendal_sink::OpenDalSinkWriter; +use crate::sink::file_sink::opendal_sink::{FileSink, OpendalSinkBackend}; +use crate::sink::file_sink::OpenDalSinkWriter; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; -use crate::sink::{ - DummySinkCommitCoordinator, Result, Sink, SinkError, SinkFormatDesc, SinkParam, - SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, -}; +use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; #[derive(Deserialize, Debug, Clone, WithOptions)] pub struct FsCommon { @@ -46,32 +44,7 @@ pub struct FsConfig { pub const FS_SINK: &str = "fs"; -impl FsConfig { - pub fn from_hashmap(properties: HashMap) -> Result { - let config = serde_json::from_value::(serde_json::to_value(properties).unwrap()) - .map_err(|e| SinkError::Config(anyhow!(e)))?; - if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { - return Err(SinkError::Config(anyhow!( - "`{}` must be {}, or {}", - SINK_TYPE_OPTION, - SINK_TYPE_APPEND_ONLY, - SINK_TYPE_UPSERT - ))); - } - Ok(config) - } -} - -#[derive(Debug)] -pub struct FsSink { - pub config: FsConfig, - schema: Schema, - is_append_only: bool, - pk_indices: Vec, - format_desc: SinkFormatDesc, -} - -impl FsSink { +impl FileSink { pub fn new_fs_sink(config: FsConfig) -> Result { // Create fs builder. let mut builder = Fs::default(); @@ -85,51 +58,33 @@ impl FsSink { } } -impl Sink for FsSink { - type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf; +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct FsSink; + +impl OpendalSinkBackend for FsSink { + type Properties = FsConfig; const SINK_NAME: &'static str = FS_SINK; - async fn validate(&self) -> Result<()> { - let _op = Self::new_fs_sink(self.config.clone())?; - Ok(()) + fn from_hashmap(hash_map: HashMap) -> Result { + let config = serde_json::from_value::(serde_json::to_value(hash_map).unwrap()) + .map_err(|e| SinkError::Config(anyhow!(e)))?; + if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { + return Err(SinkError::Config(anyhow!( + "`{}` must be {}, or {}", + SINK_TYPE_OPTION, + SINK_TYPE_APPEND_ONLY, + SINK_TYPE_UPSERT + ))); + } + Ok(config) } - async fn new_log_sinker( - &self, - writer_param: crate::sink::SinkWriterParam, - ) -> Result { - let op = Self::new_fs_sink(self.config.clone())?; - let path = self.config.common.path.as_ref(); - - Ok(OpenDalSinkWriter::new( - op, - path, - self.schema.clone(), - self.pk_indices.clone(), - self.is_append_only, - writer_param.executor_id, - self.format_desc.encode.clone(), - )? - .into_log_sinker(writer_param.sink_metrics)) + fn new_operator(properties: FsConfig) -> Result { + FileSink::::new_fs_sink(properties) } -} - -impl TryFrom for FsSink { - type Error = SinkError; - fn try_from(param: SinkParam) -> std::result::Result { - let schema = param.schema(); - let config = FsConfig::from_hashmap(param.properties)?; - Ok(Self { - config, - schema, - is_append_only: param.sink_type.is_append_only(), - pk_indices: param.downstream_pk, - format_desc: param - .format_desc - .ok_or_else(|| SinkError::Config(anyhow!("missing FORMAT ... ENCODE ...")))?, - }) + fn get_path(properties: &Self::Properties) -> String { + (*properties.common.path).to_string() } } diff --git a/src/connector/src/sink/opendal_sink/gcs.rs b/src/connector/src/sink/file_sink/gcs.rs similarity index 62% rename from src/connector/src/sink/opendal_sink/gcs.rs rename to src/connector/src/sink/file_sink/gcs.rs index cc04f3fc4c248..8e7f1aab7f1ec 100644 --- a/src/connector/src/sink/opendal_sink/gcs.rs +++ b/src/connector/src/sink/file_sink/gcs.rs @@ -22,12 +22,10 @@ use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; -use crate::sink::opendal_sink::OpenDalSinkWriter; +use super::opendal_sink::FileSink; +use crate::sink::file_sink::opendal_sink::OpendalSinkBackend; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; -use crate::sink::{ - DummySinkCommitCoordinator, Result, Sink, SinkError, SinkFormatDesc, SinkParam, - SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, -}; +use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; const GCS_WRITE_BUFFER_SIZE: usize = 16 * 1024 * 1024; #[derive(Deserialize, Debug, Clone, WithOptions)] @@ -61,32 +59,7 @@ pub struct GcsConfig { pub const GCS_SINK: &str = "gcs"; -impl GcsConfig { - pub fn from_hashmap(properties: HashMap) -> Result { - let config = serde_json::from_value::(serde_json::to_value(properties).unwrap()) - .map_err(|e| SinkError::Config(anyhow!(e)))?; - if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { - return Err(SinkError::Config(anyhow!( - "`{}` must be {}, or {}", - SINK_TYPE_OPTION, - SINK_TYPE_APPEND_ONLY, - SINK_TYPE_UPSERT - ))); - } - Ok(config) - } -} - -#[derive(Debug)] -pub struct GcsSink { - pub config: GcsConfig, - schema: Schema, - pk_indices: Vec, - is_append_only: bool, - format_desc: SinkFormatDesc, -} - -impl GcsSink { +impl FileSink { pub fn new_gcs_sink(config: GcsConfig) -> Result { // Create gcs builder. let mut builder = Gcs::default(); @@ -114,50 +87,33 @@ impl GcsSink { } } -impl Sink for GcsSink { - type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf; +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct GcsSink; + +impl OpendalSinkBackend for GcsSink { + type Properties = GcsConfig; const SINK_NAME: &'static str = GCS_SINK; - async fn validate(&self) -> Result<()> { - let _op = Self::new_gcs_sink(self.config.clone())?; - Ok(()) + fn from_hashmap(hash_map: HashMap) -> Result { + let config = serde_json::from_value::(serde_json::to_value(hash_map).unwrap()) + .map_err(|e| SinkError::Config(anyhow!(e)))?; + if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { + return Err(SinkError::Config(anyhow!( + "`{}` must be {}, or {}", + SINK_TYPE_OPTION, + SINK_TYPE_APPEND_ONLY, + SINK_TYPE_UPSERT + ))); + } + Ok(config) } - async fn new_log_sinker( - &self, - writer_param: crate::sink::SinkWriterParam, - ) -> Result { - let op = Self::new_gcs_sink(self.config.clone())?; - let path = self.config.common.path.as_ref(); - Ok(OpenDalSinkWriter::new( - op, - path, - self.schema.clone(), - self.pk_indices.clone(), - self.is_append_only, - writer_param.executor_id, - self.format_desc.encode.clone(), - )? - .into_log_sinker(writer_param.sink_metrics)) + fn new_operator(properties: GcsConfig) -> Result { + FileSink::::new_gcs_sink(properties) } -} -impl TryFrom for GcsSink { - type Error = SinkError; - - fn try_from(param: SinkParam) -> std::result::Result { - let schema = param.schema(); - let config = GcsConfig::from_hashmap(param.properties)?; - Ok(Self { - config, - schema, - is_append_only: param.sink_type.is_append_only(), - pk_indices: param.downstream_pk, - format_desc: param - .format_desc - .ok_or_else(|| SinkError::Config(anyhow!("missing FORMAT ... ENCODE ...")))?, - }) + fn get_path(properties: &Self::Properties) -> String { + (*properties.common.path).to_string() } } diff --git a/src/connector/src/sink/opendal_sink/mod.rs b/src/connector/src/sink/file_sink/mod.rs similarity index 99% rename from src/connector/src/sink/opendal_sink/mod.rs rename to src/connector/src/sink/file_sink/mod.rs index f5c30cf0278c2..cdaa3b30c33ad 100644 --- a/src/connector/src/sink/opendal_sink/mod.rs +++ b/src/connector/src/sink/file_sink/mod.rs @@ -14,6 +14,7 @@ pub mod fs; pub mod gcs; +pub mod opendal_sink; pub mod s3; use std::collections::HashMap; use std::sync::Arc; @@ -118,7 +119,7 @@ impl OpenDalSinkWriter { }) } - async fn create_object_writer(&mut self, epoch: u64) -> Result<(OpendalWriter)> { + async fn create_object_writer(&mut self, epoch: u64) -> Result { let suffix = match self.encode_type { SinkEncode::Json => "json", SinkEncode::Parquet => "parquet", diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs new file mode 100644 index 0000000000000..4215110ac60fa --- /dev/null +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -0,0 +1,95 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::marker::PhantomData; + +use anyhow::anyhow; +use opendal::Operator; +use risingwave_common::catalog::Schema; + +use crate::sink::file_sink::OpenDalSinkWriter; +use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; +use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkError, SinkFormatDesc, SinkParam}; + +#[derive(Debug, Clone)] +pub struct FileSink { + pub(crate) op: Operator, + pub(crate) path: String, + // prefix is used to reduce the number of objects to be listed + pub(crate) schema: Schema, + pub(crate) is_append_only: bool, + pub(crate) pk_indices: Vec, + pub(crate) format_desc: SinkFormatDesc, + pub(crate) marker: PhantomData, +} + +pub trait OpendalSinkBackend: Send + Sync + 'static + Clone + PartialEq { + type Properties: Send + Sync; + const SINK_NAME: &'static str; + + fn from_hashmap(hash_map: HashMap) -> Result; + fn new_operator(properties: Self::Properties) -> Result; + fn get_path(properties: &Self::Properties) -> String; +} + +impl Sink for FileSink { + type Coordinator = DummySinkCommitCoordinator; + type LogSinker = LogSinkerOf; + + const SINK_NAME: &'static str = S::SINK_NAME; + + async fn validate(&self) -> Result<()> { + Ok(()) + } + + async fn new_log_sinker( + &self, + writer_param: crate::sink::SinkWriterParam, + ) -> Result { + Ok(OpenDalSinkWriter::new( + self.op.clone(), + &self.path, + self.schema.clone(), + self.pk_indices.clone(), + self.is_append_only, + writer_param.executor_id, + self.format_desc.encode.clone(), + )? + .into_log_sinker(writer_param.sink_metrics)) + } +} + +impl TryFrom for FileSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let schema = param.schema(); + let config = S::from_hashmap(param.properties)?; + let path = S::get_path(&config); + let op = S::new_operator(config)?; + + Ok(Self { + op, + path, + schema, + is_append_only: param.sink_type.is_append_only(), + pk_indices: param.downstream_pk, + format_desc: param + .format_desc + .ok_or_else(|| SinkError::Config(anyhow!("missing FORMAT ... ENCODE ...")))?, + marker: PhantomData, + }) + } +} diff --git a/src/connector/src/sink/opendal_sink/s3.rs b/src/connector/src/sink/file_sink/s3.rs similarity index 63% rename from src/connector/src/sink/opendal_sink/s3.rs rename to src/connector/src/sink/file_sink/s3.rs index eace046e90c4f..ebb4107ef9a3d 100644 --- a/src/connector/src/sink/opendal_sink/s3.rs +++ b/src/connector/src/sink/file_sink/s3.rs @@ -22,13 +22,10 @@ use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; -use crate::sink::opendal_sink::OpenDalSinkWriter; -use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; -use crate::sink::{ - DummySinkCommitCoordinator, Result, Sink, SinkError, SinkFormatDesc, SinkParam, - SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, -}; - +use super::opendal_sink::FileSink; +use crate::sink::file_sink::opendal_sink::OpendalSinkBackend; +use crate::sink::file_sink::OpenDalSinkWriter; +use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; #[derive(Deserialize, Debug, Clone, WithOptions)] pub struct S3Common { #[serde(rename = "s3.region_name")] @@ -58,32 +55,7 @@ pub struct S3Config { pub const S3_SINK: &str = "s3"; -impl S3Config { - pub fn from_hashmap(properties: HashMap) -> Result { - let config = serde_json::from_value::(serde_json::to_value(properties).unwrap()) - .map_err(|e| SinkError::Config(anyhow!(e)))?; - if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { - return Err(SinkError::Config(anyhow!( - "`{}` must be {}, or {}", - SINK_TYPE_OPTION, - SINK_TYPE_APPEND_ONLY, - SINK_TYPE_UPSERT - ))); - } - Ok(config) - } -} - -#[derive(Debug)] -pub struct S3Sink { - pub config: S3Config, - schema: Schema, - is_append_only: bool, - pk_indices: Vec, - format_desc: SinkFormatDesc, -} - -impl S3Sink { +impl FileSink { pub fn new_s3_sink(config: S3Config) -> Result { // Create s3 builder. let mut builder = S3::default(); @@ -125,50 +97,33 @@ impl S3Sink { } } -impl Sink for S3Sink { - type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf; +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct S3Sink; + +impl OpendalSinkBackend for S3Sink { + type Properties = S3Config; const SINK_NAME: &'static str = S3_SINK; - async fn validate(&self) -> Result<()> { - let _op = Self::new_s3_sink(self.config.clone())?; - Ok(()) + fn from_hashmap(hash_map: HashMap) -> Result { + let config = serde_json::from_value::(serde_json::to_value(hash_map).unwrap()) + .map_err(|e| SinkError::Config(anyhow!(e)))?; + if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { + return Err(SinkError::Config(anyhow!( + "`{}` must be {}, or {}", + SINK_TYPE_OPTION, + SINK_TYPE_APPEND_ONLY, + SINK_TYPE_UPSERT + ))); + } + Ok(config) } - async fn new_log_sinker( - &self, - writer_param: crate::sink::SinkWriterParam, - ) -> Result { - let op = Self::new_s3_sink(self.config.clone())?; - let path = self.config.common.path.as_ref(); - Ok(OpenDalSinkWriter::new( - op, - path, - self.schema.clone(), - self.pk_indices.clone(), - self.is_append_only, - writer_param.executor_id, - self.format_desc.encode.clone(), - )? - .into_log_sinker(writer_param.sink_metrics)) + fn new_operator(properties: S3Config) -> Result { + FileSink::::new_s3_sink(properties) } -} -impl TryFrom for S3Sink { - type Error = SinkError; - - fn try_from(param: SinkParam) -> std::result::Result { - let schema = param.schema(); - let config = S3Config::from_hashmap(param.properties)?; - Ok(Self { - config, - schema, - is_append_only: param.sink_type.is_append_only(), - pk_indices: param.downstream_pk, - format_desc: param - .format_desc - .ok_or_else(|| SinkError::Config(anyhow!("missing FORMAT ... ENCODE ...")))?, - }) + fn get_path(properties: &Self::Properties) -> String { + (*properties.common.path).to_string() } } diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 0d8975683d00f..488fa2271d95d 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -11,7 +11,6 @@ // 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 big_query; pub mod boxed; pub mod catalog; @@ -22,6 +21,7 @@ pub mod doris; pub mod doris_starrocks_connector; pub mod elasticsearch; pub mod encoder; +pub mod file_sink; pub mod formatter; pub mod iceberg; pub mod kafka; @@ -29,7 +29,6 @@ pub mod kinesis; pub mod log_store; pub mod mock_coordination_client; pub mod nats; -pub mod opendal_sink; pub mod pulsar; pub mod redis; pub mod remote; @@ -67,10 +66,11 @@ use self::mock_coordination_client::{MockMetaClient, SinkCoordinationRpcClientEn use crate::error::ConnectorError; use crate::sink::catalog::desc::SinkDesc; use crate::sink::catalog::{SinkCatalog, SinkId}; +use crate::sink::file_sink::gcs::GcsSink; +use crate::sink::file_sink::s3::S3Sink; use crate::sink::log_store::{LogReader, LogStoreReadItem, LogStoreResult, TruncateOffset}; use crate::sink::writer::SinkWriter; use crate::ConnectorParams; - const BOUNDED_CHANNEL_SIZE: usize = 16; #[macro_export] macro_rules! for_all_sinks { @@ -91,8 +91,8 @@ macro_rules! for_all_sinks { { HttpJava, $crate::sink::remote::HttpJavaSink }, { Doris, $crate::sink::doris::DorisSink }, { Starrocks, $crate::sink::starrocks::StarrocksSink }, - { S3, $crate::sink::opendal_sink::s3::S3Sink }, - { Gcs, $crate::sink::opendal_sink::gcs::GcsSink }, + { S3, $crate::sink::file_sink::opendal_sink::FileSink }, + { Gcs, $crate::sink::file_sink::opendal_sink::FileSink }, { DeltaLake, $crate::sink::deltalake::DeltaLakeSink }, { BigQuery, $crate::sink::big_query::BigQuerySink }, { Test, $crate::sink::test_sink::TestSink }, diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index 9cce3b2716c65..36366f9207383 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -758,9 +758,11 @@ fn bind_sink_format_desc(value: ConnectorSchema) -> Result { static CONNECTORS_COMPATIBLE_FORMATS: LazyLock>>> = LazyLock::new(|| { + use risingwave_connector::sink::file_sink::gcs::GcsSink; + use risingwave_connector::sink::file_sink::opendal_sink::FileSink; + use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::kafka::KafkaSink; use risingwave_connector::sink::kinesis::KinesisSink; - use risingwave_connector::sink::opendal_sink::s3::S3Sink; use risingwave_connector::sink::pulsar::PulsarSink; use risingwave_connector::sink::redis::RedisSink; use risingwave_connector::sink::Sink as _; @@ -771,7 +773,10 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock vec![Encode::Json, Encode::Avro], Format::Debezium => vec![Encode::Json], ), - S3Sink::SINK_NAME => hashmap!( + FileSink::::SINK_NAME => hashmap!( + Format::Plain => vec![Encode::Json, Encode::Parquet], + ), + FileSink::::SINK_NAME => hashmap!( Format::Plain => vec![Encode::Json, Encode::Parquet], ), KinesisSink::SINK_NAME => hashmap!( diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index ffa25e3127a81..35c6d67c7c3e1 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -30,6 +30,8 @@ use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::desc::SinkDesc; use risingwave_connector::sink::catalog::{SinkFormat, SinkFormatDesc, SinkId, SinkType}; +use risingwave_connector::sink::file_sink::gcs::GcsSink; +use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::iceberg::ICEBERG_SINK; use risingwave_connector::sink::trivial::TABLE_SINK; use risingwave_connector::sink::{ diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs index 82e97342eea82..ee9f43be439bf 100644 --- a/src/stream/src/from_proto/sink.rs +++ b/src/stream/src/from_proto/sink.rs @@ -19,6 +19,8 @@ use risingwave_common::catalog::{ColumnCatalog, Schema}; use risingwave_common::types::DataType; use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::{SinkFormatDesc, SinkType}; +use risingwave_connector::sink::file_sink::gcs::GcsSink; +use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::{ SinkError, SinkMetaClient, SinkParam, SinkWriterParam, CONNECTOR_TYPE_KEY, SINK_TYPE_OPTION, }; From da6a4dd4ec352d301a8551c8a2d8a4b04ebb448d Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 19 Jun 2024 18:32:04 +0800 Subject: [PATCH 16/76] add fs sink for test --- src/connector/src/sink/mod.rs | 2 ++ src/frontend/src/handler/create_sink.rs | 4 ++++ src/frontend/src/optimizer/plan_node/stream_sink.rs | 1 + src/stream/src/from_proto/sink.rs | 1 + 4 files changed, 8 insertions(+) diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 488fa2271d95d..7285c78150c24 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -66,6 +66,7 @@ use self::mock_coordination_client::{MockMetaClient, SinkCoordinationRpcClientEn use crate::error::ConnectorError; use crate::sink::catalog::desc::SinkDesc; use crate::sink::catalog::{SinkCatalog, SinkId}; +use crate::sink::file_sink::fs::FsSink; use crate::sink::file_sink::gcs::GcsSink; use crate::sink::file_sink::s3::S3Sink; use crate::sink::log_store::{LogReader, LogStoreReadItem, LogStoreResult, TruncateOffset}; @@ -93,6 +94,7 @@ macro_rules! for_all_sinks { { Starrocks, $crate::sink::starrocks::StarrocksSink }, { S3, $crate::sink::file_sink::opendal_sink::FileSink }, { Gcs, $crate::sink::file_sink::opendal_sink::FileSink }, + { Fs, $crate::sink::file_sink::opendal_sink::FileSink }, { DeltaLake, $crate::sink::deltalake::DeltaLakeSink }, { BigQuery, $crate::sink::big_query::BigQuerySink }, { Test, $crate::sink::test_sink::TestSink }, diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index 36366f9207383..f5be708a3406b 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -758,6 +758,7 @@ fn bind_sink_format_desc(value: ConnectorSchema) -> Result { static CONNECTORS_COMPATIBLE_FORMATS: LazyLock>>> = LazyLock::new(|| { + use risingwave_connector::sink::file_sink::fs::FsSink; use risingwave_connector::sink::file_sink::gcs::GcsSink; use risingwave_connector::sink::file_sink::opendal_sink::FileSink; use risingwave_connector::sink::file_sink::s3::S3Sink; @@ -779,6 +780,9 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock::SINK_NAME => hashmap!( Format::Plain => vec![Encode::Json, Encode::Parquet], ), + FileSink::::SINK_NAME => hashmap!( + Format::Plain => vec![Encode::Json, Encode::Parquet], + ), KinesisSink::SINK_NAME => hashmap!( Format::Plain => vec![Encode::Json], Format::Upsert => vec![Encode::Json], diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 35c6d67c7c3e1..935db3151348a 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -30,6 +30,7 @@ use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::desc::SinkDesc; use risingwave_connector::sink::catalog::{SinkFormat, SinkFormatDesc, SinkId, SinkType}; +use risingwave_connector::sink::file_sink::fs::FsSink; use risingwave_connector::sink::file_sink::gcs::GcsSink; use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::iceberg::ICEBERG_SINK; diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs index ee9f43be439bf..b8f87f59eb7fe 100644 --- a/src/stream/src/from_proto/sink.rs +++ b/src/stream/src/from_proto/sink.rs @@ -19,6 +19,7 @@ use risingwave_common::catalog::{ColumnCatalog, Schema}; use risingwave_common::types::DataType; use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::{SinkFormatDesc, SinkType}; +use risingwave_connector::sink::file_sink::fs::FsSink; use risingwave_connector::sink::file_sink::gcs::GcsSink; use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::{ From ac785be6db57a0c40f58ede5b63dd34f5fa24510 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 20 Jun 2024 15:55:42 +0800 Subject: [PATCH 17/76] add comments --- src/connector/src/sink/file_sink/fs.rs | 4 +- src/connector/src/sink/file_sink/gcs.rs | 3 +- src/connector/src/sink/file_sink/mod.rs | 43 +++++++++++++------ .../src/sink/file_sink/opendal_sink.rs | 28 +++++++++++- src/connector/src/sink/file_sink/s3.rs | 3 +- 5 files changed, 60 insertions(+), 21 deletions(-) diff --git a/src/connector/src/sink/file_sink/fs.rs b/src/connector/src/sink/file_sink/fs.rs index f0c5ef01726d5..41eb5f3d86cb8 100644 --- a/src/connector/src/sink/file_sink/fs.rs +++ b/src/connector/src/sink/file_sink/fs.rs @@ -17,18 +17,16 @@ use anyhow::anyhow; use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::Fs; use opendal::Operator; -use risingwave_common::catalog::Schema; use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; use crate::sink::file_sink::opendal_sink::{FileSink, OpendalSinkBackend}; -use crate::sink::file_sink::OpenDalSinkWriter; -use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; #[derive(Deserialize, Debug, Clone, WithOptions)] pub struct FsCommon { + /// The directory where the sink file is located. #[serde(rename = "fs.path", default)] pub path: String, } diff --git a/src/connector/src/sink/file_sink/gcs.rs b/src/connector/src/sink/file_sink/gcs.rs index 8e7f1aab7f1ec..37890fe8cfd27 100644 --- a/src/connector/src/sink/file_sink/gcs.rs +++ b/src/connector/src/sink/file_sink/gcs.rs @@ -17,14 +17,12 @@ use anyhow::anyhow; use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::Gcs; use opendal::Operator; -use risingwave_common::catalog::Schema; use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; use super::opendal_sink::FileSink; use crate::sink::file_sink::opendal_sink::OpendalSinkBackend; -use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; const GCS_WRITE_BUFFER_SIZE: usize = 16 * 1024 * 1024; @@ -41,6 +39,7 @@ pub struct GcsCommon { #[serde(rename = "gcs.service_account", default)] pub service_account: Option, + /// The directory where the sink file is located #[serde(rename = "gcs.path", default)] pub path: String, diff --git a/src/connector/src/sink/file_sink/mod.rs b/src/connector/src/sink/file_sink/mod.rs index cdaa3b30c33ad..72c980da85f36 100644 --- a/src/connector/src/sink/file_sink/mod.rs +++ b/src/connector/src/sink/file_sink/mod.rs @@ -47,8 +47,21 @@ pub struct OpenDalSinkWriter { encode_type: SinkEncode, } +/// The `FileWriterEnum` enum represents different types of file writers used for various sink +/// implementations. +/// +/// # Variants +/// +/// - `ParquetFileWriter`: Represents a Parquet file writer using the `AsyncArrowWriter` +/// for writing data to a Parquet file. It accepts an implementation of W: `AsyncWrite` + `Unpin` + `Send` +/// as the underlying writer. In this case, the `OpendalWriter` serves as the underlying writer. +/// +/// - `FileWriter`: Represents a file writer for sinks other than Parquet. It uses the `OpendalWriter` +/// directly for writing data to the file. +/// +/// The choice of writer used during the actual writing process depends on the encode type of the sink. enum FileWriterEnum { - ParquetWriter(AsyncArrowWriter), + ParquetFileWriter(AsyncArrowWriter), FileWriter(OpendalWriter), } @@ -77,10 +90,13 @@ impl SinkWriter for OpenDalSinkWriter { Ok(()) } + /// For the file sink, currently, the sink decoupling feature is not enabled. + /// When a checkpoint arrives, the force commit is performed to write the data to the file. + /// In the future if flush and checkpoint is decoupled, we should enable sink decouple accordingly. async fn barrier(&mut self, is_checkpoint: bool) -> Result<()> { if is_checkpoint && let Some(sink_writer) = self.sink_writer.take() { match sink_writer { - FileWriterEnum::ParquetWriter(mut w) => { + FileWriterEnum::ParquetFileWriter(w) => { let _ = w.close().await?; } FileWriterEnum::FileWriter(mut w) => w.close().await?, @@ -120,17 +136,16 @@ impl OpenDalSinkWriter { } async fn create_object_writer(&mut self, epoch: u64) -> Result { + // todo: specify more file suffixes based on encode_type. let suffix = match self.encode_type { SinkEncode::Json => "json", SinkEncode::Parquet => "parquet", _ => unimplemented!(), }; + let object_name = format!( "{}/epoch_{}_executor_{}.{}", - self.write_path, - epoch.to_string(), - self.executor_id.to_string(), - suffix.to_string(), + self.write_path, epoch, self.executor_id, suffix, ); Ok(self .operator @@ -155,12 +170,14 @@ impl OpenDalSinkWriter { if let Some(created_by) = parquet_config.created_by.as_ref() { props = props.set_created_by(created_by.to_string()); } - self.sink_writer = Some(FileWriterEnum::ParquetWriter(AsyncArrowWriter::try_new( - object_writer, - self.schema.clone(), - SINK_WRITE_BUFFER_SIZE, - Some(props.build()), - )?)); + self.sink_writer = Some(FileWriterEnum::ParquetFileWriter( + AsyncArrowWriter::try_new( + object_writer, + self.schema.clone(), + SINK_WRITE_BUFFER_SIZE, + Some(props.build()), + )?, + )); } SinkEncode::Json => { self.sink_writer = Some(FileWriterEnum::FileWriter(object_writer)); @@ -182,7 +199,7 @@ impl OpenDalSinkWriter { .as_mut() .ok_or_else(|| SinkError::Opendal("Sink writer is not created.".to_string()))? { - FileWriterEnum::ParquetWriter(w) => { + FileWriterEnum::ParquetFileWriter(w) => { let batch = to_record_batch_with_schema(self.schema.clone(), &chunk.compact())?; w.write(&batch).await?; } diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index 4215110ac60fa..822c482d6e769 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -23,18 +23,44 @@ use crate::sink::file_sink::OpenDalSinkWriter; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkError, SinkFormatDesc, SinkParam}; +/// The `FileSink` struct represents a file sink that uses the `OpendalSinkBackend` trait +/// for its backend implementation. +/// +/// # Type Parameters +/// +/// - S: The type parameter S represents the concrete implementation of the `OpendalSinkBackend` +/// trait used by this file sink. #[derive(Debug, Clone)] pub struct FileSink { pub(crate) op: Operator, + /// The path to the file where the sink writes data. pub(crate) path: String, - // prefix is used to reduce the number of objects to be listed + /// The schema describing the structure of the data being written to the file sink. pub(crate) schema: Schema, pub(crate) is_append_only: bool, + /// A vector of indices representing the primary key columns in the schema. pub(crate) pk_indices: Vec, + /// The description of the sink's format. pub(crate) format_desc: SinkFormatDesc, pub(crate) marker: PhantomData, } +/// The `OpendalSinkBackend` trait unifies the behavior of various sink backends +/// implemented through OpenDAL(https://github.com/apache/opendal). +/// +/// # Type Parameters +/// +/// - Properties: Represents the necessary parameters for establishing a backend. +/// +/// # Constants +/// +/// - `SINK_NAME`: A static string representing the name of the sink. +/// +/// # Functions +/// +/// - `from_hashmap`: Automatically parse the required parameters from the input create sink statement. +/// - `new_operator`: Creates a new operator using the provided backend properties. +/// - `get_path`: Returns the path of the sink file specified by the user's create sink statement. pub trait OpendalSinkBackend: Send + Sync + 'static + Clone + PartialEq { type Properties: Send + Sync; const SINK_NAME: &'static str; diff --git a/src/connector/src/sink/file_sink/s3.rs b/src/connector/src/sink/file_sink/s3.rs index ebb4107ef9a3d..eaf85811c0e54 100644 --- a/src/connector/src/sink/file_sink/s3.rs +++ b/src/connector/src/sink/file_sink/s3.rs @@ -17,14 +17,12 @@ use anyhow::anyhow; use opendal::layers::{LoggingLayer, RetryLayer}; use opendal::services::S3; use opendal::Operator; -use risingwave_common::catalog::Schema; use serde::Deserialize; use serde_with::serde_as; use with_options::WithOptions; use super::opendal_sink::FileSink; use crate::sink::file_sink::opendal_sink::OpendalSinkBackend; -use crate::sink::file_sink::OpenDalSinkWriter; use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; #[derive(Deserialize, Debug, Clone, WithOptions)] pub struct S3Common { @@ -32,6 +30,7 @@ pub struct S3Common { pub region_name: String, #[serde(rename = "s3.bucket_name")] pub bucket_name: String, + /// The directory where the sink file is located. #[serde(rename = "s3.path", default)] pub path: String, #[serde(rename = "s3.credentials.access", default)] From a3c64494223864fc5d12a54b929f4f2b234e285c Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 20 Jun 2024 16:14:27 +0800 Subject: [PATCH 18/76] minor for parquet change --- src/connector/src/sink/file_sink/mod.rs | 15 ++------------- 1 file changed, 2 insertions(+), 13 deletions(-) diff --git a/src/connector/src/sink/file_sink/mod.rs b/src/connector/src/sink/file_sink/mod.rs index 72c980da85f36..cedc28ee50c2a 100644 --- a/src/connector/src/sink/file_sink/mod.rs +++ b/src/connector/src/sink/file_sink/mod.rs @@ -22,10 +22,9 @@ use std::sync::Arc; use anyhow::anyhow; use arrow_schema::{DataType as ArrowDataType, Field as ArrowField, SchemaRef}; use async_trait::async_trait; -use icelake::config::ParquetWriterConfig; use opendal::{Operator, Writer as OpendalWriter}; use parquet::arrow::AsyncArrowWriter; -use parquet::file::properties::{WriterProperties, WriterVersion}; +use parquet::file::properties::WriterProperties; use risingwave_common::array::{to_record_batch_with_schema, Op, StreamChunk}; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::Schema; @@ -159,17 +158,7 @@ impl OpenDalSinkWriter { let object_writer = self.create_object_writer(epoch).await?; match self.encode_type { SinkEncode::Parquet => { - let parquet_config = ParquetWriterConfig::default(); - let mut props = WriterProperties::builder() - .set_writer_version(WriterVersion::PARQUET_1_0) - .set_bloom_filter_enabled(parquet_config.enable_bloom_filter) - .set_compression(parquet_config.compression) - .set_max_row_group_size(parquet_config.max_row_group_size) - .set_write_batch_size(parquet_config.write_batch_size) - .set_data_page_size_limit(parquet_config.data_page_size); - if let Some(created_by) = parquet_config.created_by.as_ref() { - props = props.set_created_by(created_by.to_string()); - } + let props = WriterProperties::builder(); self.sink_writer = Some(FileWriterEnum::ParquetFileWriter( AsyncArrowWriter::try_new( object_writer, From ac20951e584ff149ea494f06a8dc68810f8e3547 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 24 Jun 2024 09:42:33 +0800 Subject: [PATCH 19/76] todo: upgrade to opendal 0.47 --- src/connector/src/sink/file_sink/mod.rs | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/connector/src/sink/file_sink/mod.rs b/src/connector/src/sink/file_sink/mod.rs index cedc28ee50c2a..f2acc224ccb95 100644 --- a/src/connector/src/sink/file_sink/mod.rs +++ b/src/connector/src/sink/file_sink/mod.rs @@ -32,6 +32,7 @@ use risingwave_common::catalog::Schema; use crate::sink::catalog::SinkEncode; use crate::sink::{Result, SinkError, SinkWriter}; +// Todo(wcy-fdu): upgrade to opendal 0.47. const SINK_WRITE_BUFFER_SIZE: usize = 16 * 1024 * 1024; pub struct OpenDalSinkWriter { @@ -67,6 +68,8 @@ enum FileWriterEnum { #[async_trait] impl SinkWriter for OpenDalSinkWriter { async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { + // Note: epoch is used to name the output files. + // Todo: after enabling sink decouple, use the new naming convention. let epoch = self.epoch.ok_or_else(|| { SinkError::Opendal("epoch has not been initialize, call `begin_epoch`".to_string()) })?; @@ -76,6 +79,7 @@ impl SinkWriter for OpenDalSinkWriter { if self.is_append_only { self.append_only(chunk).await } else { + // currently file sink only supports append only mode. unimplemented!() } } @@ -135,13 +139,15 @@ impl OpenDalSinkWriter { } async fn create_object_writer(&mut self, epoch: u64) -> Result { - // todo: specify more file suffixes based on encode_type. + // Todo: specify more file suffixes based on encode_type. let suffix = match self.encode_type { SinkEncode::Json => "json", SinkEncode::Parquet => "parquet", _ => unimplemented!(), }; + // Note: sink decoupling is not currently supported, which means that output files will not be batched across checkpoints. + // The current implementation writes files every time a checkpoint arrives, so the naming convention is `epoch + executor_id + .suffix`. let object_name = format!( "{}/epoch_{}_executor_{}.{}", self.write_path, epoch, self.executor_id, suffix, @@ -170,6 +176,7 @@ impl OpenDalSinkWriter { } SinkEncode::Json => { self.sink_writer = Some(FileWriterEnum::FileWriter(object_writer)); + unimplemented!(); } _ => unimplemented!(), } From f5559511eb7a3d026b3dd43fdd415f25925001ae Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 9 Jul 2024 10:40:39 +0800 Subject: [PATCH 20/76] remove json encode, minor refactor --- src/connector/src/sink/encoder/mod.rs | 2 -- src/connector/src/sink/file_sink/mod.rs | 15 +++++++-------- src/connector/src/sink/mod.rs | 10 +++++----- src/frontend/src/handler/create_sink.rs | 6 +++--- 4 files changed, 15 insertions(+), 18 deletions(-) diff --git a/src/connector/src/sink/encoder/mod.rs b/src/connector/src/sink/encoder/mod.rs index 1ba0288cc517e..3254447e27077 100644 --- a/src/connector/src/sink/encoder/mod.rs +++ b/src/connector/src/sink/encoder/mod.rs @@ -22,13 +22,11 @@ use crate::sink::Result; mod avro; mod json; -// mod parquet; mod proto; pub mod template; pub use avro::{AvroEncoder, AvroHeader}; pub use json::JsonEncoder; -// pub use parquet::ParquetEncoder; pub use proto::ProtoEncoder; /// Encode a row of a relation into diff --git a/src/connector/src/sink/file_sink/mod.rs b/src/connector/src/sink/file_sink/mod.rs index f2acc224ccb95..0632e73e8e35c 100644 --- a/src/connector/src/sink/file_sink/mod.rs +++ b/src/connector/src/sink/file_sink/mod.rs @@ -71,7 +71,7 @@ impl SinkWriter for OpenDalSinkWriter { // Note: epoch is used to name the output files. // Todo: after enabling sink decouple, use the new naming convention. let epoch = self.epoch.ok_or_else(|| { - SinkError::Opendal("epoch has not been initialize, call `begin_epoch`".to_string()) + SinkError::File("epoch has not been initialize, call `begin_epoch`".to_string()) })?; if self.sink_writer.is_none() { self.create_sink_writer(epoch).await?; @@ -141,7 +141,6 @@ impl OpenDalSinkWriter { async fn create_object_writer(&mut self, epoch: u64) -> Result { // Todo: specify more file suffixes based on encode_type. let suffix = match self.encode_type { - SinkEncode::Json => "json", SinkEncode::Parquet => "parquet", _ => unimplemented!(), }; @@ -149,7 +148,7 @@ impl OpenDalSinkWriter { // Note: sink decoupling is not currently supported, which means that output files will not be batched across checkpoints. // The current implementation writes files every time a checkpoint arrives, so the naming convention is `epoch + executor_id + .suffix`. let object_name = format!( - "{}/epoch_{}_executor_{}.{}", + "{}/{}_{}.{}", self.write_path, epoch, self.executor_id, suffix, ); Ok(self @@ -174,10 +173,10 @@ impl OpenDalSinkWriter { )?, )); } - SinkEncode::Json => { - self.sink_writer = Some(FileWriterEnum::FileWriter(object_writer)); - unimplemented!(); - } + // SinkEncode::Json => { + // self.sink_writer = Some(FileWriterEnum::FileWriter(object_writer)); + // unimplemented!(); + // } _ => unimplemented!(), } @@ -193,7 +192,7 @@ impl OpenDalSinkWriter { match self .sink_writer .as_mut() - .ok_or_else(|| SinkError::Opendal("Sink writer is not created.".to_string()))? + .ok_or_else(|| SinkError::File("Sink writer is not created.".to_string()))? { FileWriterEnum::ParquetFileWriter(w) => { let batch = to_record_batch_with_schema(self.schema.clone(), &chunk.compact())?; diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 7285c78150c24..64eebcf530dc2 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -532,8 +532,8 @@ pub enum SinkError { ), #[error("Starrocks error: {0}")] Starrocks(String), - #[error("Opendal error: {0}")] - Opendal(String), + #[error("File error: {0}")] + File(String), #[error("Pulsar error: {0}")] Pulsar( #[source] @@ -568,19 +568,19 @@ impl From for SinkError { impl From for SinkError { fn from(error: OpendalError) -> Self { - SinkError::Opendal(error.to_string()) + SinkError::File(error.to_string()) } } impl From for SinkError { fn from(error: parquet::errors::ParquetError) -> Self { - SinkError::Opendal(error.to_string()) + SinkError::File(error.to_string()) } } impl From for SinkError { fn from(error: ArrayError) -> Self { - SinkError::Opendal(error.to_string()) + SinkError::File(error.to_string()) } } diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index f5be708a3406b..063c20efb70fb 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -775,13 +775,13 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock vec![Encode::Json], ), FileSink::::SINK_NAME => hashmap!( - Format::Plain => vec![Encode::Json, Encode::Parquet], + Format::Plain => vec![Encode::Parquet], ), FileSink::::SINK_NAME => hashmap!( - Format::Plain => vec![Encode::Json, Encode::Parquet], + Format::Plain => vec![Encode::Parquet], ), FileSink::::SINK_NAME => hashmap!( - Format::Plain => vec![Encode::Json, Encode::Parquet], + Format::Plain => vec![Encode::Parquet], ), KinesisSink::SINK_NAME => hashmap!( Format::Plain => vec![Encode::Json], From 4fccf52da9d0972dc08c8a486e1999e9bfb68c31 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 9 Jul 2024 17:45:00 +0800 Subject: [PATCH 21/76] fmt and clippy --- proto/connector_service.proto | 2 - proto/plan_common.proto | 2 +- .../src/sink/file_sink/opendal_sink.rs | 2 +- src/connector/src/sink/mod.rs | 6 +- src/connector/with_options_sink.yaml | 63 +++++++++++++++++++ 5 files changed, 68 insertions(+), 7 deletions(-) diff --git a/proto/connector_service.proto b/proto/connector_service.proto index e31a4ab938967..da2c2b88087ea 100644 --- a/proto/connector_service.proto +++ b/proto/connector_service.proto @@ -29,8 +29,6 @@ message SinkParam { string sink_name = 8; } - - message SinkWriterStreamRequest { message StartSink { SinkParam sink_param = 1; diff --git a/proto/plan_common.proto b/proto/plan_common.proto index ea6cc035b7519..a99ecde38375d 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -150,7 +150,7 @@ enum EncodeType { ENCODE_TYPE_TEMPLATE = 7; ENCODE_TYPE_NONE = 8; ENCODE_TYPE_TEXT = 9; - ENCODE_TYPE_PARQUET= 10; + ENCODE_TYPE_PARQUET = 10; } enum RowFormatType { diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index c426444ad922d..596e0617a006e 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -45,7 +45,7 @@ pub struct FileSink { } /// The `OpendalSinkBackend` trait unifies the behavior of various sink backends -/// implemented through OpenDAL(https://github.com/apache/opendal). +/// implemented through OpenDAL(``). /// /// # Type Parameters /// diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 28c138a896552..145c291561570 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -623,19 +623,19 @@ impl From for SinkError { impl From for SinkError { fn from(error: OpendalError) -> Self { - SinkError::File(error.to_string()) + SinkError::File(error.to_report_string()) } } impl From for SinkError { fn from(error: parquet::errors::ParquetError) -> Self { - SinkError::File(error.to_string()) + SinkError::File(error.to_report_string()) } } impl From for SinkError { fn from(error: ArrayError) -> Self { - SinkError::File(error.to_string()) + SinkError::File(error.to_report_string()) } } diff --git a/src/connector/with_options_sink.yaml b/src/connector/with_options_sink.yaml index 27362900dd054..281664b11bf42 100644 --- a/src/connector/with_options_sink.yaml +++ b/src/connector/with_options_sink.yaml @@ -208,6 +208,38 @@ DynamoDbConfig: required: false alias: - profile +FsConfig: + fields: + - name: fs.path + field_type: String + comments: The directory where the sink file is located. + required: false + default: Default::default + - name: r#type + field_type: String + required: true +GcsConfig: + fields: + - name: gcs.bucket_name + field_type: String + required: true + - name: gcs.credential + field_type: String + comments: The base64 encoded credential key. If not set, ADC will be used. + required: false + - name: gcs.service_account + field_type: String + comments: If credential/ADC is not set. The service account can be used to provide the credential info. + required: false + default: Default::default + - name: gcs.path + field_type: String + comments: The directory where the sink file is located + required: false + default: Default::default + - name: r#type + field_type: String + required: true GooglePubSubConfig: fields: - name: pubsub.project_id @@ -747,6 +779,37 @@ RedisConfig: - name: redis.url field_type: String required: true +S3Config: + fields: + - name: s3.region_name + field_type: String + required: true + - name: s3.bucket_name + field_type: String + required: true + - name: s3.path + field_type: String + comments: The directory where the sink file is located. + required: false + default: Default::default + - name: s3.credentials.access + field_type: String + required: false + default: Default::default + - name: s3.credentials.secret + field_type: String + required: false + default: Default::default + - name: s3.endpoint_url + field_type: String + required: false + - name: s3.assume_role + field_type: String + required: false + default: Default::default + - name: r#type + field_type: String + required: true SnowflakeConfig: fields: - name: snowflake.database From aeda6741220fbbc3aebd97d5dc25e4f78049d08c Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 10 Jul 2024 19:21:39 +0800 Subject: [PATCH 22/76] minor --- src/connector/src/sink/file_sink/mod.rs | 19 ++++++++----------- .../src/sink/file_sink/opendal_sink.rs | 2 +- 2 files changed, 9 insertions(+), 12 deletions(-) diff --git a/src/connector/src/sink/file_sink/mod.rs b/src/connector/src/sink/file_sink/mod.rs index 59ad1fd352484..f31658819c4a2 100644 --- a/src/connector/src/sink/file_sink/mod.rs +++ b/src/connector/src/sink/file_sink/mod.rs @@ -1,4 +1,3 @@ -use icelake::io_v2::track_writer::TrackWriter; // Copyright 2024 RisingWave Labs // // Licensed under the Apache License, Version 2.0 (the "License"); @@ -12,19 +11,20 @@ use icelake::io_v2::track_writer::TrackWriter; // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::{array::arrow::IcebergArrowConvert, bitmap::Bitmap}; +use risingwave_common::array::arrow::IcebergArrowConvert; +use risingwave_common::bitmap::Bitmap; +use tokio_util::compat::{Compat, FuturesAsyncWriteCompatExt}; pub mod fs; pub mod gcs; pub mod opendal_sink; pub mod s3; use std::collections::HashMap; -use std::sync::atomic::AtomicI64; use std::sync::Arc; use arrow_schema_iceberg::SchemaRef; use async_trait::async_trait; -use opendal::{Operator, Writer as OpendalWriter}; +use opendal::{FuturesAsyncWriter, Operator, Writer as OpendalWriter}; use parquet::arrow::AsyncArrowWriter; use parquet::file::properties::WriterProperties; use risingwave_common::array::{Op, StreamChunk}; @@ -58,7 +58,7 @@ pub struct OpenDalSinkWriter { /// /// The choice of writer used during the actual writing process depends on the encode type of the sink. enum FileWriterEnum { - ParquetFileWriter(AsyncArrowWriter), + ParquetFileWriter(AsyncArrowWriter>), } #[async_trait] @@ -156,14 +156,11 @@ impl OpenDalSinkWriter { match self.encode_type { SinkEncode::Parquet => { let props = WriterProperties::builder(); - let written_size = Arc::new(AtomicI64::new(0)); - let track_writer = TrackWriter::new( - object_writer.into_futures_async_write(), - written_size.clone(), - ); + let parquet_writer: tokio_util::compat::Compat = + object_writer.into_futures_async_write().compat_write(); self.sink_writer = Some(FileWriterEnum::ParquetFileWriter( AsyncArrowWriter::try_new( - track_writer, + parquet_writer, self.schema.clone(), Some(props.build()), )?, diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index 596e0617a006e..9e1a4ed787bae 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -45,7 +45,7 @@ pub struct FileSink { } /// The `OpendalSinkBackend` trait unifies the behavior of various sink backends -/// implemented through OpenDAL(``). +/// implemented through `OpenDAL`(``). /// /// # Type Parameters /// From e905920fd83dee884dca04d873d8fc2d5fd77636 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 11 Jul 2024 13:54:46 +0800 Subject: [PATCH 23/76] minor --- src/connector/src/sink/file_sink/mod.rs | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/src/connector/src/sink/file_sink/mod.rs b/src/connector/src/sink/file_sink/mod.rs index f31658819c4a2..4670c9f616914 100644 --- a/src/connector/src/sink/file_sink/mod.rs +++ b/src/connector/src/sink/file_sink/mod.rs @@ -85,9 +85,6 @@ impl SinkWriter for OpenDalSinkWriter { Ok(()) } - async fn abort(&mut self) -> Result<()> { - Ok(()) - } /// For the file sink, currently, the sink decoupling feature is not enabled. /// When a checkpoint arrives, the force commit is performed to write the data to the file. @@ -104,9 +101,7 @@ impl SinkWriter for OpenDalSinkWriter { Ok(()) } - async fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc) -> Result<()> { - Ok(()) - } + } impl OpenDalSinkWriter { From 1011412eb70c7f6ab497b1df465e7dcf3473117f Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 26 Jul 2024 14:44:47 +0800 Subject: [PATCH 24/76] add ci test --- ci/workflows/main-cron.yml | 4 +- ...ource.py => fs_parquet_source_and_sink.py} | 111 +++++++++- src/connector/src/sink/catalog/mod.rs | 7 +- src/connector/src/sink/file_sink/mod.rs | 3 - .../src/sink/file_sink/opendal_sink.rs | 196 +++++++++++++++++- src/frontend/src/handler/create_sink.rs | 3 +- 6 files changed, 305 insertions(+), 19 deletions(-) rename e2e_test/s3/{fs_parquet_source.py => fs_parquet_source_and_sink.py} (57%) diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 1f264957f3953..3cc5d2cf36ae4 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -508,9 +508,9 @@ steps: timeout_in_minutes: 25 retry: *auto-retry - - label: "S3_v2 source check on parquet file" + - label: "S3_v2 source and sink on parquet file" key: "s3-v2-source-check-parquet-file" - command: "ci/scripts/s3-source-test.sh -p ci-release -s fs_parquet_source.py" + command: "ci/scripts/s3-source-test.sh -p ci-release -s fs_parquet_source_and_sink.py" if: | !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-s3-source-tests" diff --git a/e2e_test/s3/fs_parquet_source.py b/e2e_test/s3/fs_parquet_source_and_sink.py similarity index 57% rename from e2e_test/s3/fs_parquet_source.py rename to e2e_test/s3/fs_parquet_source_and_sink.py index 64060928c7755..8dc0990b58530 100644 --- a/e2e_test/s3/fs_parquet_source.py +++ b/e2e_test/s3/fs_parquet_source_and_sink.py @@ -95,12 +95,109 @@ def _assert_eq(field, got, expect): _assert_eq('count(*)', result[0], total_rows) _assert_eq('sum(id)', result[1], (total_rows - 1) * total_rows / 2) - print('Test pass') + print('File source test pass!') - cur.execute(f'drop table {_table()}') cur.close() conn.close() +def do_sink(config, file_num, item_num_per_file, prefix): + conn = psycopg2.connect( + host="localhost", + port="4566", + user="root", + database="dev" + ) + + # Open a cursor to execute SQL statements + cur = conn.cursor() + + def _table(): + return 's3_test_parquet' + + # Execute a SELECT statement + cur.execute(f'''CREATE sink s1 as select + id, + name, + sex, + mark, + test_int, + test_real, + test_double_precision, + test_varchar, + test_bytea, + test_date, + test_time, + test_timestamp, + test_timestamptz + from {_table()} WITH ( + connector = 's3', + match_pattern = '*.parquet', + s3.region_name = '{config['S3_REGION']}', + s3.bucket_name = '{config['S3_BUCKET']}', + s3.credentials.access = '{config['S3_ACCESS_KEY']}', + s3.credentials.secret = '{config['S3_SECRET_KEY']}', + s3.endpoint_url = 'https://{config['S3_ENDPOINT']}' + s3.path = '', + s3.file_type = 'parquet', + type = 'append-only', + force_append_only='true' + ) FORMAT PLAIN ENCODE PARQUET(force_append_only='true');''') + + print('Sink into s3...') + # Execute a SELECT statement + cur.execute(f'''CREATE TABLE test_sink_table( + id bigint primary key, + name TEXT, + sex bigint, + mark bigint, + test_int int, + test_real real, + test_double_precision double precision, + test_varchar varchar, + test_bytea bytea, + test_date date, + test_time time, + test_timestamp timestamp, + test_timestamptz timestamptz, + ) WITH ( + connector = 's3_v2', + match_pattern = '*.parquet', + s3.region_name = '{config['S3_REGION']}', + s3.bucket_name = '{config['S3_BUCKET']}', + s3.credentials.access = '{config['S3_ACCESS_KEY']}', + s3.credentials.secret = '{config['S3_SECRET_KEY']}', + s3.endpoint_url = 'https://{config['S3_ENDPOINT']}' + ) FORMAT PLAIN ENCODE PARQUET;''') + + total_rows = file_num * item_num_per_file + MAX_RETRIES = 40 + for retry_no in range(MAX_RETRIES): + cur.execute(f'select count(*) from test_sink_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 10s") + sleep(10) + + stmt = f'select count(*), sum(id) from test_sink_table' + print(f'Execute reading sink files: {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) + + print('File sink test pass!') + + cur.close() + conn.close() + + if __name__ == "__main__": FILE_NUM = 10 @@ -134,4 +231,12 @@ def _assert_eq(field, got, expect): # clean up s3 files for idx, _ in enumerate(data): - client.remove_object(config["S3_BUCKET"], _s3(idx)) \ No newline at end of file + client.remove_object(config["S3_BUCKET"], _s3(idx)) + + do_sink(config, FILE_NUM, ITEM_NUM_PER_FILE, run_id) + + # clean up s3 files + for idx, _ in enumerate(data): + client.remove_object(config["S3_BUCKET"], _s3(idx)) + + \ No newline at end of file diff --git a/src/connector/src/sink/catalog/mod.rs b/src/connector/src/sink/catalog/mod.rs index e6e0e767d3faa..861fc24f48c2d 100644 --- a/src/connector/src/sink/catalog/mod.rs +++ b/src/connector/src/sink/catalog/mod.rs @@ -239,12 +239,7 @@ impl TryFrom for SinkFormatDesc { E::Template => SinkEncode::Template, E::Avro => SinkEncode::Avro, E::Parquet => SinkEncode::Parquet, - e @ (E::Unspecified - | E::Native - | E::Csv - | E::Bytes - | E::None - | E::Text) => { + e @ (E::Unspecified | E::Native | E::Csv | E::Bytes | E::None | E::Text) => { return Err(SinkError::Config(anyhow!( "sink encode unsupported: {}", e.as_str_name() diff --git a/src/connector/src/sink/file_sink/mod.rs b/src/connector/src/sink/file_sink/mod.rs index 4670c9f616914..b46df3b956f6a 100644 --- a/src/connector/src/sink/file_sink/mod.rs +++ b/src/connector/src/sink/file_sink/mod.rs @@ -85,7 +85,6 @@ impl SinkWriter for OpenDalSinkWriter { Ok(()) } - /// For the file sink, currently, the sink decoupling feature is not enabled. /// When a checkpoint arrives, the force commit is performed to write the data to the file. /// In the future if flush and checkpoint is decoupled, we should enable sink decouple accordingly. @@ -100,8 +99,6 @@ impl SinkWriter for OpenDalSinkWriter { Ok(()) } - - } impl OpenDalSinkWriter { diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index 9e1a4ed787bae..c8818091064bd 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -12,16 +12,27 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::BTreeMap; +use std::collections::{BTreeMap, HashMap}; use std::marker::PhantomData; +use std::sync::Arc; use anyhow::anyhow; -use opendal::Operator; +use arrow_schema_iceberg::SchemaRef; +use async_trait::async_trait; +use opendal::{FuturesAsyncWriter, Operator, Writer as OpendalWriter}; +use parquet::arrow::AsyncArrowWriter; +use parquet::file::properties::WriterProperties; +use risingwave_common::array::arrow::IcebergArrowConvert; +use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::Schema; +use tokio_util::compat::{Compat, FuturesAsyncWriteCompatExt}; -use crate::sink::file_sink::OpenDalSinkWriter; +use crate::sink::catalog::SinkEncode; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; -use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkError, SinkFormatDesc, SinkParam}; +use crate::sink::{ + DummySinkCommitCoordinator, Result, Sink, SinkError, SinkFormatDesc, SinkParam, SinkWriter, +}; /// The `FileSink` struct represents a file sink that uses the `OpendalSinkBackend` trait /// for its backend implementation. @@ -116,3 +127,180 @@ impl TryFrom for FileSink { }) } } + +pub struct OpenDalSinkWriter { + schema: SchemaRef, + operator: Operator, + sink_writer: Option, + is_append_only: bool, + write_path: String, + epoch: Option, + executor_id: u64, + encode_type: SinkEncode, +} + +/// The `FileWriterEnum` enum represents different types of file writers used for various sink +/// implementations. +/// +/// # Variants +/// +/// - `ParquetFileWriter`: Represents a Parquet file writer using the `AsyncArrowWriter` +/// for writing data to a Parquet file. It accepts an implementation of W: `AsyncWrite` + `Unpin` + `Send` +/// as the underlying writer. In this case, the `OpendalWriter` serves as the underlying writer. +/// +/// - `FileWriter`: Represents a file writer for sinks other than Parquet. It uses the `OpendalWriter` +/// directly for writing data to the file. +/// +/// The choice of writer used during the actual writing process depends on the encode type of the sink. +enum FileWriterEnum { + ParquetFileWriter(AsyncArrowWriter>), +} + +#[async_trait] +impl SinkWriter for OpenDalSinkWriter { + async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { + // Note: epoch is used to name the output files. + // Todo: after enabling sink decouple, use the new naming convention. + let epoch = self.epoch.ok_or_else(|| { + SinkError::File("epoch has not been initialize, call `begin_epoch`".to_string()) + })?; + if self.sink_writer.is_none() { + self.create_sink_writer(epoch).await?; + } + if self.is_append_only { + self.append_only(chunk).await + } else { + // currently file sink only supports append only mode. + unimplemented!() + } + } + + async fn begin_epoch(&mut self, epoch: u64) -> Result<()> { + self.epoch = Some(epoch); + Ok(()) + } + + /// For the file sink, currently, the sink decoupling feature is not enabled. + /// When a checkpoint arrives, the force commit is performed to write the data to the file. + /// In the future if flush and checkpoint is decoupled, we should enable sink decouple accordingly. + async fn barrier(&mut self, is_checkpoint: bool) -> Result<()> { + if is_checkpoint && let Some(sink_writer) = self.sink_writer.take() { + match sink_writer { + FileWriterEnum::ParquetFileWriter(w) => { + let _ = w.close().await?; + } + }; + } + + Ok(()) + } +} + +impl OpenDalSinkWriter { + pub fn new( + operator: Operator, + write_path: &str, + rw_schema: Schema, + is_append_only: bool, + executor_id: u64, + encode_type: SinkEncode, + ) -> Result { + let arrow_schema = convert_rw_schema_to_arrow_schema(rw_schema)?; + Ok(Self { + schema: Arc::new(arrow_schema), + write_path: write_path.to_string(), + operator, + sink_writer: None, + is_append_only, + epoch: None, + executor_id, + encode_type, + }) + } + + async fn create_object_writer(&mut self, epoch: u64) -> Result { + // Todo: specify more file suffixes based on encode_type. + let suffix = match self.encode_type { + SinkEncode::Parquet => "parquet", + _ => unimplemented!(), + }; + + // Note: sink decoupling is not currently supported, which means that output files will not be batched across checkpoints. + // The current implementation writes files every time a checkpoint arrives, so the naming convention is `epoch + executor_id + .suffix`. + let object_name = format!( + "{}/{}_{}.{}", + self.write_path, epoch, self.executor_id, suffix, + ); + Ok(self + .operator + .writer_with(&object_name) + .concurrent(8) + .await?) + } + + async fn create_sink_writer(&mut self, epoch: u64) -> Result<()> { + let object_writer = self.create_object_writer(epoch).await?; + match self.encode_type { + SinkEncode::Parquet => { + let props = WriterProperties::builder(); + let parquet_writer: tokio_util::compat::Compat = + object_writer.into_futures_async_write().compat_write(); + self.sink_writer = Some(FileWriterEnum::ParquetFileWriter( + AsyncArrowWriter::try_new( + parquet_writer, + self.schema.clone(), + Some(props.build()), + )?, + )); + } + // SinkEncode::Json => { + // self.sink_writer = Some(FileWriterEnum::FileWriter(object_writer)); + // unimplemented!(); + // } + _ => unimplemented!(), + } + + Ok(()) + } + + async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { + let (mut chunk, ops) = chunk.compact().into_parts(); + let filters = + chunk.visibility() & ops.iter().map(|op| *op == Op::Insert).collect::(); + chunk.set_visibility(filters); + + match self + .sink_writer + .as_mut() + .ok_or_else(|| SinkError::File("Sink writer is not created.".to_string()))? + { + FileWriterEnum::ParquetFileWriter(w) => { + let batch = + IcebergArrowConvert.to_record_batch(self.schema.clone(), &chunk.compact())?; + w.write(&batch).await?; + } + } + + Ok(()) + } +} + +fn convert_rw_schema_to_arrow_schema( + rw_schema: risingwave_common::catalog::Schema, +) -> anyhow::Result { + let mut schema_fields = HashMap::new(); + rw_schema.fields.iter().for_each(|field| { + let res = schema_fields.insert(&field.name, &field.data_type); + // This assert is to make sure there is no duplicate field name in the schema. + assert!(res.is_none()) + }); + let mut arrow_fields = vec![]; + for rw_field in &rw_schema.fields { + let arrow_field = IcebergArrowConvert + .to_arrow_field(&rw_field.name.clone(), &rw_field.data_type.clone())?; + + arrow_fields.push(arrow_field); + } + + Ok(arrow_schema_iceberg::Schema::new(arrow_fields)) +} diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index 8c7cb819a78f8..9500fa991396e 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -781,7 +781,8 @@ fn bind_sink_format_desc(session: &SessionImpl, value: ConnectorSchema) -> Resul E::Protobuf => SinkEncode::Protobuf, E::Avro => SinkEncode::Avro, E::Template => SinkEncode::Template, - e @ (E::Native | E::Csv | E::Bytes | E::None | E::Text | E::Parquet) => { + E::Parquet => SinkEncode::Parquet, + e @ (E::Native | E::Csv | E::Bytes | E::None | E::Text) => { return Err(ErrorCode::BindError(format!("sink encode unsupported: {e}")).into()); } }; From c39a1abf80ca225da8a206ce29a8f6373496a07d Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 26 Jul 2024 14:49:02 +0800 Subject: [PATCH 25/76] refactor, code move --- src/connector/src/sink/file_sink/mod.rs | 188 ------------------------ 1 file changed, 188 deletions(-) diff --git a/src/connector/src/sink/file_sink/mod.rs b/src/connector/src/sink/file_sink/mod.rs index b46df3b956f6a..4e416500e29e4 100644 --- a/src/connector/src/sink/file_sink/mod.rs +++ b/src/connector/src/sink/file_sink/mod.rs @@ -19,193 +19,5 @@ pub mod fs; pub mod gcs; pub mod opendal_sink; pub mod s3; -use std::collections::HashMap; -use std::sync::Arc; -use arrow_schema_iceberg::SchemaRef; -use async_trait::async_trait; -use opendal::{FuturesAsyncWriter, Operator, Writer as OpendalWriter}; -use parquet::arrow::AsyncArrowWriter; -use parquet::file::properties::WriterProperties; -use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::catalog::Schema; -use crate::sink::catalog::SinkEncode; -use crate::sink::{Result, SinkError, SinkWriter}; - -pub struct OpenDalSinkWriter { - schema: SchemaRef, - operator: Operator, - sink_writer: Option, - is_append_only: bool, - write_path: String, - epoch: Option, - executor_id: u64, - encode_type: SinkEncode, -} - -/// The `FileWriterEnum` enum represents different types of file writers used for various sink -/// implementations. -/// -/// # Variants -/// -/// - `ParquetFileWriter`: Represents a Parquet file writer using the `AsyncArrowWriter` -/// for writing data to a Parquet file. It accepts an implementation of W: `AsyncWrite` + `Unpin` + `Send` -/// as the underlying writer. In this case, the `OpendalWriter` serves as the underlying writer. -/// -/// - `FileWriter`: Represents a file writer for sinks other than Parquet. It uses the `OpendalWriter` -/// directly for writing data to the file. -/// -/// The choice of writer used during the actual writing process depends on the encode type of the sink. -enum FileWriterEnum { - ParquetFileWriter(AsyncArrowWriter>), -} - -#[async_trait] -impl SinkWriter for OpenDalSinkWriter { - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { - // Note: epoch is used to name the output files. - // Todo: after enabling sink decouple, use the new naming convention. - let epoch = self.epoch.ok_or_else(|| { - SinkError::File("epoch has not been initialize, call `begin_epoch`".to_string()) - })?; - if self.sink_writer.is_none() { - self.create_sink_writer(epoch).await?; - } - if self.is_append_only { - self.append_only(chunk).await - } else { - // currently file sink only supports append only mode. - unimplemented!() - } - } - - async fn begin_epoch(&mut self, epoch: u64) -> Result<()> { - self.epoch = Some(epoch); - Ok(()) - } - - /// For the file sink, currently, the sink decoupling feature is not enabled. - /// When a checkpoint arrives, the force commit is performed to write the data to the file. - /// In the future if flush and checkpoint is decoupled, we should enable sink decouple accordingly. - async fn barrier(&mut self, is_checkpoint: bool) -> Result<()> { - if is_checkpoint && let Some(sink_writer) = self.sink_writer.take() { - match sink_writer { - FileWriterEnum::ParquetFileWriter(w) => { - let _ = w.close().await?; - } - }; - } - - Ok(()) - } -} - -impl OpenDalSinkWriter { - pub fn new( - operator: Operator, - write_path: &str, - rw_schema: Schema, - is_append_only: bool, - executor_id: u64, - encode_type: SinkEncode, - ) -> Result { - let arrow_schema = convert_rw_schema_to_arrow_schema(rw_schema)?; - Ok(Self { - schema: Arc::new(arrow_schema), - write_path: write_path.to_string(), - operator, - sink_writer: None, - is_append_only, - epoch: None, - executor_id, - encode_type, - }) - } - - async fn create_object_writer(&mut self, epoch: u64) -> Result { - // Todo: specify more file suffixes based on encode_type. - let suffix = match self.encode_type { - SinkEncode::Parquet => "parquet", - _ => unimplemented!(), - }; - - // Note: sink decoupling is not currently supported, which means that output files will not be batched across checkpoints. - // The current implementation writes files every time a checkpoint arrives, so the naming convention is `epoch + executor_id + .suffix`. - let object_name = format!( - "{}/{}_{}.{}", - self.write_path, epoch, self.executor_id, suffix, - ); - Ok(self - .operator - .writer_with(&object_name) - .concurrent(8) - .await?) - } - - async fn create_sink_writer(&mut self, epoch: u64) -> Result<()> { - let object_writer = self.create_object_writer(epoch).await?; - match self.encode_type { - SinkEncode::Parquet => { - let props = WriterProperties::builder(); - let parquet_writer: tokio_util::compat::Compat = - object_writer.into_futures_async_write().compat_write(); - self.sink_writer = Some(FileWriterEnum::ParquetFileWriter( - AsyncArrowWriter::try_new( - parquet_writer, - self.schema.clone(), - Some(props.build()), - )?, - )); - } - // SinkEncode::Json => { - // self.sink_writer = Some(FileWriterEnum::FileWriter(object_writer)); - // unimplemented!(); - // } - _ => unimplemented!(), - } - - Ok(()) - } - - async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { - let (mut chunk, ops) = chunk.compact().into_parts(); - let filters = - chunk.visibility() & ops.iter().map(|op| *op == Op::Insert).collect::(); - chunk.set_visibility(filters); - - match self - .sink_writer - .as_mut() - .ok_or_else(|| SinkError::File("Sink writer is not created.".to_string()))? - { - FileWriterEnum::ParquetFileWriter(w) => { - let batch = - IcebergArrowConvert.to_record_batch(self.schema.clone(), &chunk.compact())?; - w.write(&batch).await?; - } - } - - Ok(()) - } -} - -fn convert_rw_schema_to_arrow_schema( - rw_schema: risingwave_common::catalog::Schema, -) -> anyhow::Result { - let mut schema_fields = HashMap::new(); - rw_schema.fields.iter().for_each(|field| { - let res = schema_fields.insert(&field.name, &field.data_type); - // This assert is to make sure there is no duplicate field name in the schema. - assert!(res.is_none()) - }); - let mut arrow_fields = vec![]; - for rw_field in &rw_schema.fields { - let arrow_field = IcebergArrowConvert - .to_arrow_field(&rw_field.name.clone(), &rw_field.data_type.clone())?; - - arrow_fields.push(arrow_field); - } - - Ok(arrow_schema_iceberg::Schema::new(arrow_fields)) -} From f86aed6193b11381eea1d36936a55378c1b4f4be Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 26 Jul 2024 14:55:24 +0800 Subject: [PATCH 26/76] mionor --- src/connector/src/sink/file_sink/mod.rs | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/connector/src/sink/file_sink/mod.rs b/src/connector/src/sink/file_sink/mod.rs index 4e416500e29e4..39e0f0208f884 100644 --- a/src/connector/src/sink/file_sink/mod.rs +++ b/src/connector/src/sink/file_sink/mod.rs @@ -11,13 +11,8 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::array::arrow::IcebergArrowConvert; -use risingwave_common::bitmap::Bitmap; -use tokio_util::compat::{Compat, FuturesAsyncWriteCompatExt}; pub mod fs; pub mod gcs; pub mod opendal_sink; pub mod s3; - - From 30426818d577290baf9acec9d578aaa333681cd3 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 29 Jul 2024 12:43:03 +0800 Subject: [PATCH 27/76] format python file --- e2e_test/s3/fs_parquet_source_and_sink.py | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/e2e_test/s3/fs_parquet_source_and_sink.py b/e2e_test/s3/fs_parquet_source_and_sink.py index 8dc0990b58530..30e83e00b936e 100644 --- a/e2e_test/s3/fs_parquet_source_and_sink.py +++ b/e2e_test/s3/fs_parquet_source_and_sink.py @@ -168,7 +168,7 @@ def _table(): s3.credentials.secret = '{config['S3_SECRET_KEY']}', s3.endpoint_url = 'https://{config['S3_ENDPOINT']}' ) FORMAT PLAIN ENCODE PARQUET;''') - + total_rows = file_num * item_num_per_file MAX_RETRIES = 40 for retry_no in range(MAX_RETRIES): @@ -232,11 +232,10 @@ def _assert_eq(field, got, expect): # clean up s3 files for idx, _ in enumerate(data): client.remove_object(config["S3_BUCKET"], _s3(idx)) - + do_sink(config, FILE_NUM, ITEM_NUM_PER_FILE, run_id) - + # clean up s3 files for idx, _ in enumerate(data): client.remove_object(config["S3_BUCKET"], _s3(idx)) - - \ No newline at end of file + From 3d477052653335bb1c29049d34727bb70cf65ad4 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 29 Jul 2024 14:20:48 +0800 Subject: [PATCH 28/76] minor --- src/connector/src/sink/file_sink/opendal_sink.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index c8818091064bd..589e3282c7ef9 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -154,6 +154,10 @@ pub struct OpenDalSinkWriter { /// The choice of writer used during the actual writing process depends on the encode type of the sink. enum FileWriterEnum { ParquetFileWriter(AsyncArrowWriter>), + // `FileWriter` is used to write files other than parquet format to the file system. + // Since only sinks in parquet format are currently supported, its implementation is temporarily commented out. + + // FileWriter(OpendalWriter) } #[async_trait] From 4d73654f61b68f665863c19e6c49402173badf44 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 29 Jul 2024 16:16:36 +0800 Subject: [PATCH 29/76] fix duplicate table name in ci --- e2e_test/s3/fs_parquet_source_and_sink.py | 5 +++-- src/connector/src/sink/file_sink/opendal_sink.rs | 5 +++++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/e2e_test/s3/fs_parquet_source_and_sink.py b/e2e_test/s3/fs_parquet_source_and_sink.py index 30e83e00b936e..5bb17104258ee 100644 --- a/e2e_test/s3/fs_parquet_source_and_sink.py +++ b/e2e_test/s3/fs_parquet_source_and_sink.py @@ -115,7 +115,7 @@ def _table(): return 's3_test_parquet' # Execute a SELECT statement - cur.execute(f'''CREATE sink s1 as select + cur.execute(f'''CREATE sink test_file_sink as select id, name, sex, @@ -193,7 +193,8 @@ def _assert_eq(field, got, expect): _assert_eq('sum(id)', result[1], (total_rows - 1) * total_rows / 2) print('File sink test pass!') - + cur.execute(f'drop sink test_file_sink') + cur.execute(f'drop table test_sink_table') cur.close() conn.close() diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index 589e3282c7ef9..f926ef972a866 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -87,6 +87,11 @@ impl Sink for FileSink { const SINK_NAME: &'static str = S::SINK_NAME; async fn validate(&self) -> Result<()> { + if !self.is_append_only { + return Err(SinkError::Config( + anyhow!("File only supports append-only mode at present, please change the query to append-only, or use `force_append_only = 'true'`") + )); + } Ok(()) } From 3f502dd60b962085a9a685b3828d11e0d7196c4c Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 2 Aug 2024 13:50:56 +0800 Subject: [PATCH 30/76] resolve comments --- risedev.yml | 21 +++++-------------- src/connector/src/sink/file_sink/fs.rs | 4 ++-- src/connector/src/sink/file_sink/gcs.rs | 16 ++++++-------- .../src/sink/file_sink/opendal_sink.rs | 20 +++++++----------- src/connector/src/sink/file_sink/s3.rs | 4 ++-- 5 files changed, 23 insertions(+), 42 deletions(-) diff --git a/risedev.yml b/risedev.yml index 65db3cc375e89..43062af322ac7 100644 --- a/risedev.yml +++ b/risedev.yml @@ -296,27 +296,14 @@ profile: port: 15690 dashboard-port: 15691 exporter-port: 11250 + meta-backend: etcd - use: meta-node port: 25690 dashboard-port: 25691 exporter-port: 21250 + meta-backend: etcd - use: compactor - 3meta: - steps: - - use: meta-node - port: 5690 - dashboard-port: 5691 - exporter-port: 1250 - - use: meta-node - port: 15690 - dashboard-port: 15691 - exporter-port: 11250 - - use: meta-node - port: 25690 - dashboard-port: 25691 - exporter-port: 21250 - 3etcd-3meta-1cn-1fe: steps: - use: minio @@ -344,10 +331,12 @@ profile: port: 15690 dashboard-port: 15691 exporter-port: 11250 + meta-backend: etcd - use: meta-node port: 25690 dashboard-port: 25691 exporter-port: 21250 + meta-backend: etcd - use: compactor - use: compute-node - use: frontend @@ -1631,4 +1620,4 @@ template: # If `user-managed` is true, user is responsible for starting the service # to serve at the above address and port in any way they see fit. - user-managed: false + user-managed: false \ No newline at end of file diff --git a/src/connector/src/sink/file_sink/fs.rs b/src/connector/src/sink/file_sink/fs.rs index f49e4cdb29ad9..719272eb15cef 100644 --- a/src/connector/src/sink/file_sink/fs.rs +++ b/src/connector/src/sink/file_sink/fs.rs @@ -27,7 +27,7 @@ use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SI #[derive(Deserialize, Debug, Clone, WithOptions)] pub struct FsCommon { /// The directory where the sink file is located. - #[serde(rename = "fs.path", default)] + #[serde(rename = "fs.path")] pub path: String, } @@ -37,7 +37,7 @@ pub struct FsConfig { #[serde(flatten)] pub common: FsCommon, - pub r#type: String, // accept "append-only" or "upsert" + pub r#type: String, // accept "append-only" } pub const FS_SINK: &str = "fs"; diff --git a/src/connector/src/sink/file_sink/gcs.rs b/src/connector/src/sink/file_sink/gcs.rs index 82f254b9e64ab..366092aec605a 100644 --- a/src/connector/src/sink/file_sink/gcs.rs +++ b/src/connector/src/sink/file_sink/gcs.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. -use std::collections::{BTreeMap, HashMap}; +use std::collections::BTreeMap; use anyhow::anyhow; use opendal::layers::{LoggingLayer, RetryLayer}; @@ -36,14 +36,11 @@ pub struct GcsCommon { /// If credential/ADC is not set. The service account can be used to provide the credential info. #[serde(rename = "gcs.service_account", default)] - pub service_account: Option, + pub service_account: String, /// The directory where the sink file is located - #[serde(rename = "gcs.path", default)] + #[serde(rename = "gcs.path")] pub path: String, - - #[serde(flatten)] - pub unknown_fields: HashMap, } #[serde_as] @@ -52,7 +49,7 @@ pub struct GcsConfig { #[serde(flatten)] pub common: GcsCommon, - pub r#type: String, // accept "append-only" or "upsert" + pub r#type: String, // accept "append-only" } pub const GCS_SINK: &str = "gcs"; @@ -74,9 +71,8 @@ impl FileSink { } } - if let Some(service_account) = config.common.service_account { - builder.service_account(&service_account); - } + builder.service_account(&config.common.service_account); + let operator: Operator = Operator::new(builder)? .layer(LoggingLayer::default()) .layer(RetryLayer::default()) diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index f926ef972a866..a124cef176079 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -89,9 +89,16 @@ impl Sink for FileSink { async fn validate(&self) -> Result<()> { if !self.is_append_only { return Err(SinkError::Config( - anyhow!("File only supports append-only mode at present, please change the query to append-only, or use `force_append_only = 'true'`") + anyhow!("File sink only supports append-only mode at present. + Please change the query to append-only, and you need to specify it explicitly after the `FORMAT ... ENCODE ...` statement. + For example, `FORMAT xxx ENCODE xxx(force_append_only='true')`") )); } + if self.format_desc.encode != SinkEncode::Parquet { + return Err(SinkError::Config(anyhow!( + "File sink only supports `PARQUET` encode at present." + ))); + } Ok(()) } @@ -153,16 +160,9 @@ pub struct OpenDalSinkWriter { /// for writing data to a Parquet file. It accepts an implementation of W: `AsyncWrite` + `Unpin` + `Send` /// as the underlying writer. In this case, the `OpendalWriter` serves as the underlying writer. /// -/// - `FileWriter`: Represents a file writer for sinks other than Parquet. It uses the `OpendalWriter` -/// directly for writing data to the file. -/// /// The choice of writer used during the actual writing process depends on the encode type of the sink. enum FileWriterEnum { ParquetFileWriter(AsyncArrowWriter>), - // `FileWriter` is used to write files other than parquet format to the file system. - // Since only sinks in parquet format are currently supported, its implementation is temporarily commented out. - - // FileWriter(OpendalWriter) } #[async_trait] @@ -262,10 +262,6 @@ impl OpenDalSinkWriter { )?, )); } - // SinkEncode::Json => { - // self.sink_writer = Some(FileWriterEnum::FileWriter(object_writer)); - // unimplemented!(); - // } _ => unimplemented!(), } diff --git a/src/connector/src/sink/file_sink/s3.rs b/src/connector/src/sink/file_sink/s3.rs index 0bfc5ba47c27e..059d64bf45b1b 100644 --- a/src/connector/src/sink/file_sink/s3.rs +++ b/src/connector/src/sink/file_sink/s3.rs @@ -31,7 +31,7 @@ pub struct S3Common { #[serde(rename = "s3.bucket_name")] pub bucket_name: String, /// The directory where the sink file is located. - #[serde(rename = "s3.path", default)] + #[serde(rename = "s3.path")] pub path: String, #[serde(rename = "s3.credentials.access", default)] pub access: Option, @@ -49,7 +49,7 @@ pub struct S3Config { #[serde(flatten)] pub common: S3Common, - pub r#type: String, // accept "append-only" or "upsert" + pub r#type: String, // accept "append-only" } pub const S3_SINK: &str = "s3"; From 2599256d9f22be6d2c5ef0732751c2dc473a41b3 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 2 Aug 2024 14:27:41 +0800 Subject: [PATCH 31/76] fix local fs sink --- src/connector/src/sink/file_sink/fs.rs | 4 +++ src/connector/src/sink/file_sink/gcs.rs | 4 +++ .../src/sink/file_sink/opendal_sink.rs | 33 ++++++++++++++++--- src/connector/src/sink/file_sink/s3.rs | 4 +++ 4 files changed, 40 insertions(+), 5 deletions(-) diff --git a/src/connector/src/sink/file_sink/fs.rs b/src/connector/src/sink/file_sink/fs.rs index 719272eb15cef..2f4afd9eff429 100644 --- a/src/connector/src/sink/file_sink/fs.rs +++ b/src/connector/src/sink/file_sink/fs.rs @@ -85,4 +85,8 @@ impl OpendalSinkBackend for FsSink { fn get_path(properties: &Self::Properties) -> String { (*properties.common.path).to_string() } + + fn get_engine_type() -> super::opendal_sink::EngineType { + super::opendal_sink::EngineType::Fs + } } diff --git a/src/connector/src/sink/file_sink/gcs.rs b/src/connector/src/sink/file_sink/gcs.rs index 366092aec605a..882ec7e5e68ee 100644 --- a/src/connector/src/sink/file_sink/gcs.rs +++ b/src/connector/src/sink/file_sink/gcs.rs @@ -110,4 +110,8 @@ impl OpendalSinkBackend for GcsSink { fn get_path(properties: &Self::Properties) -> String { (*properties.common.path).to_string() } + + fn get_engine_type() -> super::opendal_sink::EngineType { + super::opendal_sink::EngineType::Gcs + } } diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index a124cef176079..7e83d8ec7f17e 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -52,6 +52,7 @@ pub struct FileSink { /// The description of the sink's format. pub(crate) format_desc: SinkFormatDesc, + pub(crate) engine_type: EngineType, pub(crate) marker: PhantomData, } @@ -78,6 +79,14 @@ pub trait OpendalSinkBackend: Send + Sync + 'static + Clone + PartialEq { fn from_btreemap(hash_map: BTreeMap) -> Result; fn new_operator(properties: Self::Properties) -> Result; fn get_path(properties: &Self::Properties) -> String; + fn get_engine_type() -> EngineType; +} + +#[derive(Clone, Debug)] +pub enum EngineType { + Gcs, + S3, + Fs, } impl Sink for FileSink { @@ -113,6 +122,7 @@ impl Sink for FileSink { self.is_append_only, writer_param.executor_id, self.format_desc.encode.clone(), + self.engine_type.clone(), )? .into_log_sinker(writer_param.sink_metrics)) } @@ -126,7 +136,7 @@ impl TryFrom for FileSink { let config = S::from_btreemap(param.properties)?; let path = S::get_path(&config); let op = S::new_operator(config)?; - + let engine_type = S::get_engine_type(); Ok(Self { op, path, @@ -135,6 +145,7 @@ impl TryFrom for FileSink { format_desc: param .format_desc .ok_or_else(|| SinkError::Config(anyhow!("missing FORMAT ... ENCODE ...")))?, + engine_type, marker: PhantomData, }) } @@ -149,6 +160,7 @@ pub struct OpenDalSinkWriter { epoch: Option, executor_id: u64, encode_type: SinkEncode, + engine_type: EngineType, } /// The `FileWriterEnum` enum represents different types of file writers used for various sink @@ -213,6 +225,7 @@ impl OpenDalSinkWriter { is_append_only: bool, executor_id: u64, encode_type: SinkEncode, + engine_type: EngineType, ) -> Result { let arrow_schema = convert_rw_schema_to_arrow_schema(rw_schema)?; Ok(Self { @@ -223,7 +236,9 @@ impl OpenDalSinkWriter { is_append_only, epoch: None, executor_id, + encode_type, + engine_type, }) } @@ -236,10 +251,18 @@ impl OpenDalSinkWriter { // Note: sink decoupling is not currently supported, which means that output files will not be batched across checkpoints. // The current implementation writes files every time a checkpoint arrives, so the naming convention is `epoch + executor_id + .suffix`. - let object_name = format!( - "{}/{}_{}.{}", - self.write_path, epoch, self.executor_id, suffix, - ); + let object_name = match self.engine_type { + // For the local fs sink, the data will be automatically written to the defined path. + // Therefore, there is no need to specify the path in the file name. + EngineType::Fs => { + format!("{}_{}.{}", epoch, self.executor_id, suffix,) + } + _ => format!( + "{}/{}_{}.{}", + self.write_path, epoch, self.executor_id, suffix, + ), + }; + Ok(self .operator .writer_with(&object_name) diff --git a/src/connector/src/sink/file_sink/s3.rs b/src/connector/src/sink/file_sink/s3.rs index 059d64bf45b1b..de6d4cc1163eb 100644 --- a/src/connector/src/sink/file_sink/s3.rs +++ b/src/connector/src/sink/file_sink/s3.rs @@ -125,4 +125,8 @@ impl OpendalSinkBackend for S3Sink { fn get_path(properties: &Self::Properties) -> String { (*properties.common.path).to_string() } + + fn get_engine_type() -> super::opendal_sink::EngineType { + super::opendal_sink::EngineType::S3 + } } From 6f2735262d3e51641ee28c2b2faebda13de4fdbc Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 2 Aug 2024 14:37:36 +0800 Subject: [PATCH 32/76] remove java file sink --- .../com/risingwave/connector/FileSink.java | 112 ------------------ .../risingwave/connector/FileSinkConfig.java | 38 ------ .../risingwave/connector/FileSinkFactory.java | 41 ------- .../src/sink/file_sink/opendal_sink.rs | 11 +- 4 files changed, 6 insertions(+), 196 deletions(-) delete mode 100644 java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java delete mode 100644 java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkConfig.java delete mode 100644 java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java deleted file mode 100644 index af2a9e11d04bc..0000000000000 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java +++ /dev/null @@ -1,112 +0,0 @@ -// Copyright 2024 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. - -package com.risingwave.connector; - -import static io.grpc.Status.*; - -import com.google.gson.Gson; -import com.risingwave.connector.api.TableSchema; -import com.risingwave.connector.api.sink.SinkRow; -import com.risingwave.connector.api.sink.SinkWriterBase; -import java.io.File; -import java.io.FileWriter; -import java.io.IOException; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Iterator; -import java.util.UUID; -import java.util.stream.IntStream; - -public class FileSink extends SinkWriterBase { - private final FileWriter sinkWriter; - - private FileSinkConfig config; - - private boolean closed = false; - - public FileSink(FileSinkConfig config, TableSchema tableSchema) { - super(tableSchema); - - String sinkPath = config.getSinkPath(); - try { - new File(sinkPath).mkdirs(); - Path path = Paths.get(sinkPath, UUID.randomUUID() + ".dat"); - if (path.toFile().createNewFile()) { - sinkWriter = new FileWriter(path.toFile()); - } else { - throw INTERNAL.withDescription("failed to create file: " + path) - .asRuntimeException(); - } - config.setSinkPath(path.toString()); - } catch (IOException e) { - throw INTERNAL.withCause(e).asRuntimeException(); - } - - this.config = config; - } - - @Override - public void write(Iterator rows) { - while (rows.hasNext()) { - SinkRow row = rows.next(); - switch (row.getOp()) { - case INSERT: - String buf = - new Gson() - .toJson( - IntStream.range(0, row.size()) - .mapToObj(row::get) - .toArray()); - try { - sinkWriter.write(buf + System.lineSeparator()); - } catch (IOException e) { - throw INTERNAL.withCause(e).asRuntimeException(); - } - break; - default: - throw UNIMPLEMENTED - .withDescription("unsupported operation: " + row.getOp()) - .asRuntimeException(); - } - } - } - - @Override - public void sync() { - try { - sinkWriter.flush(); - } catch (IOException e) { - throw INTERNAL.withCause(e).asRuntimeException(); - } - } - - @Override - public void drop() { - try { - sinkWriter.close(); - closed = true; - } catch (IOException e) { - throw INTERNAL.withCause(e).asRuntimeException(); - } - } - - public String getSinkPath() { - return config.getSinkPath(); - } - - public boolean isClosed() { - return closed; - } -} diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkConfig.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkConfig.java deleted file mode 100644 index ec2bb55d968a5..0000000000000 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkConfig.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Copyright 2024 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. - */ - -package com.risingwave.connector; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.risingwave.connector.api.sink.CommonSinkConfig; - -public class FileSinkConfig extends CommonSinkConfig { - private String sinkPath; - - @JsonCreator - public FileSinkConfig(@JsonProperty(value = "output.path") String sinkPath) { - this.sinkPath = sinkPath; - } - - public String getSinkPath() { - return sinkPath; - } - - public void setSinkPath(String sinkPath) { - this.sinkPath = sinkPath; - } -} diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java deleted file mode 100644 index cb692d067696d..0000000000000 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java +++ /dev/null @@ -1,41 +0,0 @@ -// Copyright 2024 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. - -package com.risingwave.connector; - -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.risingwave.connector.api.TableSchema; -import com.risingwave.connector.api.sink.SinkFactory; -import com.risingwave.connector.api.sink.SinkWriter; -import com.risingwave.connector.api.sink.SinkWriterV1; -import com.risingwave.proto.Catalog.SinkType; -import java.util.Map; - -public class FileSinkFactory implements SinkFactory { - @Override - public SinkWriter createWriter(TableSchema tableSchema, Map tableProperties) { - ObjectMapper mapper = new ObjectMapper(); - FileSinkConfig config = mapper.convertValue(tableProperties, FileSinkConfig.class); - return new SinkWriterV1.Adapter(new FileSink(config, tableSchema)); - } - - @Override - public void validate( - TableSchema tableSchema, Map tableProperties, SinkType sinkType) { - ObjectMapper mapper = new ObjectMapper(); - mapper.configure(DeserializationFeature.FAIL_ON_MISSING_CREATOR_PROPERTIES, true); - mapper.convertValue(tableProperties, FileSinkConfig.class); - } -} diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index 7e83d8ec7f17e..a830aad710d97 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -97,11 +97,12 @@ impl Sink for FileSink { async fn validate(&self) -> Result<()> { if !self.is_append_only { - return Err(SinkError::Config( - anyhow!("File sink only supports append-only mode at present. - Please change the query to append-only, and you need to specify it explicitly after the `FORMAT ... ENCODE ...` statement. - For example, `FORMAT xxx ENCODE xxx(force_append_only='true')`") - )); + return Err(SinkError::Config(anyhow!( + "File sink only supports append-only mode at present. \ + Please change the query to append-only, and specify it \ + explicitly after the `FORMAT ... ENCODE ...` statement. \ + For example, `FORMAT xxx ENCODE xxx(force_append_only='true')`" + ))); } if self.format_desc.encode != SinkEncode::Parquet { return Err(SinkError::Config(anyhow!( From 530d8b591406d60113c36e65cc7be30722ee89b7 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 2 Aug 2024 14:42:59 +0800 Subject: [PATCH 33/76] bring back java file sink --- .../com/risingwave/connector/FileSink.java | 112 ++++++++++++++++++ .../risingwave/connector/FileSinkConfig.java | 38 ++++++ .../risingwave/connector/FileSinkFactory.java | 41 +++++++ 3 files changed, 191 insertions(+) create mode 100644 java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java create mode 100644 java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkConfig.java create mode 100644 java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java new file mode 100644 index 0000000000000..af2a9e11d04bc --- /dev/null +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSink.java @@ -0,0 +1,112 @@ +// Copyright 2024 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. + +package com.risingwave.connector; + +import static io.grpc.Status.*; + +import com.google.gson.Gson; +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.SinkRow; +import com.risingwave.connector.api.sink.SinkWriterBase; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Iterator; +import java.util.UUID; +import java.util.stream.IntStream; + +public class FileSink extends SinkWriterBase { + private final FileWriter sinkWriter; + + private FileSinkConfig config; + + private boolean closed = false; + + public FileSink(FileSinkConfig config, TableSchema tableSchema) { + super(tableSchema); + + String sinkPath = config.getSinkPath(); + try { + new File(sinkPath).mkdirs(); + Path path = Paths.get(sinkPath, UUID.randomUUID() + ".dat"); + if (path.toFile().createNewFile()) { + sinkWriter = new FileWriter(path.toFile()); + } else { + throw INTERNAL.withDescription("failed to create file: " + path) + .asRuntimeException(); + } + config.setSinkPath(path.toString()); + } catch (IOException e) { + throw INTERNAL.withCause(e).asRuntimeException(); + } + + this.config = config; + } + + @Override + public void write(Iterator rows) { + while (rows.hasNext()) { + SinkRow row = rows.next(); + switch (row.getOp()) { + case INSERT: + String buf = + new Gson() + .toJson( + IntStream.range(0, row.size()) + .mapToObj(row::get) + .toArray()); + try { + sinkWriter.write(buf + System.lineSeparator()); + } catch (IOException e) { + throw INTERNAL.withCause(e).asRuntimeException(); + } + break; + default: + throw UNIMPLEMENTED + .withDescription("unsupported operation: " + row.getOp()) + .asRuntimeException(); + } + } + } + + @Override + public void sync() { + try { + sinkWriter.flush(); + } catch (IOException e) { + throw INTERNAL.withCause(e).asRuntimeException(); + } + } + + @Override + public void drop() { + try { + sinkWriter.close(); + closed = true; + } catch (IOException e) { + throw INTERNAL.withCause(e).asRuntimeException(); + } + } + + public String getSinkPath() { + return config.getSinkPath(); + } + + public boolean isClosed() { + return closed; + } +} diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkConfig.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkConfig.java new file mode 100644 index 0000000000000..ec2bb55d968a5 --- /dev/null +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkConfig.java @@ -0,0 +1,38 @@ +/* + * Copyright 2024 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. + */ + +package com.risingwave.connector; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.risingwave.connector.api.sink.CommonSinkConfig; + +public class FileSinkConfig extends CommonSinkConfig { + private String sinkPath; + + @JsonCreator + public FileSinkConfig(@JsonProperty(value = "output.path") String sinkPath) { + this.sinkPath = sinkPath; + } + + public String getSinkPath() { + return sinkPath; + } + + public void setSinkPath(String sinkPath) { + this.sinkPath = sinkPath; + } +} diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java new file mode 100644 index 0000000000000..cb692d067696d --- /dev/null +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/FileSinkFactory.java @@ -0,0 +1,41 @@ +// Copyright 2024 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. + +package com.risingwave.connector; + +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.risingwave.connector.api.TableSchema; +import com.risingwave.connector.api.sink.SinkFactory; +import com.risingwave.connector.api.sink.SinkWriter; +import com.risingwave.connector.api.sink.SinkWriterV1; +import com.risingwave.proto.Catalog.SinkType; +import java.util.Map; + +public class FileSinkFactory implements SinkFactory { + @Override + public SinkWriter createWriter(TableSchema tableSchema, Map tableProperties) { + ObjectMapper mapper = new ObjectMapper(); + FileSinkConfig config = mapper.convertValue(tableProperties, FileSinkConfig.class); + return new SinkWriterV1.Adapter(new FileSink(config, tableSchema)); + } + + @Override + public void validate( + TableSchema tableSchema, Map tableProperties, SinkType sinkType) { + ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_MISSING_CREATOR_PROPERTIES, true); + mapper.convertValue(tableProperties, FileSinkConfig.class); + } +} From 24b118ccbe7738b2a07bdc510fae69a79c566eac Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 2 Aug 2024 16:04:32 +0800 Subject: [PATCH 34/76] fix ut --- src/connector/with_options_sink.yaml | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/src/connector/with_options_sink.yaml b/src/connector/with_options_sink.yaml index b043e93acc8b2..6b295a20dd8a8 100644 --- a/src/connector/with_options_sink.yaml +++ b/src/connector/with_options_sink.yaml @@ -213,8 +213,7 @@ FsConfig: - name: fs.path field_type: String comments: The directory where the sink file is located. - required: false - default: Default::default + required: true - name: r#type field_type: String required: true @@ -235,8 +234,7 @@ GcsConfig: - name: gcs.path field_type: String comments: The directory where the sink file is located - required: false - default: Default::default + required: true - name: r#type field_type: String required: true @@ -790,8 +788,7 @@ S3Config: - name: s3.path field_type: String comments: The directory where the sink file is located. - required: false - default: Default::default + required: true - name: s3.credentials.access field_type: String required: false From a05dc032cc24a7b4f495061270de8f7eeaf9bfd8 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 6 Aug 2024 19:06:43 +0800 Subject: [PATCH 35/76] minor --- src/connector/src/sink/file_sink/gcs.rs | 12 ++---------- src/connector/src/sink/file_sink/opendal_sink.rs | 10 +++------- src/connector/src/sink/formatter/mod.rs | 9 +++------ src/sqlparser/src/ast/statement.rs | 1 - 4 files changed, 8 insertions(+), 24 deletions(-) diff --git a/src/connector/src/sink/file_sink/gcs.rs b/src/connector/src/sink/file_sink/gcs.rs index 882ec7e5e68ee..82b1db414ebaa 100644 --- a/src/connector/src/sink/file_sink/gcs.rs +++ b/src/connector/src/sink/file_sink/gcs.rs @@ -32,7 +32,7 @@ pub struct GcsCommon { /// The base64 encoded credential key. If not set, ADC will be used. #[serde(rename = "gcs.credential")] - pub credential: Option, + pub credential: String, /// If credential/ADC is not set. The service account can be used to provide the credential info. #[serde(rename = "gcs.service_account", default)] @@ -61,15 +61,7 @@ impl FileSink { builder.bucket(&config.common.bucket_name); - // if credential env is set, use it. Otherwise, ADC will be used. - if let Some(cred) = config.common.credential { - builder.credential(&cred); - } else { - let cred = std::env::var("GOOGLE_APPLICATION_CREDENTIALS"); - if let Ok(cred) = cred { - builder.credential(&cred); - } - } + builder.credential(&config.common.credential); builder.service_account(&config.common.service_account); diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index a830aad710d97..1953284c60d29 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -23,8 +23,7 @@ use opendal::{FuturesAsyncWriter, Operator, Writer as OpendalWriter}; use parquet::arrow::AsyncArrowWriter; use parquet::file::properties::WriterProperties; use risingwave_common::array::arrow::IcebergArrowConvert; -use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::bitmap::Bitmap; +use risingwave_common::array::StreamChunk; use risingwave_common::catalog::Schema; use tokio_util::compat::{Compat, FuturesAsyncWriteCompatExt}; @@ -293,10 +292,7 @@ impl OpenDalSinkWriter { } async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { - let (mut chunk, ops) = chunk.compact().into_parts(); - let filters = - chunk.visibility() & ops.iter().map(|op| *op == Op::Insert).collect::(); - chunk.set_visibility(filters); + let (data_chunk, _) = chunk.compact().into_parts(); match self .sink_writer @@ -305,7 +301,7 @@ impl OpenDalSinkWriter { { FileWriterEnum::ParquetFileWriter(w) => { let batch = - IcebergArrowConvert.to_record_batch(self.schema.clone(), &chunk.compact())?; + IcebergArrowConvert.to_record_batch(self.schema.clone(), &data_chunk)?; w.write(&batch).await?; } } diff --git a/src/connector/src/sink/formatter/mod.rs b/src/connector/src/sink/formatter/mod.rs index f24425c1b946b..3f84607f2a598 100644 --- a/src/connector/src/sink/formatter/mod.rs +++ b/src/connector/src/sink/formatter/mod.rs @@ -366,14 +366,12 @@ impl SinkFormatterImpl { "ENCODE TEXT is only valid as key encode." ))); } - (F::AppendOnly, E::Avro, _)| - (F::AppendOnly, E::Parquet, _) - | - (F::AppendOnly, _, Some(E::Parquet)) + (F::AppendOnly, E::Avro, _) | (F::Upsert, E::Protobuf, _) | (F::Debezium, E::Json, Some(_)) | (F::Debezium, E::Avro | E::Protobuf | E::Template | E::Text, _) - | (F::Upsert, E::Parquet, _) + | (_, E::Parquet, _) + | (_, _, Some(E::Parquet)) | (F::AppendOnly | F::Upsert, _, Some(E::Template) | Some(E::Json) | Some(E::Avro) | Some(E::Protobuf)) // reject other encode as key encode => { return Err(SinkError::Config(anyhow!( @@ -383,7 +381,6 @@ impl SinkFormatterImpl { format_desc.key_encode ))); } - _ => todo!() }, ) } diff --git a/src/sqlparser/src/ast/statement.rs b/src/sqlparser/src/ast/statement.rs index 5bd60bd680884..db314fc8d6f6e 100644 --- a/src/sqlparser/src/ast/statement.rs +++ b/src/sqlparser/src/ast/statement.rs @@ -214,7 +214,6 @@ impl Encode { _ => parser_err!( "expected AVRO | BYTES | CSV | PROTOBUF | JSON | NATIVE | TEMPLATE | PARQUET | NONE after Encode" ), - }) } } From 6551523be5996dc206a93822c078c6536f0f996f Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 7 Aug 2024 14:24:04 +0800 Subject: [PATCH 36/76] update ut --- src/connector/with_options_sink.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/connector/with_options_sink.yaml b/src/connector/with_options_sink.yaml index 6b295a20dd8a8..d9196dfbe0006 100644 --- a/src/connector/with_options_sink.yaml +++ b/src/connector/with_options_sink.yaml @@ -225,7 +225,7 @@ GcsConfig: - name: gcs.credential field_type: String comments: The base64 encoded credential key. If not set, ADC will be used. - required: false + required: true - name: gcs.service_account field_type: String comments: If credential/ADC is not set. The service account can be used to provide the credential info. From b90077619013362e002c2dca022db284851fa3bb Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 7 Aug 2024 16:16:03 +0800 Subject: [PATCH 37/76] make clippy happy --- src/connector/src/sink/file_sink/opendal_sink.rs | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index 1953284c60d29..949a378ae2a06 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -33,13 +33,11 @@ use crate::sink::{ DummySinkCommitCoordinator, Result, Sink, SinkError, SinkFormatDesc, SinkParam, SinkWriter, }; -/// The `FileSink` struct represents a file sink that uses the `OpendalSinkBackend` trait -/// for its backend implementation. +/// The `FileSink` struct represents a file sink that uses the `OpendalSinkBackend` trait for its backend implementation. /// /// # Type Parameters /// -/// - S: The type parameter S represents the concrete implementation of the `OpendalSinkBackend` -/// trait used by this file sink. +/// - S: The type parameter S represents the concrete implementation of the `OpendalSinkBackend` trait used by this file sink. #[derive(Debug, Clone)] pub struct FileSink { pub(crate) op: Operator, @@ -169,8 +167,8 @@ pub struct OpenDalSinkWriter { /// # Variants /// /// - `ParquetFileWriter`: Represents a Parquet file writer using the `AsyncArrowWriter` -/// for writing data to a Parquet file. It accepts an implementation of W: `AsyncWrite` + `Unpin` + `Send` -/// as the underlying writer. In this case, the `OpendalWriter` serves as the underlying writer. +/// for writing data to a Parquet file. It accepts an implementation of W: `AsyncWrite` + `Unpin` + `Send` +/// as the underlying writer. In this case, the `OpendalWriter` serves as the underlying writer. /// /// The choice of writer used during the actual writing process depends on the encode type of the sink. enum FileWriterEnum { From 663e5685f84fcf490cb138bedc8bb4c0291ede7f Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 7 Aug 2024 16:49:04 +0800 Subject: [PATCH 38/76] try fix ci --- ci/scripts/e2e-source-test.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index b6eff317a79ed..5bdfa0cc93c75 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -147,6 +147,7 @@ risedev ci-kill export RISINGWAVE_CI=true echo "--- e2e, ci-kafka-plus-pubsub, kafka and pubsub source" +export RUST_MIN_STACK=4194304 RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ risedev ci-start ci-kafka ./scripts/source/prepare_ci_kafka.sh From 53aca7a6b67e322fdb2af95978e94fa01781336d Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 7 Aug 2024 17:31:49 +0800 Subject: [PATCH 39/76] try fix ci --- ci/scripts/e2e-sink-test.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/ci/scripts/e2e-sink-test.sh b/ci/scripts/e2e-sink-test.sh index eaee563c7a992..976b35e35da44 100755 --- a/ci/scripts/e2e-sink-test.sh +++ b/ci/scripts/e2e-sink-test.sh @@ -68,6 +68,7 @@ echo "--- testing remote sinks" # check sink destination postgres sqllogictest -p 4566 -d dev './e2e_test/sink/remote/jdbc.load.slt' sleep 1 +export RUST_MIN_STACK=4194304 sqllogictest -h db -p 5432 -d test './e2e_test/sink/remote/jdbc.check.pg.slt' sleep 1 From b463b34817feffbe048e8bf2440591f95d2b4da6 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Fri, 9 Aug 2024 15:33:33 +0800 Subject: [PATCH 40/76] try fix ci --- ci/scripts/e2e-sink-test.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/scripts/e2e-sink-test.sh b/ci/scripts/e2e-sink-test.sh index 976b35e35da44..5ac883702e4d0 100755 --- a/ci/scripts/e2e-sink-test.sh +++ b/ci/scripts/e2e-sink-test.sh @@ -65,10 +65,10 @@ sqllogictest -p 4566 -d dev './e2e_test/sink/sink_into_table/*.slt' sleep 1 echo "--- testing remote sinks" +export RUST_MIN_STACK=4194304 # check sink destination postgres sqllogictest -p 4566 -d dev './e2e_test/sink/remote/jdbc.load.slt' sleep 1 -export RUST_MIN_STACK=4194304 sqllogictest -h db -p 5432 -d test './e2e_test/sink/remote/jdbc.check.pg.slt' sleep 1 From 60b387dcb1de19af7bfda8835ac8dbeead66ced9 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 12 Aug 2024 14:15:41 +0800 Subject: [PATCH 41/76] empty commit for retry From 8f747a2e46029350c72a18f0e8072d00a251bdcc Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 12 Aug 2024 17:22:41 +0800 Subject: [PATCH 42/76] remove rust_min_stack in e2e sink test --- ci/scripts/e2e-sink-test.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/ci/scripts/e2e-sink-test.sh b/ci/scripts/e2e-sink-test.sh index 5ac883702e4d0..eaee563c7a992 100755 --- a/ci/scripts/e2e-sink-test.sh +++ b/ci/scripts/e2e-sink-test.sh @@ -65,7 +65,6 @@ sqllogictest -p 4566 -d dev './e2e_test/sink/sink_into_table/*.slt' sleep 1 echo "--- testing remote sinks" -export RUST_MIN_STACK=4194304 # check sink destination postgres sqllogictest -p 4566 -d dev './e2e_test/sink/remote/jdbc.load.slt' sleep 1 From b5dd3029da685aa628b9ddea2b72e1d5eaaa1f91 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 12 Aug 2024 18:55:20 +0800 Subject: [PATCH 43/76] update connector-node java to 17 and may fix ci --- ci/scripts/connector-node-integration-test.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ci/scripts/connector-node-integration-test.sh b/ci/scripts/connector-node-integration-test.sh index 8853243b66805..8f7a01ef0b827 100755 --- a/ci/scripts/connector-node-integration-test.sh +++ b/ci/scripts/connector-node-integration-test.sh @@ -5,7 +5,7 @@ set -euo pipefail source ci/scripts/common.sh -VERSION=11 +VERSION=17 while getopts 'p:v:' opt; do case ${opt} in From 1891f3c0854dc4c2bb24e469fdaf1771b5dfc4c4 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 12 Aug 2024 19:19:59 +0800 Subject: [PATCH 44/76] another try --- ci/scripts/connector-node-integration-test.sh | 2 +- java/pom.xml | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/ci/scripts/connector-node-integration-test.sh b/ci/scripts/connector-node-integration-test.sh index 8f7a01ef0b827..8853243b66805 100755 --- a/ci/scripts/connector-node-integration-test.sh +++ b/ci/scripts/connector-node-integration-test.sh @@ -5,7 +5,7 @@ set -euo pipefail source ci/scripts/common.sh -VERSION=17 +VERSION=11 while getopts 'p:v:' opt; do case ${opt} in diff --git a/java/pom.xml b/java/pom.xml index c0521bd6dfc4a..66967e3844b4b 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -58,8 +58,8 @@ - 11 - 11 + 17 + 17 1.0.0 UTF-8 UTF-8 From 1b223ffdf6132650e09f6a461f3806db09aaa325 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Mon, 12 Aug 2024 20:24:25 +0800 Subject: [PATCH 45/76] do not change pom.xml --- java/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/java/pom.xml b/java/pom.xml index 66967e3844b4b..c0521bd6dfc4a 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -58,8 +58,8 @@ - 17 - 17 + 11 + 11 1.0.0 UTF-8 UTF-8 From e3855b7c2074886c654506f4c48a0f37b48a02e0 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 13 Aug 2024 13:56:26 +0800 Subject: [PATCH 46/76] remove export RUST_MIN_STACK=4194304 --- ci/scripts/e2e-source-test.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index 5bdfa0cc93c75..b6eff317a79ed 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -147,7 +147,6 @@ risedev ci-kill export RISINGWAVE_CI=true echo "--- e2e, ci-kafka-plus-pubsub, kafka and pubsub source" -export RUST_MIN_STACK=4194304 RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ risedev ci-start ci-kafka ./scripts/source/prepare_ci_kafka.sh From b16877138869711f44bf7918fc2404d44f376a38 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 13 Aug 2024 14:25:21 +0800 Subject: [PATCH 47/76] update RUST_MIN_STACK --- ci/scripts/e2e-sink-test.sh | 2 ++ ci/scripts/e2e-source-test.sh | 1 + 2 files changed, 3 insertions(+) diff --git a/ci/scripts/e2e-sink-test.sh b/ci/scripts/e2e-sink-test.sh index eaee563c7a992..c7673101c64b6 100755 --- a/ci/scripts/e2e-sink-test.sh +++ b/ci/scripts/e2e-sink-test.sh @@ -30,6 +30,8 @@ download_and_prepare_rw "$profile" source echo "--- os limits" ulimit -a +export RUST_MIN_STACK=524288 + echo "--- Download connector node package" buildkite-agent artifact download risingwave-connector.tar.gz ./ mkdir ./connector-node diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index b6eff317a79ed..5bdfa0cc93c75 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -147,6 +147,7 @@ risedev ci-kill export RISINGWAVE_CI=true echo "--- e2e, ci-kafka-plus-pubsub, kafka and pubsub source" +export RUST_MIN_STACK=4194304 RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ risedev ci-start ci-kafka ./scripts/source/prepare_ci_kafka.sh From 7024a8a1f4a62e21a2dffb7f2ccdd2d5cec4206b Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 13 Aug 2024 14:43:22 +0800 Subject: [PATCH 48/76] try increase JVM_HEAP_SIZE --- ci/scripts/e2e-sink-test.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/ci/scripts/e2e-sink-test.sh b/ci/scripts/e2e-sink-test.sh index c7673101c64b6..e1c843920c9c1 100755 --- a/ci/scripts/e2e-sink-test.sh +++ b/ci/scripts/e2e-sink-test.sh @@ -30,8 +30,6 @@ download_and_prepare_rw "$profile" source echo "--- os limits" ulimit -a -export RUST_MIN_STACK=524288 - echo "--- Download connector node package" buildkite-agent artifact download risingwave-connector.tar.gz ./ mkdir ./connector-node @@ -67,6 +65,7 @@ sqllogictest -p 4566 -d dev './e2e_test/sink/sink_into_table/*.slt' sleep 1 echo "--- testing remote sinks" +export JVM_HEAP_SIZE=2147483648 # check sink destination postgres sqllogictest -p 4566 -d dev './e2e_test/sink/remote/jdbc.load.slt' sleep 1 From 7dc0696723b9377f7747de99c3c6cb915eacfabc Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 13 Aug 2024 15:09:35 +0800 Subject: [PATCH 49/76] try increase thread stack --- ci/scripts/e2e-sink-test.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/ci/scripts/e2e-sink-test.sh b/ci/scripts/e2e-sink-test.sh index e1c843920c9c1..bdaf50df1bb2f 100755 --- a/ci/scripts/e2e-sink-test.sh +++ b/ci/scripts/e2e-sink-test.sh @@ -66,6 +66,7 @@ sleep 1 echo "--- testing remote sinks" export JVM_HEAP_SIZE=2147483648 +export JAVA_OPTS="-Xss1m" # check sink destination postgres sqllogictest -p 4566 -d dev './e2e_test/sink/remote/jdbc.load.slt' sleep 1 From 358c8c464629f29f8a8c17b3d554976b665a34a5 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 14 Aug 2024 15:58:54 +0800 Subject: [PATCH 50/76] revert JVM_HEAP_SIZE config --- ci/scripts/e2e-sink-test.sh | 2 -- 1 file changed, 2 deletions(-) diff --git a/ci/scripts/e2e-sink-test.sh b/ci/scripts/e2e-sink-test.sh index bdaf50df1bb2f..eaee563c7a992 100755 --- a/ci/scripts/e2e-sink-test.sh +++ b/ci/scripts/e2e-sink-test.sh @@ -65,8 +65,6 @@ sqllogictest -p 4566 -d dev './e2e_test/sink/sink_into_table/*.slt' sleep 1 echo "--- testing remote sinks" -export JVM_HEAP_SIZE=2147483648 -export JAVA_OPTS="-Xss1m" # check sink destination postgres sqllogictest -p 4566 -d dev './e2e_test/sink/remote/jdbc.load.slt' sleep 1 From 4bd3a6d0a457427faa1e2e691fab9265a786d375 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 14 Aug 2024 17:57:50 +0800 Subject: [PATCH 51/76] set JVM_HEAP_SIZE = 8G*0.07 --- ci/scripts/e2e-sink-test.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ci/scripts/e2e-sink-test.sh b/ci/scripts/e2e-sink-test.sh index eaee563c7a992..e45c41fed6883 100755 --- a/ci/scripts/e2e-sink-test.sh +++ b/ci/scripts/e2e-sink-test.sh @@ -7,7 +7,7 @@ source ci/scripts/common.sh # prepare environment export CONNECTOR_LIBS_PATH="./connector-node/libs" - +export JVM_HEAP_SIZE=601295422 while getopts 'p:' opt; do case ${opt} in p ) @@ -65,6 +65,7 @@ sqllogictest -p 4566 -d dev './e2e_test/sink/sink_into_table/*.slt' sleep 1 echo "--- testing remote sinks" + # check sink destination postgres sqllogictest -p 4566 -d dev './e2e_test/sink/remote/jdbc.load.slt' sleep 1 From c1aae25ab21c384b849c75bd095554b2a689fcaa Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 14 Aug 2024 18:49:10 +0800 Subject: [PATCH 52/76] revert set JVM_HEAP_SIZE = 8G*0.07, remove file sink in macro --- ci/scripts/e2e-sink-test.sh | 1 - src/connector/src/sink/mod.rs | 6 +++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/ci/scripts/e2e-sink-test.sh b/ci/scripts/e2e-sink-test.sh index e45c41fed6883..8b4f5aec671f6 100755 --- a/ci/scripts/e2e-sink-test.sh +++ b/ci/scripts/e2e-sink-test.sh @@ -7,7 +7,6 @@ source ci/scripts/common.sh # prepare environment export CONNECTOR_LIBS_PATH="./connector-node/libs" -export JVM_HEAP_SIZE=601295422 while getopts 'p:' opt; do case ${opt} in p ) diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index d13f3828f2e3d..338f861b124c3 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -103,9 +103,9 @@ macro_rules! for_all_sinks { { HttpJava, $crate::sink::remote::HttpJavaSink }, { Doris, $crate::sink::doris::DorisSink }, { Starrocks, $crate::sink::starrocks::StarrocksSink }, - { S3, $crate::sink::file_sink::opendal_sink::FileSink }, - { Gcs, $crate::sink::file_sink::opendal_sink::FileSink }, - { Fs, $crate::sink::file_sink::opendal_sink::FileSink }, + // { S3, $crate::sink::file_sink::opendal_sink::FileSink }, + // { Gcs, $crate::sink::file_sink::opendal_sink::FileSink }, + // { Fs, $crate::sink::file_sink::opendal_sink::FileSink }, { Snowflake, $crate::sink::snowflake::SnowflakeSink }, { DeltaLake, $crate::sink::deltalake::DeltaLakeSink }, { BigQuery, $crate::sink::big_query::BigQuerySink }, From 0a4a6f1eeb2eb91162d92fda4e873ec0376a247e Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 14 Aug 2024 18:55:59 +0800 Subject: [PATCH 53/76] clippy happier --- src/connector/src/sink/mod.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 338f861b124c3..afd5688a343d4 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -75,9 +75,9 @@ use self::mock_coordination_client::{MockMetaClient, SinkCoordinationRpcClientEn use crate::error::ConnectorError; use crate::sink::catalog::desc::SinkDesc; use crate::sink::catalog::{SinkCatalog, SinkId}; -use crate::sink::file_sink::fs::FsSink; -use crate::sink::file_sink::gcs::GcsSink; -use crate::sink::file_sink::s3::S3Sink; +// use crate::sink::file_sink::fs::FsSink; +// use crate::sink::file_sink::gcs::GcsSink; +// use crate::sink::file_sink::s3::S3Sink; use crate::sink::log_store::{LogReader, LogStoreReadItem, LogStoreResult, TruncateOffset}; use crate::sink::writer::SinkWriter; const BOUNDED_CHANNEL_SIZE: usize = 16; From e8b14864090e60ce54ad0beb395145950497ee45 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 14 Aug 2024 19:06:37 +0800 Subject: [PATCH 54/76] clippy happier --- src/frontend/src/optimizer/plan_node/stream_sink.rs | 6 +++--- src/stream/src/from_proto/sink.rs | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index a12976a3fc917..9add9d041e1d6 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -26,9 +26,9 @@ use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::desc::SinkDesc; use risingwave_connector::sink::catalog::{SinkFormat, SinkFormatDesc, SinkId, SinkType}; -use risingwave_connector::sink::file_sink::fs::FsSink; -use risingwave_connector::sink::file_sink::gcs::GcsSink; -use risingwave_connector::sink::file_sink::s3::S3Sink; +// use risingwave_connector::sink::file_sink::fs::FsSink; +// use risingwave_connector::sink::file_sink::gcs::GcsSink; +// use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::iceberg::ICEBERG_SINK; use risingwave_connector::sink::trivial::TABLE_SINK; use risingwave_connector::sink::{ diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs index 4029778833c4c..194e0f1a93bac 100644 --- a/src/stream/src/from_proto/sink.rs +++ b/src/stream/src/from_proto/sink.rs @@ -20,9 +20,9 @@ use risingwave_common::secret::LocalSecretManager; use risingwave_common::types::DataType; use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::{SinkFormatDesc, SinkId, SinkType}; -use risingwave_connector::sink::file_sink::fs::FsSink; -use risingwave_connector::sink::file_sink::gcs::GcsSink; -use risingwave_connector::sink::file_sink::s3::S3Sink; +// use risingwave_connector::sink::file_sink::fs::FsSink; +// use risingwave_connector::sink::file_sink::gcs::GcsSink; +// use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::{ SinkError, SinkMetaClient, SinkParam, SinkWriterParam, CONNECTOR_TYPE_KEY, SINK_TYPE_OPTION, }; From 76d9b65ae5fd32bbbfacc4841f8b2c5aabb61bda Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 14 Aug 2024 19:35:31 +0800 Subject: [PATCH 55/76] revert remove file sink in macro, and set JVM_HEAP_SIZE = 5G --- ci/scripts/e2e-sink-test.sh | 1 + src/connector/src/sink/mod.rs | 6 +++--- src/frontend/src/optimizer/plan_node/stream_sink.rs | 6 +++--- src/stream/src/from_proto/sink.rs | 6 +++--- 4 files changed, 10 insertions(+), 9 deletions(-) diff --git a/ci/scripts/e2e-sink-test.sh b/ci/scripts/e2e-sink-test.sh index 8b4f5aec671f6..cbcef5fa057fa 100755 --- a/ci/scripts/e2e-sink-test.sh +++ b/ci/scripts/e2e-sink-test.sh @@ -7,6 +7,7 @@ source ci/scripts/common.sh # prepare environment export CONNECTOR_LIBS_PATH="./connector-node/libs" +export JVM_HEAP_SIZE=5368709120 while getopts 'p:' opt; do case ${opt} in p ) diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index afd5688a343d4..f47cecfafb073 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -103,9 +103,9 @@ macro_rules! for_all_sinks { { HttpJava, $crate::sink::remote::HttpJavaSink }, { Doris, $crate::sink::doris::DorisSink }, { Starrocks, $crate::sink::starrocks::StarrocksSink }, - // { S3, $crate::sink::file_sink::opendal_sink::FileSink }, - // { Gcs, $crate::sink::file_sink::opendal_sink::FileSink }, - // { Fs, $crate::sink::file_sink::opendal_sink::FileSink }, + { S3, $crate::sink::file_sink::opendal_sink::FileSink }, + { Gcs, $crate::sink::file_sink::opendal_sink::FileSink }, + { Fs, $crate::sink::file_sink::opendal_sink::FileSink }, { Snowflake, $crate::sink::snowflake::SnowflakeSink }, { DeltaLake, $crate::sink::deltalake::DeltaLakeSink }, { BigQuery, $crate::sink::big_query::BigQuerySink }, diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 9add9d041e1d6..a12976a3fc917 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -26,9 +26,9 @@ use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::desc::SinkDesc; use risingwave_connector::sink::catalog::{SinkFormat, SinkFormatDesc, SinkId, SinkType}; -// use risingwave_connector::sink::file_sink::fs::FsSink; -// use risingwave_connector::sink::file_sink::gcs::GcsSink; -// use risingwave_connector::sink::file_sink::s3::S3Sink; +use risingwave_connector::sink::file_sink::fs::FsSink; +use risingwave_connector::sink::file_sink::gcs::GcsSink; +use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::iceberg::ICEBERG_SINK; use risingwave_connector::sink::trivial::TABLE_SINK; use risingwave_connector::sink::{ diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs index 194e0f1a93bac..4029778833c4c 100644 --- a/src/stream/src/from_proto/sink.rs +++ b/src/stream/src/from_proto/sink.rs @@ -20,9 +20,9 @@ use risingwave_common::secret::LocalSecretManager; use risingwave_common::types::DataType; use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::{SinkFormatDesc, SinkId, SinkType}; -// use risingwave_connector::sink::file_sink::fs::FsSink; -// use risingwave_connector::sink::file_sink::gcs::GcsSink; -// use risingwave_connector::sink::file_sink::s3::S3Sink; +use risingwave_connector::sink::file_sink::fs::FsSink; +use risingwave_connector::sink::file_sink::gcs::GcsSink; +use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::{ SinkError, SinkMetaClient, SinkParam, SinkWriterParam, CONNECTOR_TYPE_KEY, SINK_TYPE_OPTION, }; From 171bd2c472269ddd321019f728e554ee80d22b4e Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 14 Aug 2024 19:47:23 +0800 Subject: [PATCH 56/76] revert remove file sink in macro, and set JVM_HEAP_SIZE = 5G --- src/connector/src/sink/mod.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index f47cecfafb073..d13f3828f2e3d 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -75,9 +75,9 @@ use self::mock_coordination_client::{MockMetaClient, SinkCoordinationRpcClientEn use crate::error::ConnectorError; use crate::sink::catalog::desc::SinkDesc; use crate::sink::catalog::{SinkCatalog, SinkId}; -// use crate::sink::file_sink::fs::FsSink; -// use crate::sink::file_sink::gcs::GcsSink; -// use crate::sink::file_sink::s3::S3Sink; +use crate::sink::file_sink::fs::FsSink; +use crate::sink::file_sink::gcs::GcsSink; +use crate::sink::file_sink::s3::S3Sink; use crate::sink::log_store::{LogReader, LogStoreReadItem, LogStoreResult, TruncateOffset}; use crate::sink::writer::SinkWriter; const BOUNDED_CHANNEL_SIZE: usize = 16; From a21b30ebbdf8fe192b097c49b2cb14f62dee85ac Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 14 Aug 2024 20:13:12 +0800 Subject: [PATCH 57/76] only keep local fs sink --- ci/scripts/e2e-sink-test.sh | 1 - src/connector/src/sink/mod.rs | 8 ++++---- src/frontend/src/optimizer/plan_node/stream_sink.rs | 4 ++-- src/stream/src/from_proto/sink.rs | 4 ++-- 4 files changed, 8 insertions(+), 9 deletions(-) diff --git a/ci/scripts/e2e-sink-test.sh b/ci/scripts/e2e-sink-test.sh index cbcef5fa057fa..8b4f5aec671f6 100755 --- a/ci/scripts/e2e-sink-test.sh +++ b/ci/scripts/e2e-sink-test.sh @@ -7,7 +7,6 @@ source ci/scripts/common.sh # prepare environment export CONNECTOR_LIBS_PATH="./connector-node/libs" -export JVM_HEAP_SIZE=5368709120 while getopts 'p:' opt; do case ${opt} in p ) diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index d13f3828f2e3d..b288abc7cc150 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -76,8 +76,8 @@ use crate::error::ConnectorError; use crate::sink::catalog::desc::SinkDesc; use crate::sink::catalog::{SinkCatalog, SinkId}; use crate::sink::file_sink::fs::FsSink; -use crate::sink::file_sink::gcs::GcsSink; -use crate::sink::file_sink::s3::S3Sink; +// use crate::sink::file_sink::gcs::GcsSink; +// use crate::sink::file_sink::s3::S3Sink; use crate::sink::log_store::{LogReader, LogStoreReadItem, LogStoreResult, TruncateOffset}; use crate::sink::writer::SinkWriter; const BOUNDED_CHANNEL_SIZE: usize = 16; @@ -103,8 +103,8 @@ macro_rules! for_all_sinks { { HttpJava, $crate::sink::remote::HttpJavaSink }, { Doris, $crate::sink::doris::DorisSink }, { Starrocks, $crate::sink::starrocks::StarrocksSink }, - { S3, $crate::sink::file_sink::opendal_sink::FileSink }, - { Gcs, $crate::sink::file_sink::opendal_sink::FileSink }, + // { S3, $crate::sink::file_sink::opendal_sink::FileSink }, + // { Gcs, $crate::sink::file_sink::opendal_sink::FileSink }, { Fs, $crate::sink::file_sink::opendal_sink::FileSink }, { Snowflake, $crate::sink::snowflake::SnowflakeSink }, { DeltaLake, $crate::sink::deltalake::DeltaLakeSink }, diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index a12976a3fc917..e2d1893a55340 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -27,8 +27,8 @@ use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::desc::SinkDesc; use risingwave_connector::sink::catalog::{SinkFormat, SinkFormatDesc, SinkId, SinkType}; use risingwave_connector::sink::file_sink::fs::FsSink; -use risingwave_connector::sink::file_sink::gcs::GcsSink; -use risingwave_connector::sink::file_sink::s3::S3Sink; +// use risingwave_connector::sink::file_sink::gcs::GcsSink; +// use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::iceberg::ICEBERG_SINK; use risingwave_connector::sink::trivial::TABLE_SINK; use risingwave_connector::sink::{ diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs index 4029778833c4c..b4697e53ad5a7 100644 --- a/src/stream/src/from_proto/sink.rs +++ b/src/stream/src/from_proto/sink.rs @@ -21,8 +21,8 @@ use risingwave_common::types::DataType; use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::{SinkFormatDesc, SinkId, SinkType}; use risingwave_connector::sink::file_sink::fs::FsSink; -use risingwave_connector::sink::file_sink::gcs::GcsSink; -use risingwave_connector::sink::file_sink::s3::S3Sink; +// use risingwave_connector::sink::file_sink::gcs::GcsSink; +// use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::{ SinkError, SinkMetaClient, SinkParam, SinkWriterParam, CONNECTOR_TYPE_KEY, SINK_TYPE_OPTION, }; From a037b25b089cda037f61579481ff393ba9edbaf9 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 14 Aug 2024 21:50:55 +0800 Subject: [PATCH 58/76] keep local fs and s3 --- src/connector/src/sink/mod.rs | 4 ++-- src/frontend/src/optimizer/plan_node/stream_sink.rs | 2 +- src/stream/src/from_proto/sink.rs | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index b288abc7cc150..47a4a39f7fd9d 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -77,7 +77,7 @@ use crate::sink::catalog::desc::SinkDesc; use crate::sink::catalog::{SinkCatalog, SinkId}; use crate::sink::file_sink::fs::FsSink; // use crate::sink::file_sink::gcs::GcsSink; -// use crate::sink::file_sink::s3::S3Sink; +use crate::sink::file_sink::s3::S3Sink; use crate::sink::log_store::{LogReader, LogStoreReadItem, LogStoreResult, TruncateOffset}; use crate::sink::writer::SinkWriter; const BOUNDED_CHANNEL_SIZE: usize = 16; @@ -103,7 +103,7 @@ macro_rules! for_all_sinks { { HttpJava, $crate::sink::remote::HttpJavaSink }, { Doris, $crate::sink::doris::DorisSink }, { Starrocks, $crate::sink::starrocks::StarrocksSink }, - // { S3, $crate::sink::file_sink::opendal_sink::FileSink }, + { S3, $crate::sink::file_sink::opendal_sink::FileSink }, // { Gcs, $crate::sink::file_sink::opendal_sink::FileSink }, { Fs, $crate::sink::file_sink::opendal_sink::FileSink }, { Snowflake, $crate::sink::snowflake::SnowflakeSink }, diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index e2d1893a55340..4c371e3867ba1 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -28,7 +28,7 @@ use risingwave_connector::sink::catalog::desc::SinkDesc; use risingwave_connector::sink::catalog::{SinkFormat, SinkFormatDesc, SinkId, SinkType}; use risingwave_connector::sink::file_sink::fs::FsSink; // use risingwave_connector::sink::file_sink::gcs::GcsSink; -// use risingwave_connector::sink::file_sink::s3::S3Sink; +use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::iceberg::ICEBERG_SINK; use risingwave_connector::sink::trivial::TABLE_SINK; use risingwave_connector::sink::{ diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs index b4697e53ad5a7..c81c99dfee1da 100644 --- a/src/stream/src/from_proto/sink.rs +++ b/src/stream/src/from_proto/sink.rs @@ -22,7 +22,7 @@ use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::{SinkFormatDesc, SinkId, SinkType}; use risingwave_connector::sink::file_sink::fs::FsSink; // use risingwave_connector::sink::file_sink::gcs::GcsSink; -// use risingwave_connector::sink::file_sink::s3::S3Sink; +use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::{ SinkError, SinkMetaClient, SinkParam, SinkWriterParam, CONNECTOR_TYPE_KEY, SINK_TYPE_OPTION, }; From 89f289efa8758d6aed55f7ec6155dc8ae7d74e88 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 14 Aug 2024 22:18:12 +0800 Subject: [PATCH 59/76] keep gcs sink --- src/connector/src/sink/mod.rs | 12 ++++++------ src/frontend/src/optimizer/plan_node/stream_sink.rs | 6 +++--- src/stream/src/from_proto/sink.rs | 6 +++--- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 47a4a39f7fd9d..11a46f11e1ec0 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -75,9 +75,9 @@ use self::mock_coordination_client::{MockMetaClient, SinkCoordinationRpcClientEn use crate::error::ConnectorError; use crate::sink::catalog::desc::SinkDesc; use crate::sink::catalog::{SinkCatalog, SinkId}; -use crate::sink::file_sink::fs::FsSink; -// use crate::sink::file_sink::gcs::GcsSink; -use crate::sink::file_sink::s3::S3Sink; +// use crate::sink::file_sink::fs::FsSink; +use crate::sink::file_sink::gcs::GcsSink; +// use crate::sink::file_sink::s3::S3Sink; use crate::sink::log_store::{LogReader, LogStoreReadItem, LogStoreResult, TruncateOffset}; use crate::sink::writer::SinkWriter; const BOUNDED_CHANNEL_SIZE: usize = 16; @@ -103,9 +103,9 @@ macro_rules! for_all_sinks { { HttpJava, $crate::sink::remote::HttpJavaSink }, { Doris, $crate::sink::doris::DorisSink }, { Starrocks, $crate::sink::starrocks::StarrocksSink }, - { S3, $crate::sink::file_sink::opendal_sink::FileSink }, - // { Gcs, $crate::sink::file_sink::opendal_sink::FileSink }, - { Fs, $crate::sink::file_sink::opendal_sink::FileSink }, + // { S3, $crate::sink::file_sink::opendal_sink::FileSink }, + { Gcs, $crate::sink::file_sink::opendal_sink::FileSink }, + // { Fs, $crate::sink::file_sink::opendal_sink::FileSink }, { Snowflake, $crate::sink::snowflake::SnowflakeSink }, { DeltaLake, $crate::sink::deltalake::DeltaLakeSink }, { BigQuery, $crate::sink::big_query::BigQuerySink }, diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 4c371e3867ba1..bfbed51dfda5d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -26,9 +26,9 @@ use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::desc::SinkDesc; use risingwave_connector::sink::catalog::{SinkFormat, SinkFormatDesc, SinkId, SinkType}; -use risingwave_connector::sink::file_sink::fs::FsSink; -// use risingwave_connector::sink::file_sink::gcs::GcsSink; -use risingwave_connector::sink::file_sink::s3::S3Sink; +// use risingwave_connector::sink::file_sink::fs::FsSink; +use risingwave_connector::sink::file_sink::gcs::GcsSink; +// use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::iceberg::ICEBERG_SINK; use risingwave_connector::sink::trivial::TABLE_SINK; use risingwave_connector::sink::{ diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs index c81c99dfee1da..e3da020dcd2ec 100644 --- a/src/stream/src/from_proto/sink.rs +++ b/src/stream/src/from_proto/sink.rs @@ -20,9 +20,9 @@ use risingwave_common::secret::LocalSecretManager; use risingwave_common::types::DataType; use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::{SinkFormatDesc, SinkId, SinkType}; -use risingwave_connector::sink::file_sink::fs::FsSink; -// use risingwave_connector::sink::file_sink::gcs::GcsSink; -use risingwave_connector::sink::file_sink::s3::S3Sink; +// use risingwave_connector::sink::file_sink::fs::FsSink; +use risingwave_connector::sink::file_sink::gcs::GcsSink; +// use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::{ SinkError, SinkMetaClient, SinkParam, SinkWriterParam, CONNECTOR_TYPE_KEY, SINK_TYPE_OPTION, }; From 4510c484da6c7a78fab95ee6b58e8d19bfc77fc0 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 14 Aug 2024 22:47:19 +0800 Subject: [PATCH 60/76] keep s3 sink --- src/connector/src/sink/mod.rs | 8 ++++---- src/frontend/src/optimizer/plan_node/stream_sink.rs | 4 ++-- src/stream/src/from_proto/sink.rs | 4 ++-- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 11a46f11e1ec0..21d5dad4ad7d5 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -76,8 +76,8 @@ use crate::error::ConnectorError; use crate::sink::catalog::desc::SinkDesc; use crate::sink::catalog::{SinkCatalog, SinkId}; // use crate::sink::file_sink::fs::FsSink; -use crate::sink::file_sink::gcs::GcsSink; -// use crate::sink::file_sink::s3::S3Sink; +// use crate::sink::file_sink::gcs::GcsSink; +use crate::sink::file_sink::s3::S3Sink; use crate::sink::log_store::{LogReader, LogStoreReadItem, LogStoreResult, TruncateOffset}; use crate::sink::writer::SinkWriter; const BOUNDED_CHANNEL_SIZE: usize = 16; @@ -103,8 +103,8 @@ macro_rules! for_all_sinks { { HttpJava, $crate::sink::remote::HttpJavaSink }, { Doris, $crate::sink::doris::DorisSink }, { Starrocks, $crate::sink::starrocks::StarrocksSink }, - // { S3, $crate::sink::file_sink::opendal_sink::FileSink }, - { Gcs, $crate::sink::file_sink::opendal_sink::FileSink }, + { S3, $crate::sink::file_sink::opendal_sink::FileSink }, + // { Gcs, $crate::sink::file_sink::opendal_sink::FileSink }, // { Fs, $crate::sink::file_sink::opendal_sink::FileSink }, { Snowflake, $crate::sink::snowflake::SnowflakeSink }, { DeltaLake, $crate::sink::deltalake::DeltaLakeSink }, diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index bfbed51dfda5d..9bd88301b01bb 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -27,8 +27,8 @@ use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::desc::SinkDesc; use risingwave_connector::sink::catalog::{SinkFormat, SinkFormatDesc, SinkId, SinkType}; // use risingwave_connector::sink::file_sink::fs::FsSink; -use risingwave_connector::sink::file_sink::gcs::GcsSink; -// use risingwave_connector::sink::file_sink::s3::S3Sink; +// use risingwave_connector::sink::file_sink::gcs::GcsSink; +use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::iceberg::ICEBERG_SINK; use risingwave_connector::sink::trivial::TABLE_SINK; use risingwave_connector::sink::{ diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs index e3da020dcd2ec..d0a8e661b978c 100644 --- a/src/stream/src/from_proto/sink.rs +++ b/src/stream/src/from_proto/sink.rs @@ -21,8 +21,8 @@ use risingwave_common::types::DataType; use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::{SinkFormatDesc, SinkId, SinkType}; // use risingwave_connector::sink::file_sink::fs::FsSink; -use risingwave_connector::sink::file_sink::gcs::GcsSink; -// use risingwave_connector::sink::file_sink::s3::S3Sink; +// use risingwave_connector::sink::file_sink::gcs::GcsSink; +use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::{ SinkError, SinkMetaClient, SinkParam, SinkWriterParam, CONNECTOR_TYPE_KEY, SINK_TYPE_OPTION, }; From d32bd0be3f5ad63abf929c4fee75f5c950ec677f Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 15 Aug 2024 12:32:43 +0800 Subject: [PATCH 61/76] keep s3, gcs sink --- src/connector/src/sink/mod.rs | 4 ++-- src/frontend/src/optimizer/plan_node/stream_sink.rs | 2 +- src/stream/src/from_proto/sink.rs | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 21d5dad4ad7d5..3fb832f897119 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -76,7 +76,7 @@ use crate::error::ConnectorError; use crate::sink::catalog::desc::SinkDesc; use crate::sink::catalog::{SinkCatalog, SinkId}; // use crate::sink::file_sink::fs::FsSink; -// use crate::sink::file_sink::gcs::GcsSink; +use crate::sink::file_sink::gcs::GcsSink; use crate::sink::file_sink::s3::S3Sink; use crate::sink::log_store::{LogReader, LogStoreReadItem, LogStoreResult, TruncateOffset}; use crate::sink::writer::SinkWriter; @@ -104,7 +104,7 @@ macro_rules! for_all_sinks { { Doris, $crate::sink::doris::DorisSink }, { Starrocks, $crate::sink::starrocks::StarrocksSink }, { S3, $crate::sink::file_sink::opendal_sink::FileSink }, - // { Gcs, $crate::sink::file_sink::opendal_sink::FileSink }, + { Gcs, $crate::sink::file_sink::opendal_sink::FileSink }, // { Fs, $crate::sink::file_sink::opendal_sink::FileSink }, { Snowflake, $crate::sink::snowflake::SnowflakeSink }, { DeltaLake, $crate::sink::deltalake::DeltaLakeSink }, diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 9bd88301b01bb..4e1dfc683b663 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -27,7 +27,7 @@ use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::desc::SinkDesc; use risingwave_connector::sink::catalog::{SinkFormat, SinkFormatDesc, SinkId, SinkType}; // use risingwave_connector::sink::file_sink::fs::FsSink; -// use risingwave_connector::sink::file_sink::gcs::GcsSink; +use risingwave_connector::sink::file_sink::gcs::GcsSink; use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::iceberg::ICEBERG_SINK; use risingwave_connector::sink::trivial::TABLE_SINK; diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs index d0a8e661b978c..ef011d936fb10 100644 --- a/src/stream/src/from_proto/sink.rs +++ b/src/stream/src/from_proto/sink.rs @@ -21,7 +21,7 @@ use risingwave_common::types::DataType; use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::{SinkFormatDesc, SinkId, SinkType}; // use risingwave_connector::sink::file_sink::fs::FsSink; -// use risingwave_connector::sink::file_sink::gcs::GcsSink; +use risingwave_connector::sink::file_sink::gcs::GcsSink; use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::{ SinkError, SinkMetaClient, SinkParam, SinkWriterParam, CONNECTOR_TYPE_KEY, SINK_TYPE_OPTION, From 1779470a96944fb0abc3e36b06c971f1c4961fd0 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 15 Aug 2024 13:10:40 +0800 Subject: [PATCH 62/76] random commit --- src/connector/src/sink/file_sink/opendal_sink.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index 949a378ae2a06..925d5460ec86b 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -50,7 +50,7 @@ pub struct FileSink { /// The description of the sink's format. pub(crate) format_desc: SinkFormatDesc, pub(crate) engine_type: EngineType, - pub(crate) marker: PhantomData, + pub(crate) _marker: PhantomData, } /// The `OpendalSinkBackend` trait unifies the behavior of various sink backends @@ -144,7 +144,7 @@ impl TryFrom for FileSink { .format_desc .ok_or_else(|| SinkError::Config(anyhow!("missing FORMAT ... ENCODE ...")))?, engine_type, - marker: PhantomData, + _marker: PhantomData, }) } } From a5fa64306dd425b9ea833d41d7b3d54b992375a0 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 15 Aug 2024 13:55:41 +0800 Subject: [PATCH 63/76] add clone for sync Properties --- src/connector/src/sink/file_sink/opendal_sink.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index 925d5460ec86b..72c56bef61ef3 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -70,7 +70,7 @@ pub struct FileSink { /// - `new_operator`: Creates a new operator using the provided backend properties. /// - `get_path`: Returns the path of the sink file specified by the user's create sink statement. pub trait OpendalSinkBackend: Send + Sync + 'static + Clone + PartialEq { - type Properties: Send + Sync; + type Properties: Send + Sync + Clone + WithOptions; const SINK_NAME: &'static str; fn from_btreemap(hash_map: BTreeMap) -> Result; From a9214bfc8ca3727f8d99c473fd69d948c7ca83bf Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 15 Aug 2024 14:00:04 +0800 Subject: [PATCH 64/76] add clone for sync Properties --- src/connector/src/sink/file_sink/opendal_sink.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index 72c56bef61ef3..ae16796ac0a59 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -70,7 +70,7 @@ pub struct FileSink { /// - `new_operator`: Creates a new operator using the provided backend properties. /// - `get_path`: Returns the path of the sink file specified by the user's create sink statement. pub trait OpendalSinkBackend: Send + Sync + 'static + Clone + PartialEq { - type Properties: Send + Sync + Clone + WithOptions; + type Properties: Send + Sync + Clone; const SINK_NAME: &'static str; fn from_btreemap(hash_map: BTreeMap) -> Result; From 49ed345503e1bb6b81d71bb513d9a8f0c5129f63 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 15 Aug 2024 14:34:20 +0800 Subject: [PATCH 65/76] enhacne sink Properties --- src/connector/src/sink/file_sink/fs.rs | 12 +++++++++++- src/connector/src/sink/file_sink/gcs.rs | 12 +++++++++++- src/connector/src/sink/file_sink/opendal_sink.rs | 4 +++- src/connector/src/sink/file_sink/s3.rs | 12 +++++++++++- 4 files changed, 36 insertions(+), 4 deletions(-) diff --git a/src/connector/src/sink/file_sink/fs.rs b/src/connector/src/sink/file_sink/fs.rs index 2f4afd9eff429..bc2f0ab5b2857 100644 --- a/src/connector/src/sink/file_sink/fs.rs +++ b/src/connector/src/sink/file_sink/fs.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. -use std::collections::BTreeMap; +use std::collections::{BTreeMap, HashMap}; use anyhow::anyhow; use opendal::layers::{LoggingLayer, RetryLayer}; @@ -23,6 +23,7 @@ use with_options::WithOptions; use crate::sink::file_sink::opendal_sink::{FileSink, OpendalSinkBackend}; use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; +use crate::source::UnknownFields; #[derive(Deserialize, Debug, Clone, WithOptions)] pub struct FsCommon { @@ -38,6 +39,15 @@ pub struct FsConfig { pub common: FsCommon, pub r#type: String, // accept "append-only" + + #[serde(flatten)] + pub unknown_fields: HashMap, +} + +impl UnknownFields for FsConfig { + fn unknown_fields(&self) -> HashMap { + self.unknown_fields.clone() + } } pub const FS_SINK: &str = "fs"; diff --git a/src/connector/src/sink/file_sink/gcs.rs b/src/connector/src/sink/file_sink/gcs.rs index 82b1db414ebaa..e41c0ddda5ef3 100644 --- a/src/connector/src/sink/file_sink/gcs.rs +++ b/src/connector/src/sink/file_sink/gcs.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. -use std::collections::BTreeMap; +use std::collections::{BTreeMap, HashMap}; use anyhow::anyhow; use opendal::layers::{LoggingLayer, RetryLayer}; @@ -24,6 +24,7 @@ use with_options::WithOptions; use super::opendal_sink::FileSink; use crate::sink::file_sink::opendal_sink::OpendalSinkBackend; use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; +use crate::source::UnknownFields; #[derive(Deserialize, Debug, Clone, WithOptions)] pub struct GcsCommon { @@ -50,6 +51,15 @@ pub struct GcsConfig { pub common: GcsCommon, pub r#type: String, // accept "append-only" + + #[serde(flatten)] + pub unknown_fields: HashMap, +} + +impl UnknownFields for GcsConfig { + fn unknown_fields(&self) -> HashMap { + self.unknown_fields.clone() + } } pub const GCS_SINK: &str = "gcs"; diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index ae16796ac0a59..a53cc54664b09 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -32,6 +32,8 @@ use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{ DummySinkCommitCoordinator, Result, Sink, SinkError, SinkFormatDesc, SinkParam, SinkWriter, }; +use crate::source::TryFromBTreeMap; +use crate::with_options::WithOptions; /// The `FileSink` struct represents a file sink that uses the `OpendalSinkBackend` trait for its backend implementation. /// @@ -70,7 +72,7 @@ pub struct FileSink { /// - `new_operator`: Creates a new operator using the provided backend properties. /// - `get_path`: Returns the path of the sink file specified by the user's create sink statement. pub trait OpendalSinkBackend: Send + Sync + 'static + Clone + PartialEq { - type Properties: Send + Sync + Clone; + type Properties: TryFromBTreeMap + Send + Sync + Clone + WithOptions; const SINK_NAME: &'static str; fn from_btreemap(hash_map: BTreeMap) -> Result; diff --git a/src/connector/src/sink/file_sink/s3.rs b/src/connector/src/sink/file_sink/s3.rs index de6d4cc1163eb..db7d7eea34090 100644 --- a/src/connector/src/sink/file_sink/s3.rs +++ b/src/connector/src/sink/file_sink/s3.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. -use std::collections::BTreeMap; +use std::collections::{BTreeMap, HashMap}; use anyhow::anyhow; use opendal::layers::{LoggingLayer, RetryLayer}; @@ -24,6 +24,7 @@ use with_options::WithOptions; use super::opendal_sink::FileSink; use crate::sink::file_sink::opendal_sink::OpendalSinkBackend; use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; +use crate::source::UnknownFields; #[derive(Deserialize, Debug, Clone, WithOptions)] pub struct S3Common { #[serde(rename = "s3.region_name")] @@ -50,6 +51,9 @@ pub struct S3Config { pub common: S3Common, pub r#type: String, // accept "append-only" + + #[serde(flatten)] + pub unknown_fields: HashMap, } pub const S3_SINK: &str = "s3"; @@ -99,6 +103,12 @@ impl FileSink { #[derive(Debug, Clone, Copy, PartialEq, Eq)] pub struct S3Sink; +impl UnknownFields for S3Config { + fn unknown_fields(&self) -> HashMap { + self.unknown_fields.clone() + } +} + impl OpendalSinkBackend for S3Sink { type Properties = S3Config; From 3d41044d6903af4e911c3528f02a1c5b9d61b591 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 15 Aug 2024 14:59:09 +0800 Subject: [PATCH 66/76] minor --- src/connector/src/sink/file_sink/fs.rs | 8 ++++---- src/connector/src/sink/file_sink/gcs.rs | 8 ++++---- src/connector/src/sink/file_sink/opendal_sink.rs | 8 ++++---- src/connector/src/sink/file_sink/s3.rs | 8 ++++---- 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/connector/src/sink/file_sink/fs.rs b/src/connector/src/sink/file_sink/fs.rs index bc2f0ab5b2857..581f66ec7a793 100644 --- a/src/connector/src/sink/file_sink/fs.rs +++ b/src/connector/src/sink/file_sink/fs.rs @@ -74,8 +74,8 @@ impl OpendalSinkBackend for FsSink { const SINK_NAME: &'static str = FS_SINK; - fn from_btreemap(hash_map: BTreeMap) -> Result { - let config = serde_json::from_value::(serde_json::to_value(hash_map).unwrap()) + fn from_btreemap(btree_map: BTreeMap) -> Result { + let config = serde_json::from_value::(serde_json::to_value(btree_map).unwrap()) .map_err(|e| SinkError::Config(anyhow!(e)))?; if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { return Err(SinkError::Config(anyhow!( @@ -92,8 +92,8 @@ impl OpendalSinkBackend for FsSink { FileSink::::new_fs_sink(properties) } - fn get_path(properties: &Self::Properties) -> String { - (*properties.common.path).to_string() + fn get_path(properties: Self::Properties) -> String { + properties.common.path } fn get_engine_type() -> super::opendal_sink::EngineType { diff --git a/src/connector/src/sink/file_sink/gcs.rs b/src/connector/src/sink/file_sink/gcs.rs index e41c0ddda5ef3..c38669909c732 100644 --- a/src/connector/src/sink/file_sink/gcs.rs +++ b/src/connector/src/sink/file_sink/gcs.rs @@ -91,8 +91,8 @@ impl OpendalSinkBackend for GcsSink { const SINK_NAME: &'static str = GCS_SINK; - fn from_btreemap(hash_map: BTreeMap) -> Result { - let config = serde_json::from_value::(serde_json::to_value(hash_map).unwrap()) + fn from_btreemap(btree_map: BTreeMap) -> Result { + let config = serde_json::from_value::(serde_json::to_value(btree_map).unwrap()) .map_err(|e| SinkError::Config(anyhow!(e)))?; if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { return Err(SinkError::Config(anyhow!( @@ -109,8 +109,8 @@ impl OpendalSinkBackend for GcsSink { FileSink::::new_gcs_sink(properties) } - fn get_path(properties: &Self::Properties) -> String { - (*properties.common.path).to_string() + fn get_path(properties: Self::Properties) -> String { + properties.common.path } fn get_engine_type() -> super::opendal_sink::EngineType { diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index a53cc54664b09..3c98e104b6c23 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -75,9 +75,9 @@ pub trait OpendalSinkBackend: Send + Sync + 'static + Clone + PartialEq { type Properties: TryFromBTreeMap + Send + Sync + Clone + WithOptions; const SINK_NAME: &'static str; - fn from_btreemap(hash_map: BTreeMap) -> Result; + fn from_btreemap(btree_map: BTreeMap) -> Result; fn new_operator(properties: Self::Properties) -> Result; - fn get_path(properties: &Self::Properties) -> String; + fn get_path(properties: Self::Properties) -> String; fn get_engine_type() -> EngineType; } @@ -134,8 +134,8 @@ impl TryFrom for FileSink { fn try_from(param: SinkParam) -> std::result::Result { let schema = param.schema(); let config = S::from_btreemap(param.properties)?; - let path = S::get_path(&config); - let op = S::new_operator(config)?; + let path = S::get_path(config.clone()); + let op = S::new_operator(config.clone())?; let engine_type = S::get_engine_type(); Ok(Self { op, diff --git a/src/connector/src/sink/file_sink/s3.rs b/src/connector/src/sink/file_sink/s3.rs index db7d7eea34090..417094600e61d 100644 --- a/src/connector/src/sink/file_sink/s3.rs +++ b/src/connector/src/sink/file_sink/s3.rs @@ -114,8 +114,8 @@ impl OpendalSinkBackend for S3Sink { const SINK_NAME: &'static str = S3_SINK; - fn from_btreemap(hash_map: BTreeMap) -> Result { - let config = serde_json::from_value::(serde_json::to_value(hash_map).unwrap()) + fn from_btreemap(btree_map: BTreeMap) -> Result { + let config = serde_json::from_value::(serde_json::to_value(btree_map).unwrap()) .map_err(|e| SinkError::Config(anyhow!(e)))?; if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { return Err(SinkError::Config(anyhow!( @@ -132,8 +132,8 @@ impl OpendalSinkBackend for S3Sink { FileSink::::new_s3_sink(properties) } - fn get_path(properties: &Self::Properties) -> String { - (*properties.common.path).to_string() + fn get_path(properties: Self::Properties) -> String { + properties.common.path } fn get_engine_type() -> super::opendal_sink::EngineType { From fb9f13ea325067632ddc6c525592875c955b7a4b Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 15 Aug 2024 15:29:47 +0800 Subject: [PATCH 67/76] give another try --- src/connector/src/sink/mod.rs | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 3fb832f897119..9b9c6335db775 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -76,8 +76,6 @@ use crate::error::ConnectorError; use crate::sink::catalog::desc::SinkDesc; use crate::sink::catalog::{SinkCatalog, SinkId}; // use crate::sink::file_sink::fs::FsSink; -use crate::sink::file_sink::gcs::GcsSink; -use crate::sink::file_sink::s3::S3Sink; use crate::sink::log_store::{LogReader, LogStoreReadItem, LogStoreResult, TruncateOffset}; use crate::sink::writer::SinkWriter; const BOUNDED_CHANNEL_SIZE: usize = 16; @@ -103,8 +101,8 @@ macro_rules! for_all_sinks { { HttpJava, $crate::sink::remote::HttpJavaSink }, { Doris, $crate::sink::doris::DorisSink }, { Starrocks, $crate::sink::starrocks::StarrocksSink }, - { S3, $crate::sink::file_sink::opendal_sink::FileSink }, - { Gcs, $crate::sink::file_sink::opendal_sink::FileSink }, + { S3, $crate::sink::file_sink::opendal_sink::FileSink<$crate::sink::file_sink::s3::S3Sink>}, + { Gcs, $crate::sink::file_sink::opendal_sink::FileSink<$crate::sink::file_sink::gcs::GcsSink> }, // { Fs, $crate::sink::file_sink::opendal_sink::FileSink }, { Snowflake, $crate::sink::snowflake::SnowflakeSink }, { DeltaLake, $crate::sink::deltalake::DeltaLakeSink }, From 67466d402e22ac5bee35cd056a4e7dc930f2d7df Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 15 Aug 2024 15:45:05 +0800 Subject: [PATCH 68/76] make clippy happy --- Cargo.lock | 1 - Cargo.toml | 2 +- src/frontend/src/optimizer/plan_node/stream_sink.rs | 2 -- src/stream/src/from_proto/sink.rs | 2 -- 4 files changed, 1 insertion(+), 6 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 7183c990bfce9..7fcf2adccb562 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8000,7 +8000,6 @@ dependencies = [ [[package]] name = "otlp-embedded" version = "0.0.1" -source = "git+https://github.com/risingwavelabs/otlp-embedded?rev=e6cd165b9bc85783b42c106e99186b86b73e3507#e6cd165b9bc85783b42c106e99186b86b73e3507" dependencies = [ "axum 0.7.4", "datasize", diff --git a/Cargo.toml b/Cargo.toml index 5bfab4feb27fb..1e8e819d2371b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -131,7 +131,7 @@ hashbrown = { version = "0.14", features = ["ahash", "inline-more", "nightly"] } criterion = { version = "0.5", features = ["async_futures"] } tonic = { package = "madsim-tonic", version = "0.5.1" } tonic-build = { package = "madsim-tonic-build", version = "0.5" } -otlp-embedded = { git = "https://github.com/risingwavelabs/otlp-embedded", rev = "e6cd165b9bc85783b42c106e99186b86b73e3507" } +otlp-embedded = { path = "/Users/wangcongyi/project/otlp-embedded" } prost = { version = "0.13" } prost-build = { version = "0.13" } icelake = { git = "https://github.com/risingwavelabs/icelake.git", rev = "1860eb315183a5f3f72b4097c1e40d49407f8373", features = [ diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 4e1dfc683b663..98d27883a54b6 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -27,8 +27,6 @@ use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::desc::SinkDesc; use risingwave_connector::sink::catalog::{SinkFormat, SinkFormatDesc, SinkId, SinkType}; // use risingwave_connector::sink::file_sink::fs::FsSink; -use risingwave_connector::sink::file_sink::gcs::GcsSink; -use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::iceberg::ICEBERG_SINK; use risingwave_connector::sink::trivial::TABLE_SINK; use risingwave_connector::sink::{ diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs index ef011d936fb10..f71ba543c201f 100644 --- a/src/stream/src/from_proto/sink.rs +++ b/src/stream/src/from_proto/sink.rs @@ -21,8 +21,6 @@ use risingwave_common::types::DataType; use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::{SinkFormatDesc, SinkId, SinkType}; // use risingwave_connector::sink::file_sink::fs::FsSink; -use risingwave_connector::sink::file_sink::gcs::GcsSink; -use risingwave_connector::sink::file_sink::s3::S3Sink; use risingwave_connector::sink::{ SinkError, SinkMetaClient, SinkParam, SinkWriterParam, CONNECTOR_TYPE_KEY, SINK_TYPE_OPTION, }; From 46031f288cada129d7e5ee1ec054e297617e852d Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Thu, 15 Aug 2024 15:57:37 +0800 Subject: [PATCH 69/76] fix cargo.toml --- Cargo.lock | 1 + Cargo.toml | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/Cargo.lock b/Cargo.lock index 7fcf2adccb562..7183c990bfce9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8000,6 +8000,7 @@ dependencies = [ [[package]] name = "otlp-embedded" version = "0.0.1" +source = "git+https://github.com/risingwavelabs/otlp-embedded?rev=e6cd165b9bc85783b42c106e99186b86b73e3507#e6cd165b9bc85783b42c106e99186b86b73e3507" dependencies = [ "axum 0.7.4", "datasize", diff --git a/Cargo.toml b/Cargo.toml index 1e8e819d2371b..5bfab4feb27fb 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -131,7 +131,7 @@ hashbrown = { version = "0.14", features = ["ahash", "inline-more", "nightly"] } criterion = { version = "0.5", features = ["async_futures"] } tonic = { package = "madsim-tonic", version = "0.5.1" } tonic-build = { package = "madsim-tonic-build", version = "0.5" } -otlp-embedded = { path = "/Users/wangcongyi/project/otlp-embedded" } +otlp-embedded = { git = "https://github.com/risingwavelabs/otlp-embedded", rev = "e6cd165b9bc85783b42c106e99186b86b73e3507" } prost = { version = "0.13" } prost-build = { version = "0.13" } icelake = { git = "https://github.com/risingwavelabs/icelake.git", rev = "1860eb315183a5f3f72b4097c1e40d49407f8373", features = [ From ac9f869b464170344b35474e35b26df797313f57 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 20 Aug 2024 18:18:00 +0800 Subject: [PATCH 70/76] use box for operator --- src/connector/src/sink/file_sink/opendal_sink.rs | 8 ++++---- src/connector/src/sink/mod.rs | 4 ++-- src/frontend/src/optimizer/plan_node/stream_sink.rs | 2 +- src/stream/src/from_proto/sink.rs | 2 +- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index 3c98e104b6c23..204451f0b7c96 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -42,7 +42,7 @@ use crate::with_options::WithOptions; /// - S: The type parameter S represents the concrete implementation of the `OpendalSinkBackend` trait used by this file sink. #[derive(Debug, Clone)] pub struct FileSink { - pub(crate) op: Operator, + pub(crate) op: Box, /// The path to the file where the sink writes data. pub(crate) path: String, /// The schema describing the structure of the data being written to the file sink. @@ -138,7 +138,7 @@ impl TryFrom for FileSink { let op = S::new_operator(config.clone())?; let engine_type = S::get_engine_type(); Ok(Self { - op, + op: Box::new(op), path, schema, is_append_only: param.sink_type.is_append_only(), @@ -153,7 +153,7 @@ impl TryFrom for FileSink { pub struct OpenDalSinkWriter { schema: SchemaRef, - operator: Operator, + operator: Box, sink_writer: Option, is_append_only: bool, write_path: String, @@ -219,7 +219,7 @@ impl SinkWriter for OpenDalSinkWriter { impl OpenDalSinkWriter { pub fn new( - operator: Operator, + operator: Box, write_path: &str, rw_schema: Schema, is_append_only: bool, diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 9b9c6335db775..7779587a28723 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -75,7 +75,7 @@ use self::mock_coordination_client::{MockMetaClient, SinkCoordinationRpcClientEn use crate::error::ConnectorError; use crate::sink::catalog::desc::SinkDesc; use crate::sink::catalog::{SinkCatalog, SinkId}; -// use crate::sink::file_sink::fs::FsSink; +use crate::sink::file_sink::fs::FsSink; use crate::sink::log_store::{LogReader, LogStoreReadItem, LogStoreResult, TruncateOffset}; use crate::sink::writer::SinkWriter; const BOUNDED_CHANNEL_SIZE: usize = 16; @@ -103,7 +103,7 @@ macro_rules! for_all_sinks { { Starrocks, $crate::sink::starrocks::StarrocksSink }, { S3, $crate::sink::file_sink::opendal_sink::FileSink<$crate::sink::file_sink::s3::S3Sink>}, { Gcs, $crate::sink::file_sink::opendal_sink::FileSink<$crate::sink::file_sink::gcs::GcsSink> }, - // { Fs, $crate::sink::file_sink::opendal_sink::FileSink }, + { Fs, $crate::sink::file_sink::opendal_sink::FileSink }, { Snowflake, $crate::sink::snowflake::SnowflakeSink }, { DeltaLake, $crate::sink::deltalake::DeltaLakeSink }, { BigQuery, $crate::sink::big_query::BigQuerySink }, diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 98d27883a54b6..2717c454e6435 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -26,7 +26,7 @@ use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::desc::SinkDesc; use risingwave_connector::sink::catalog::{SinkFormat, SinkFormatDesc, SinkId, SinkType}; -// use risingwave_connector::sink::file_sink::fs::FsSink; +use risingwave_connector::sink::file_sink::fs::FsSink; use risingwave_connector::sink::iceberg::ICEBERG_SINK; use risingwave_connector::sink::trivial::TABLE_SINK; use risingwave_connector::sink::{ diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs index f71ba543c201f..c53c123a48cb5 100644 --- a/src/stream/src/from_proto/sink.rs +++ b/src/stream/src/from_proto/sink.rs @@ -20,7 +20,7 @@ use risingwave_common::secret::LocalSecretManager; use risingwave_common::types::DataType; use risingwave_connector::match_sink_name_str; use risingwave_connector::sink::catalog::{SinkFormatDesc, SinkId, SinkType}; -// use risingwave_connector::sink::file_sink::fs::FsSink; +use risingwave_connector::sink::file_sink::fs::FsSink; use risingwave_connector::sink::{ SinkError, SinkMetaClient, SinkParam, SinkWriterParam, CONNECTOR_TYPE_KEY, SINK_TYPE_OPTION, }; From 854c9bacaaf976c038b8ff2b616426bb48752b5d Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Tue, 20 Aug 2024 20:12:36 +0800 Subject: [PATCH 71/76] use box in dispatch_sink --- src/bench/sink_bench/main.rs | 2 +- src/connector/src/sink/file_sink/opendal_sink.rs | 8 ++++---- src/connector/src/sink/mod.rs | 4 ++-- src/stream/src/executor/sink.rs | 2 +- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/bench/sink_bench/main.rs b/src/bench/sink_bench/main.rs index 0aa70ad00c364..05eabdb96e913 100644 --- a/src/bench/sink_bench/main.rs +++ b/src/bench/sink_bench/main.rs @@ -549,7 +549,7 @@ async fn main() { println!("Start Sink Bench!, Wait {:?}s", BENCH_TIME); tokio::spawn(async move { dispatch_sink!(sink, sink, { - consume_log_stream(sink, mock_range_log_reader, sink_writer_param).boxed() + consume_log_stream(*sink, mock_range_log_reader, sink_writer_param).boxed() }) .await .unwrap(); diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index 204451f0b7c96..3c98e104b6c23 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -42,7 +42,7 @@ use crate::with_options::WithOptions; /// - S: The type parameter S represents the concrete implementation of the `OpendalSinkBackend` trait used by this file sink. #[derive(Debug, Clone)] pub struct FileSink { - pub(crate) op: Box, + pub(crate) op: Operator, /// The path to the file where the sink writes data. pub(crate) path: String, /// The schema describing the structure of the data being written to the file sink. @@ -138,7 +138,7 @@ impl TryFrom for FileSink { let op = S::new_operator(config.clone())?; let engine_type = S::get_engine_type(); Ok(Self { - op: Box::new(op), + op, path, schema, is_append_only: param.sink_type.is_append_only(), @@ -153,7 +153,7 @@ impl TryFrom for FileSink { pub struct OpenDalSinkWriter { schema: SchemaRef, - operator: Box, + operator: Operator, sink_writer: Option, is_append_only: bool, write_path: String, @@ -219,7 +219,7 @@ impl SinkWriter for OpenDalSinkWriter { impl OpenDalSinkWriter { pub fn new( - operator: Box, + operator: Operator, write_path: &str, rw_schema: Schema, is_append_only: bool, diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 7779587a28723..d44edf4a29e49 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -485,14 +485,14 @@ macro_rules! def_sink_impl { #[derive(Debug)] pub enum SinkImpl { $( - $variant_name($sink_type), + $variant_name(Box<$sink_type>), )* } $( impl From<$sink_type> for SinkImpl { fn from(sink: $sink_type) -> SinkImpl { - SinkImpl::$variant_name(sink) + SinkImpl::$variant_name(Box::new(sink)) } } )* diff --git a/src/stream/src/executor/sink.rs b/src/stream/src/executor/sink.rs index ef8b57781cd95..9159ff1a67957 100644 --- a/src/stream/src/executor/sink.rs +++ b/src/stream/src/executor/sink.rs @@ -229,7 +229,7 @@ impl SinkExecutor { dispatch_sink!(self.sink, sink, { let consume_log_stream = Self::execute_consume_log( - sink, + *sink, log_reader, self.input_columns, self.sink_param, From 4edff17e03eee421012a61cf31212ac0f307e558 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 21 Aug 2024 14:55:17 +0800 Subject: [PATCH 72/76] try Box new operator --- src/connector/src/sink/file_sink/fs.rs | 6 +++--- src/connector/src/sink/file_sink/gcs.rs | 6 +++--- src/connector/src/sink/file_sink/opendal_sink.rs | 4 ++-- src/connector/src/sink/file_sink/s3.rs | 6 +++--- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/connector/src/sink/file_sink/fs.rs b/src/connector/src/sink/file_sink/fs.rs index 581f66ec7a793..1b88bb1481cd6 100644 --- a/src/connector/src/sink/file_sink/fs.rs +++ b/src/connector/src/sink/file_sink/fs.rs @@ -53,7 +53,7 @@ impl UnknownFields for FsConfig { pub const FS_SINK: &str = "fs"; impl FileSink { - pub fn new_fs_sink(config: FsConfig) -> Result { + pub fn new_fs_sink(config: FsConfig) -> Result> { // Create fs builder. let mut builder = Fs::default(); // Create fs backend builder. @@ -62,7 +62,7 @@ impl FileSink { .layer(LoggingLayer::default()) .layer(RetryLayer::default()) .finish(); - Ok(operator) + Ok(Box::new(operator)) } } @@ -88,7 +88,7 @@ impl OpendalSinkBackend for FsSink { Ok(config) } - fn new_operator(properties: FsConfig) -> Result { + fn new_operator(properties: FsConfig) -> Result> { FileSink::::new_fs_sink(properties) } diff --git a/src/connector/src/sink/file_sink/gcs.rs b/src/connector/src/sink/file_sink/gcs.rs index c38669909c732..ad18a754bac23 100644 --- a/src/connector/src/sink/file_sink/gcs.rs +++ b/src/connector/src/sink/file_sink/gcs.rs @@ -65,7 +65,7 @@ impl UnknownFields for GcsConfig { pub const GCS_SINK: &str = "gcs"; impl FileSink { - pub fn new_gcs_sink(config: GcsConfig) -> Result { + pub fn new_gcs_sink(config: GcsConfig) -> Result> { // Create gcs builder. let mut builder = Gcs::default(); @@ -79,7 +79,7 @@ impl FileSink { .layer(LoggingLayer::default()) .layer(RetryLayer::default()) .finish(); - Ok(operator) + Ok(Box::new(operator)) } } @@ -105,7 +105,7 @@ impl OpendalSinkBackend for GcsSink { Ok(config) } - fn new_operator(properties: GcsConfig) -> Result { + fn new_operator(properties: GcsConfig) -> Result> { FileSink::::new_gcs_sink(properties) } diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index 3c98e104b6c23..fbd561e64e7b1 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -76,7 +76,7 @@ pub trait OpendalSinkBackend: Send + Sync + 'static + Clone + PartialEq { const SINK_NAME: &'static str; fn from_btreemap(btree_map: BTreeMap) -> Result; - fn new_operator(properties: Self::Properties) -> Result; + fn new_operator(properties: Self::Properties) -> Result>; fn get_path(properties: Self::Properties) -> String; fn get_engine_type() -> EngineType; } @@ -135,7 +135,7 @@ impl TryFrom for FileSink { let schema = param.schema(); let config = S::from_btreemap(param.properties)?; let path = S::get_path(config.clone()); - let op = S::new_operator(config.clone())?; + let op = *S::new_operator(config.clone())?; let engine_type = S::get_engine_type(); Ok(Self { op, diff --git a/src/connector/src/sink/file_sink/s3.rs b/src/connector/src/sink/file_sink/s3.rs index 417094600e61d..be012b834725f 100644 --- a/src/connector/src/sink/file_sink/s3.rs +++ b/src/connector/src/sink/file_sink/s3.rs @@ -59,7 +59,7 @@ pub struct S3Config { pub const S3_SINK: &str = "s3"; impl FileSink { - pub fn new_s3_sink(config: S3Config) -> Result { + pub fn new_s3_sink(config: S3Config) -> Result> { // Create s3 builder. let mut builder = S3::default(); builder.bucket(&config.common.bucket_name); @@ -96,7 +96,7 @@ impl FileSink { .layer(RetryLayer::default()) .finish(); - Ok(operator) + Ok(Box::new(operator)) } } @@ -128,7 +128,7 @@ impl OpendalSinkBackend for S3Sink { Ok(config) } - fn new_operator(properties: S3Config) -> Result { + fn new_operator(properties: S3Config) -> Result> { FileSink::::new_s3_sink(properties) } From a6b91516938e2699caeae6c36965510b7eaddc60 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 21 Aug 2024 15:22:04 +0800 Subject: [PATCH 73/76] try Box get_path --- src/connector/src/sink/file_sink/fs.rs | 8 ++++---- src/connector/src/sink/file_sink/gcs.rs | 8 ++++---- src/connector/src/sink/file_sink/opendal_sink.rs | 8 ++++---- src/connector/src/sink/file_sink/s3.rs | 8 ++++---- 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/connector/src/sink/file_sink/fs.rs b/src/connector/src/sink/file_sink/fs.rs index 1b88bb1481cd6..22e81adcfb68b 100644 --- a/src/connector/src/sink/file_sink/fs.rs +++ b/src/connector/src/sink/file_sink/fs.rs @@ -74,7 +74,7 @@ impl OpendalSinkBackend for FsSink { const SINK_NAME: &'static str = FS_SINK; - fn from_btreemap(btree_map: BTreeMap) -> Result { + fn from_btreemap(btree_map: BTreeMap) -> Result> { let config = serde_json::from_value::(serde_json::to_value(btree_map).unwrap()) .map_err(|e| SinkError::Config(anyhow!(e)))?; if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { @@ -85,15 +85,15 @@ impl OpendalSinkBackend for FsSink { SINK_TYPE_UPSERT ))); } - Ok(config) + Ok(Box::new(config)) } fn new_operator(properties: FsConfig) -> Result> { FileSink::::new_fs_sink(properties) } - fn get_path(properties: Self::Properties) -> String { - properties.common.path + fn get_path(properties: Self::Properties) -> Box { + properties.common.path.into_boxed_str() } fn get_engine_type() -> super::opendal_sink::EngineType { diff --git a/src/connector/src/sink/file_sink/gcs.rs b/src/connector/src/sink/file_sink/gcs.rs index ad18a754bac23..c41a0b952cd02 100644 --- a/src/connector/src/sink/file_sink/gcs.rs +++ b/src/connector/src/sink/file_sink/gcs.rs @@ -91,7 +91,7 @@ impl OpendalSinkBackend for GcsSink { const SINK_NAME: &'static str = GCS_SINK; - fn from_btreemap(btree_map: BTreeMap) -> Result { + fn from_btreemap(btree_map: BTreeMap) -> Result> { let config = serde_json::from_value::(serde_json::to_value(btree_map).unwrap()) .map_err(|e| SinkError::Config(anyhow!(e)))?; if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { @@ -102,15 +102,15 @@ impl OpendalSinkBackend for GcsSink { SINK_TYPE_UPSERT ))); } - Ok(config) + Ok(Box::new(config)) } fn new_operator(properties: GcsConfig) -> Result> { FileSink::::new_gcs_sink(properties) } - fn get_path(properties: Self::Properties) -> String { - properties.common.path + fn get_path(properties: Self::Properties) -> Box { + properties.common.path.into_boxed_str() } fn get_engine_type() -> super::opendal_sink::EngineType { diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index fbd561e64e7b1..f04169dfe3215 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -75,9 +75,9 @@ pub trait OpendalSinkBackend: Send + Sync + 'static + Clone + PartialEq { type Properties: TryFromBTreeMap + Send + Sync + Clone + WithOptions; const SINK_NAME: &'static str; - fn from_btreemap(btree_map: BTreeMap) -> Result; + fn from_btreemap(btree_map: BTreeMap) -> Result>; fn new_operator(properties: Self::Properties) -> Result>; - fn get_path(properties: Self::Properties) -> String; + fn get_path(properties: Self::Properties) -> Box; fn get_engine_type() -> EngineType; } @@ -133,8 +133,8 @@ impl TryFrom for FileSink { fn try_from(param: SinkParam) -> std::result::Result { let schema = param.schema(); - let config = S::from_btreemap(param.properties)?; - let path = S::get_path(config.clone()); + let config = *S::from_btreemap(param.properties)?; + let path = S::get_path(config.clone()).to_string(); let op = *S::new_operator(config.clone())?; let engine_type = S::get_engine_type(); Ok(Self { diff --git a/src/connector/src/sink/file_sink/s3.rs b/src/connector/src/sink/file_sink/s3.rs index be012b834725f..263610c6bd207 100644 --- a/src/connector/src/sink/file_sink/s3.rs +++ b/src/connector/src/sink/file_sink/s3.rs @@ -114,7 +114,7 @@ impl OpendalSinkBackend for S3Sink { const SINK_NAME: &'static str = S3_SINK; - fn from_btreemap(btree_map: BTreeMap) -> Result { + fn from_btreemap(btree_map: BTreeMap) -> Result> { let config = serde_json::from_value::(serde_json::to_value(btree_map).unwrap()) .map_err(|e| SinkError::Config(anyhow!(e)))?; if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { @@ -125,15 +125,15 @@ impl OpendalSinkBackend for S3Sink { SINK_TYPE_UPSERT ))); } - Ok(config) + Ok(Box::new(config)) } fn new_operator(properties: S3Config) -> Result> { FileSink::::new_s3_sink(properties) } - fn get_path(properties: Self::Properties) -> String { - properties.common.path + fn get_path(properties: Self::Properties) -> Box { + properties.common.path.into_boxed_str() } fn get_engine_type() -> super::opendal_sink::EngineType { From 223f7da6d86d453087160fff294589dcc1d5978f Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 21 Aug 2024 16:23:51 +0800 Subject: [PATCH 74/76] try again --- src/connector/src/sink/file_sink/opendal_sink.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index f04169dfe3215..1250024f5a8e0 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -42,7 +42,7 @@ use crate::with_options::WithOptions; /// - S: The type parameter S represents the concrete implementation of the `OpendalSinkBackend` trait used by this file sink. #[derive(Debug, Clone)] pub struct FileSink { - pub(crate) op: Operator, + pub(crate) op: Box, /// The path to the file where the sink writes data. pub(crate) path: String, /// The schema describing the structure of the data being written to the file sink. @@ -135,7 +135,7 @@ impl TryFrom for FileSink { let schema = param.schema(); let config = *S::from_btreemap(param.properties)?; let path = S::get_path(config.clone()).to_string(); - let op = *S::new_operator(config.clone())?; + let op = S::new_operator(config.clone())?; let engine_type = S::get_engine_type(); Ok(Self { op, @@ -153,7 +153,7 @@ impl TryFrom for FileSink { pub struct OpenDalSinkWriter { schema: SchemaRef, - operator: Operator, + operator: Box, sink_writer: Option, is_append_only: bool, write_path: String, @@ -219,7 +219,7 @@ impl SinkWriter for OpenDalSinkWriter { impl OpenDalSinkWriter { pub fn new( - operator: Operator, + operator: Box, write_path: &str, rw_schema: Schema, is_append_only: bool, From 64100c0988809c96a977db1c89036880c61b2fcc Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 21 Aug 2024 19:52:43 +0800 Subject: [PATCH 75/76] boxed future --- src/stream/src/executor/sink.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/stream/src/executor/sink.rs b/src/stream/src/executor/sink.rs index 9159ff1a67957..23162241ef298 100644 --- a/src/stream/src/executor/sink.rs +++ b/src/stream/src/executor/sink.rs @@ -227,7 +227,7 @@ impl SinkExecutor { actor_id, ); - dispatch_sink!(self.sink, sink, { + let consume_log_stream_future = dispatch_sink!(self.sink, sink, { let consume_log_stream = Self::execute_consume_log( *sink, log_reader, @@ -239,9 +239,9 @@ impl SinkExecutor { .instrument_await(format!("consume_log (sink_id {sink_id})")) .map_ok(|never| match never {}); // unify return type to `Message` - // TODO: may try to remove the boxed - select(consume_log_stream.into_stream(), write_log_stream).boxed() - }) + consume_log_stream.boxed() + }); + select(consume_log_stream_future.into_stream(), write_log_stream) }) .into_stream() .flatten() From 807c86967c68914798cf30dcb7cd6bf06fb8e895 Mon Sep 17 00:00:00 2001 From: congyi <15605187270@163.com> Date: Wed, 21 Aug 2024 23:26:33 +0800 Subject: [PATCH 76/76] revert box in FileSink struct --- src/connector/src/sink/file_sink/fs.rs | 14 +++++++------- src/connector/src/sink/file_sink/gcs.rs | 14 +++++++------- src/connector/src/sink/file_sink/opendal_sink.rs | 14 +++++++------- src/connector/src/sink/file_sink/s3.rs | 14 +++++++------- 4 files changed, 28 insertions(+), 28 deletions(-) diff --git a/src/connector/src/sink/file_sink/fs.rs b/src/connector/src/sink/file_sink/fs.rs index 22e81adcfb68b..581f66ec7a793 100644 --- a/src/connector/src/sink/file_sink/fs.rs +++ b/src/connector/src/sink/file_sink/fs.rs @@ -53,7 +53,7 @@ impl UnknownFields for FsConfig { pub const FS_SINK: &str = "fs"; impl FileSink { - pub fn new_fs_sink(config: FsConfig) -> Result> { + pub fn new_fs_sink(config: FsConfig) -> Result { // Create fs builder. let mut builder = Fs::default(); // Create fs backend builder. @@ -62,7 +62,7 @@ impl FileSink { .layer(LoggingLayer::default()) .layer(RetryLayer::default()) .finish(); - Ok(Box::new(operator)) + Ok(operator) } } @@ -74,7 +74,7 @@ impl OpendalSinkBackend for FsSink { const SINK_NAME: &'static str = FS_SINK; - fn from_btreemap(btree_map: BTreeMap) -> Result> { + fn from_btreemap(btree_map: BTreeMap) -> Result { let config = serde_json::from_value::(serde_json::to_value(btree_map).unwrap()) .map_err(|e| SinkError::Config(anyhow!(e)))?; if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { @@ -85,15 +85,15 @@ impl OpendalSinkBackend for FsSink { SINK_TYPE_UPSERT ))); } - Ok(Box::new(config)) + Ok(config) } - fn new_operator(properties: FsConfig) -> Result> { + fn new_operator(properties: FsConfig) -> Result { FileSink::::new_fs_sink(properties) } - fn get_path(properties: Self::Properties) -> Box { - properties.common.path.into_boxed_str() + fn get_path(properties: Self::Properties) -> String { + properties.common.path } fn get_engine_type() -> super::opendal_sink::EngineType { diff --git a/src/connector/src/sink/file_sink/gcs.rs b/src/connector/src/sink/file_sink/gcs.rs index c41a0b952cd02..c38669909c732 100644 --- a/src/connector/src/sink/file_sink/gcs.rs +++ b/src/connector/src/sink/file_sink/gcs.rs @@ -65,7 +65,7 @@ impl UnknownFields for GcsConfig { pub const GCS_SINK: &str = "gcs"; impl FileSink { - pub fn new_gcs_sink(config: GcsConfig) -> Result> { + pub fn new_gcs_sink(config: GcsConfig) -> Result { // Create gcs builder. let mut builder = Gcs::default(); @@ -79,7 +79,7 @@ impl FileSink { .layer(LoggingLayer::default()) .layer(RetryLayer::default()) .finish(); - Ok(Box::new(operator)) + Ok(operator) } } @@ -91,7 +91,7 @@ impl OpendalSinkBackend for GcsSink { const SINK_NAME: &'static str = GCS_SINK; - fn from_btreemap(btree_map: BTreeMap) -> Result> { + fn from_btreemap(btree_map: BTreeMap) -> Result { let config = serde_json::from_value::(serde_json::to_value(btree_map).unwrap()) .map_err(|e| SinkError::Config(anyhow!(e)))?; if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { @@ -102,15 +102,15 @@ impl OpendalSinkBackend for GcsSink { SINK_TYPE_UPSERT ))); } - Ok(Box::new(config)) + Ok(config) } - fn new_operator(properties: GcsConfig) -> Result> { + fn new_operator(properties: GcsConfig) -> Result { FileSink::::new_gcs_sink(properties) } - fn get_path(properties: Self::Properties) -> Box { - properties.common.path.into_boxed_str() + fn get_path(properties: Self::Properties) -> String { + properties.common.path } fn get_engine_type() -> super::opendal_sink::EngineType { diff --git a/src/connector/src/sink/file_sink/opendal_sink.rs b/src/connector/src/sink/file_sink/opendal_sink.rs index 1250024f5a8e0..f157d0c861d17 100644 --- a/src/connector/src/sink/file_sink/opendal_sink.rs +++ b/src/connector/src/sink/file_sink/opendal_sink.rs @@ -42,7 +42,7 @@ use crate::with_options::WithOptions; /// - S: The type parameter S represents the concrete implementation of the `OpendalSinkBackend` trait used by this file sink. #[derive(Debug, Clone)] pub struct FileSink { - pub(crate) op: Box, + pub(crate) op: Operator, /// The path to the file where the sink writes data. pub(crate) path: String, /// The schema describing the structure of the data being written to the file sink. @@ -75,9 +75,9 @@ pub trait OpendalSinkBackend: Send + Sync + 'static + Clone + PartialEq { type Properties: TryFromBTreeMap + Send + Sync + Clone + WithOptions; const SINK_NAME: &'static str; - fn from_btreemap(btree_map: BTreeMap) -> Result>; - fn new_operator(properties: Self::Properties) -> Result>; - fn get_path(properties: Self::Properties) -> Box; + fn from_btreemap(btree_map: BTreeMap) -> Result; + fn new_operator(properties: Self::Properties) -> Result; + fn get_path(properties: Self::Properties) -> String; fn get_engine_type() -> EngineType; } @@ -133,7 +133,7 @@ impl TryFrom for FileSink { fn try_from(param: SinkParam) -> std::result::Result { let schema = param.schema(); - let config = *S::from_btreemap(param.properties)?; + let config = S::from_btreemap(param.properties)?; let path = S::get_path(config.clone()).to_string(); let op = S::new_operator(config.clone())?; let engine_type = S::get_engine_type(); @@ -153,7 +153,7 @@ impl TryFrom for FileSink { pub struct OpenDalSinkWriter { schema: SchemaRef, - operator: Box, + operator: Operator, sink_writer: Option, is_append_only: bool, write_path: String, @@ -219,7 +219,7 @@ impl SinkWriter for OpenDalSinkWriter { impl OpenDalSinkWriter { pub fn new( - operator: Box, + operator: Operator, write_path: &str, rw_schema: Schema, is_append_only: bool, diff --git a/src/connector/src/sink/file_sink/s3.rs b/src/connector/src/sink/file_sink/s3.rs index 263610c6bd207..417094600e61d 100644 --- a/src/connector/src/sink/file_sink/s3.rs +++ b/src/connector/src/sink/file_sink/s3.rs @@ -59,7 +59,7 @@ pub struct S3Config { pub const S3_SINK: &str = "s3"; impl FileSink { - pub fn new_s3_sink(config: S3Config) -> Result> { + pub fn new_s3_sink(config: S3Config) -> Result { // Create s3 builder. let mut builder = S3::default(); builder.bucket(&config.common.bucket_name); @@ -96,7 +96,7 @@ impl FileSink { .layer(RetryLayer::default()) .finish(); - Ok(Box::new(operator)) + Ok(operator) } } @@ -114,7 +114,7 @@ impl OpendalSinkBackend for S3Sink { const SINK_NAME: &'static str = S3_SINK; - fn from_btreemap(btree_map: BTreeMap) -> Result> { + fn from_btreemap(btree_map: BTreeMap) -> Result { let config = serde_json::from_value::(serde_json::to_value(btree_map).unwrap()) .map_err(|e| SinkError::Config(anyhow!(e)))?; if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { @@ -125,15 +125,15 @@ impl OpendalSinkBackend for S3Sink { SINK_TYPE_UPSERT ))); } - Ok(Box::new(config)) + Ok(config) } - fn new_operator(properties: S3Config) -> Result> { + fn new_operator(properties: S3Config) -> Result { FileSink::::new_s3_sink(properties) } - fn get_path(properties: Self::Properties) -> Box { - properties.common.path.into_boxed_str() + fn get_path(properties: Self::Properties) -> String { + properties.common.path } fn get_engine_type() -> super::opendal_sink::EngineType {