risingwave_connector/sink/file_sink/
s3.rs

1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14use std::collections::{BTreeMap, HashMap};
15
16use anyhow::anyhow;
17use opendal::Operator;
18use opendal::layers::{LoggingLayer, RetryLayer};
19use opendal::services::S3;
20use serde::Deserialize;
21use serde_with::serde_as;
22use with_options::WithOptions;
23
24use super::opendal_sink::{BatchingStrategy, FileSink};
25use crate::sink::file_sink::opendal_sink::OpendalSinkBackend;
26use crate::sink::{Result, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, SinkError};
27use crate::source::UnknownFields;
28#[derive(Deserialize, Debug, Clone, WithOptions)]
29pub struct S3Common {
30    #[serde(rename = "s3.region_name", alias = "snowflake.aws_region")]
31    pub region_name: String,
32    #[serde(rename = "s3.bucket_name", alias = "snowflake.s3_bucket")]
33    pub bucket_name: String,
34    /// The directory where the sink file is located.
35    #[serde(rename = "s3.path", alias = "snowflake.s3_path", default)]
36    pub path: Option<String>,
37    #[serde(
38        rename = "s3.credentials.access",
39        alias = "snowflake.aws_access_key_id",
40        default
41    )]
42    pub access: Option<String>,
43    #[serde(
44        rename = "s3.credentials.secret",
45        alias = "snowflake.aws_secret_access_key",
46        default
47    )]
48    pub secret: Option<String>,
49    #[serde(rename = "s3.endpoint_url")]
50    pub endpoint_url: Option<String>,
51    #[serde(rename = "s3.assume_role", default)]
52    pub assume_role: Option<String>,
53}
54
55#[serde_as]
56#[derive(Clone, Debug, Deserialize, WithOptions)]
57pub struct S3Config {
58    #[serde(flatten)]
59    pub common: S3Common,
60
61    #[serde(flatten)]
62    pub batching_strategy: BatchingStrategy,
63
64    pub r#type: String, // accept "append-only"
65
66    #[serde(flatten)]
67    pub unknown_fields: HashMap<String, String>,
68}
69
70pub const S3_SINK: &str = "s3";
71
72impl<S: OpendalSinkBackend> FileSink<S> {
73    pub fn new_s3_sink(config: S3Config) -> Result<Operator> {
74        // Create s3 builder.
75        let mut builder = S3::default()
76            .bucket(&config.common.bucket_name)
77            .region(&config.common.region_name);
78
79        if let Some(endpoint_url) = config.common.endpoint_url {
80            builder = builder.endpoint(&endpoint_url);
81        }
82
83        if let Some(access) = config.common.access {
84            builder = builder.access_key_id(&access);
85        } else {
86            tracing::error!(
87                "access key id of aws s3 is not set, bucket {}",
88                config.common.bucket_name
89            );
90        }
91
92        if let Some(secret) = config.common.secret {
93            builder = builder.secret_access_key(&secret);
94        } else {
95            tracing::error!(
96                "secret access key of aws s3 is not set, bucket {}",
97                config.common.bucket_name
98            );
99        }
100
101        if let Some(assume_role) = config.common.assume_role {
102            builder = builder.role_arn(&assume_role);
103        }
104        builder = builder.disable_config_load();
105        let operator: Operator = Operator::new(builder)?
106            .layer(LoggingLayer::default())
107            .layer(RetryLayer::default())
108            .finish();
109
110        Ok(operator)
111    }
112}
113
114#[derive(Debug, Clone, Copy, PartialEq, Eq)]
115pub struct S3Sink;
116
117impl UnknownFields for S3Config {
118    fn unknown_fields(&self) -> HashMap<String, String> {
119        self.unknown_fields.clone()
120    }
121}
122
123impl OpendalSinkBackend for S3Sink {
124    type Properties = S3Config;
125
126    const SINK_NAME: &'static str = S3_SINK;
127
128    fn from_btreemap(btree_map: BTreeMap<String, String>) -> Result<Self::Properties> {
129        let config = serde_json::from_value::<S3Config>(serde_json::to_value(btree_map).unwrap())
130            .map_err(|e| SinkError::Config(anyhow!(e)))?;
131        if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT {
132            return Err(SinkError::Config(anyhow!(
133                "`{}` must be {}, or {}",
134                SINK_TYPE_OPTION,
135                SINK_TYPE_APPEND_ONLY,
136                SINK_TYPE_UPSERT
137            )));
138        }
139        Ok(config)
140    }
141
142    fn new_operator(properties: S3Config) -> Result<Operator> {
143        FileSink::<S3Sink>::new_s3_sink(properties)
144    }
145
146    fn get_path(properties: Self::Properties) -> String {
147        properties.common.path.unwrap_or_default()
148    }
149
150    fn get_engine_type() -> super::opendal_sink::EngineType {
151        super::opendal_sink::EngineType::S3
152    }
153
154    fn get_batching_strategy(properties: Self::Properties) -> BatchingStrategy {
155        BatchingStrategy {
156            max_row_count: properties.batching_strategy.max_row_count,
157            rollover_seconds: properties.batching_strategy.rollover_seconds,
158            path_partition_prefix: properties.batching_strategy.path_partition_prefix,
159        }
160    }
161}
162
163#[derive(Debug, Clone, Copy, PartialEq, Eq)]
164pub struct SnowflakeSink;
165
166pub const SNOWFLAKE_SINK: &str = "snowflake";
167
168impl OpendalSinkBackend for SnowflakeSink {
169    type Properties = S3Config;
170
171    const SINK_NAME: &'static str = SNOWFLAKE_SINK;
172
173    fn from_btreemap(btree_map: BTreeMap<String, String>) -> Result<Self::Properties> {
174        let config = serde_json::from_value::<S3Config>(serde_json::to_value(btree_map).unwrap())
175            .map_err(|e| SinkError::Config(anyhow!(e)))?;
176        if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT {
177            return Err(SinkError::Config(anyhow!(
178                "`{}` must be {}, or {}",
179                SINK_TYPE_OPTION,
180                SINK_TYPE_APPEND_ONLY,
181                SINK_TYPE_UPSERT
182            )));
183        }
184        Ok(config)
185    }
186
187    fn new_operator(properties: S3Config) -> Result<Operator> {
188        FileSink::<SnowflakeSink>::new_s3_sink(properties)
189    }
190
191    fn get_path(properties: Self::Properties) -> String {
192        properties.common.path.unwrap_or_default()
193    }
194
195    fn get_engine_type() -> super::opendal_sink::EngineType {
196        super::opendal_sink::EngineType::Snowflake
197    }
198
199    fn get_batching_strategy(properties: Self::Properties) -> BatchingStrategy {
200        BatchingStrategy {
201            max_row_count: properties.batching_strategy.max_row_count,
202            rollover_seconds: properties.batching_strategy.rollover_seconds,
203            path_partition_prefix: properties.batching_strategy.path_partition_prefix,
204        }
205    }
206}