Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

refactor(sink): Use s3 sink to replace the original snowflake backend implementation #18996

Merged
merged 9 commits into from
Nov 7, 2024
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
11 changes: 6 additions & 5 deletions e2e_test/s3/fs_sink.py
Original file line number Diff line number Diff line change
Expand Up @@ -183,6 +183,7 @@ def _table():
stmt = f'select count(*), sum(id) from test_parquet_sink_table'
print(f'Execute reading sink files: {stmt}')

print(f'Create snowflake s3 sink ')
# Execute a SELECT statement
cur.execute(f'''CREATE sink test_file_sink_json as select
id,
Expand All @@ -199,12 +200,12 @@ def _table():
test_timestamp,
test_timestamptz
from {_table()} WITH (
connector = 's3',
connector = 'snowflake',
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is weird to test snowflake sink in fs sink test. Let's keep the previous codes instead.

match_pattern = '*.parquet',
s3.region_name = 'custom',
s3.bucket_name = 'hummock001',
s3.credentials.access = 'hummockadmin',
s3.credentials.secret = 'hummockadmin',
snowflake.aws_region = 'custom',
snowflake.s3_bucket = 'hummock001',
snowflake.aws_access_key_id = 'hummockadmin',
snowflake.aws_secret_access_key = 'hummockadmin',
s3.endpoint_url = 'http://hummock001.127.0.0.1:9301',
s3.path = 'test_json_sink/',
s3.file_type = 'json',
wcy-fdu marked this conversation as resolved.
Show resolved Hide resolved
Expand Down
63 changes: 58 additions & 5 deletions src/connector/src/sink/file_sink/s3.rs
Original file line number Diff line number Diff line change
Expand Up @@ -27,16 +27,24 @@ use crate::sink::{Result, SinkError, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SI
use crate::source::UnknownFields;
#[derive(Deserialize, Debug, Clone, WithOptions)]
pub struct S3Common {
#[serde(rename = "s3.region_name")]
#[serde(rename = "s3.region_name", alias = "snowflake.aws_region")]
pub region_name: String,
#[serde(rename = "s3.bucket_name")]
#[serde(rename = "s3.bucket_name", alias = "snowflake.s3_bucket")]
pub bucket_name: String,
/// The directory where the sink file is located.
#[serde(rename = "s3.path")]
#[serde(rename = "s3.path", alias = "snowflake.s3_path")]
pub path: String,
#[serde(rename = "s3.credentials.access", default)]
#[serde(
rename = "s3.credentials.access",
alias = "snowflake.aws_access_key_id",
default
)]
pub access: Option<String>,
#[serde(rename = "s3.credentials.secret", default)]
#[serde(
rename = "s3.credentials.secret",
alias = "snowflake.aws_secret_access_key",
default
)]
pub secret: Option<String>,
#[serde(rename = "s3.endpoint_url")]
pub endpoint_url: Option<String>,
Expand Down Expand Up @@ -151,3 +159,48 @@ impl OpendalSinkBackend for S3Sink {
}
}
}

#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub struct SnowflakeSink;

pub const SNOWFLAKE_SINK: &str = "snowflake";

impl OpendalSinkBackend for SnowflakeSink {
type Properties = S3Config;

const SINK_NAME: &'static str = SNOWFLAKE_SINK;

fn from_btreemap(btree_map: BTreeMap<String, String>) -> Result<Self::Properties> {
let config = serde_json::from_value::<S3Config>(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!(
"`{}` must be {}, or {}",
SINK_TYPE_OPTION,
SINK_TYPE_APPEND_ONLY,
SINK_TYPE_UPSERT
)));
}
Ok(config)
}

fn new_operator(properties: S3Config) -> Result<Operator> {
FileSink::<SnowflakeSink>::new_s3_sink(properties)
}

fn get_path(properties: Self::Properties) -> String {
properties.common.path
}

fn get_engine_type() -> super::opendal_sink::EngineType {
super::opendal_sink::EngineType::S3
}

fn get_batching_strategy(properties: Self::Properties) -> BatchingStrategy {
BatchingStrategy {
max_row_count: properties.batching_strategy.max_row_count,
rollover_seconds: properties.batching_strategy.rollover_seconds,
path_partition_prefix: properties.batching_strategy.path_partition_prefix,
}
}
}
15 changes: 7 additions & 8 deletions src/connector/src/sink/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@ pub mod nats;
pub mod pulsar;
pub mod redis;
pub mod remote;
pub mod snowflake;
pub mod sqlserver;
pub mod starrocks;
pub mod test_sink;
Expand Down Expand Up @@ -128,7 +127,7 @@ macro_rules! for_all_sinks {
{ Webhdfs, $crate::sink::file_sink::opendal_sink::FileSink<$crate::sink::file_sink::webhdfs::WebhdfsSink>},

{ Fs, $crate::sink::file_sink::opendal_sink::FileSink<FsSink> },
{ Snowflake, $crate::sink::snowflake::SnowflakeSink },
{ Snowflake, $crate::sink::file_sink::opendal_sink::FileSink<$crate::sink::file_sink::s3::SnowflakeSink>},
{ DeltaLake, $crate::sink::deltalake::DeltaLakeSink },
{ BigQuery, $crate::sink::big_query::BigQuerySink },
{ DynamoDb, $crate::sink::dynamodb::DynamoDbSink },
Expand Down Expand Up @@ -816,12 +815,12 @@ pub enum SinkError {
Starrocks(String),
#[error("File error: {0}")]
File(String),
#[error("Snowflake error: {0}")]
Snowflake(
#[source]
#[backtrace]
anyhow::Error,
),
// #[error("Snowflake error: {0}")]
wcy-fdu marked this conversation as resolved.
Show resolved Hide resolved
// Snowflake(
// #[source]
// #[backtrace]
// anyhow::Error,
// ),
#[error("Pulsar error: {0}")]
Pulsar(
#[source]
Expand Down
Loading
Loading