risingwave_connector/sink/file_sink/
s3.rsuse std::collections::{BTreeMap, HashMap};
use anyhow::anyhow;
use opendal::layers::{LoggingLayer, RetryLayer};
use opendal::services::S3;
use opendal::Operator;
use serde::Deserialize;
use serde_with::serde_as;
use with_options::WithOptions;
use super::opendal_sink::{BatchingStrategy, 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", alias = "snowflake.aws_region")]
pub region_name: String,
#[serde(rename = "s3.bucket_name", alias = "snowflake.s3_bucket")]
pub bucket_name: String,
#[serde(rename = "s3.path", alias = "snowflake.s3_path", default)]
pub path: Option<String>,
#[serde(
rename = "s3.credentials.access",
alias = "snowflake.aws_access_key_id",
default
)]
pub access: Option<String>,
#[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>,
#[serde(rename = "s3.assume_role", default)]
pub assume_role: Option<String>,
}
#[serde_as]
#[derive(Clone, Debug, Deserialize, WithOptions)]
pub struct S3Config {
#[serde(flatten)]
pub common: S3Common,
#[serde(flatten)]
pub batching_strategy: BatchingStrategy,
pub r#type: String, #[serde(flatten)]
pub unknown_fields: HashMap<String, String>,
}
pub const S3_SINK: &str = "s3";
impl<S: OpendalSinkBackend> FileSink<S> {
pub fn new_s3_sink(config: S3Config) -> Result<Operator> {
let mut builder = S3::default()
.bucket(&config.common.bucket_name)
.region(&config.common.region_name);
if let Some(endpoint_url) = config.common.endpoint_url {
builder = builder.endpoint(&endpoint_url);
}
if let Some(access) = config.common.access {
builder = builder.access_key_id(&access);
} else {
tracing::error!(
"access key id of aws s3 is not set, bucket {}",
config.common.bucket_name
);
}
if let Some(secret) = config.common.secret {
builder = builder.secret_access_key(&secret);
} else {
tracing::error!(
"secret access key of aws s3 is not set, bucket {}",
config.common.bucket_name
);
}
if let Some(assume_role) = config.common.assume_role {
builder = builder.role_arn(&assume_role);
}
builder = builder.disable_config_load();
let operator: Operator = Operator::new(builder)?
.layer(LoggingLayer::default())
.layer(RetryLayer::default())
.finish();
Ok(operator)
}
}
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
pub struct S3Sink;
impl UnknownFields for S3Config {
fn unknown_fields(&self) -> HashMap<String, String> {
self.unknown_fields.clone()
}
}
impl OpendalSinkBackend for S3Sink {
type Properties = S3Config;
const SINK_NAME: &'static str = S3_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::<S3Sink>::new_s3_sink(properties)
}
fn get_path(properties: Self::Properties) -> String {
properties.common.path.unwrap_or_default()
}
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,
}
}
}
#[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.unwrap_or_default()
}
fn get_engine_type() -> super::opendal_sink::EngineType {
super::opendal_sink::EngineType::Snowflake
}
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,
}
}
}