risingwave_connector/sink/
dynamodb.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.
14
15use std::collections::{BTreeMap, HashMap, HashSet};
16
17use anyhow::{Context, anyhow};
18use aws_sdk_dynamodb as dynamodb;
19use aws_sdk_dynamodb::client::Client;
20use aws_smithy_types::Blob;
21use dynamodb::types::{AttributeValue, TableStatus, WriteRequest};
22use futures::prelude::TryFuture;
23use futures::prelude::future::TryFutureExt;
24use risingwave_common::array::{Op, RowRef, StreamChunk};
25use risingwave_common::catalog::Schema;
26use risingwave_common::row::Row as _;
27use risingwave_common::types::{DataType, ScalarRefImpl, ToText};
28use risingwave_common::util::iter_util::ZipEqDebug;
29use serde_derive::Deserialize;
30use serde_with::{DisplayFromStr, serde_as};
31use with_options::WithOptions;
32use write_chunk_future::{DynamoDbPayloadWriter, WriteChunkFuture};
33
34use super::log_store::DeliveryFutureManagerAddFuture;
35use super::writer::{
36    AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt,
37};
38use super::{Result, Sink, SinkError, SinkParam, SinkWriterParam};
39use crate::connector_common::AwsAuthProps;
40use crate::enforce_secret::EnforceSecret;
41use crate::error::ConnectorResult;
42
43pub const DYNAMO_DB_SINK: &str = "dynamodb";
44
45#[serde_as]
46#[derive(Deserialize, Debug, Clone, WithOptions)]
47pub struct DynamoDbConfig {
48    #[serde(rename = "table", alias = "dynamodb.table")]
49    pub table: String,
50
51    #[serde(rename = "dynamodb.max_batch_rows", default = "default_max_batch_rows")]
52    #[serde_as(as = "DisplayFromStr")]
53    #[deprecated]
54    pub max_batch_rows: usize,
55
56    #[serde(flatten)]
57    pub aws_auth_props: AwsAuthProps,
58
59    #[serde(
60        rename = "dynamodb.max_batch_item_nums",
61        default = "default_max_batch_item_nums"
62    )]
63    #[serde_as(as = "DisplayFromStr")]
64    pub max_batch_item_nums: usize,
65
66    #[serde(
67        rename = "dynamodb.max_future_send_nums",
68        default = "default_max_future_send_nums"
69    )]
70    #[serde_as(as = "DisplayFromStr")]
71    pub max_future_send_nums: usize,
72}
73
74impl EnforceSecret for DynamoDbConfig {
75    fn enforce_one(prop: &str) -> crate::error::ConnectorResult<()> {
76        AwsAuthProps::enforce_one(prop)
77    }
78}
79
80fn default_max_batch_item_nums() -> usize {
81    25
82}
83
84fn default_max_future_send_nums() -> usize {
85    256
86}
87
88fn default_max_batch_rows() -> usize {
89    1024
90}
91
92impl DynamoDbConfig {
93    pub async fn build_client(&self) -> ConnectorResult<Client> {
94        let config = &self.aws_auth_props;
95        let aws_config = config.build_config().await?;
96
97        Ok(Client::new(&aws_config))
98    }
99
100    fn from_btreemap(values: BTreeMap<String, String>) -> Result<Self> {
101        serde_json::from_value::<DynamoDbConfig>(serde_json::to_value(values).unwrap())
102            .map_err(|e| SinkError::Config(anyhow!(e)))
103    }
104}
105
106#[derive(Clone, Debug)]
107pub struct DynamoDbSink {
108    pub config: DynamoDbConfig,
109    schema: Schema,
110    pk_indices: Vec<usize>,
111}
112
113impl EnforceSecret for DynamoDbSink {
114    fn enforce_secret<'a>(
115        prop_iter: impl Iterator<Item = &'a str>,
116    ) -> crate::error::ConnectorResult<()> {
117        for prop in prop_iter {
118            DynamoDbConfig::enforce_one(prop)?;
119        }
120        Ok(())
121    }
122}
123
124impl Sink for DynamoDbSink {
125    type LogSinker = AsyncTruncateLogSinkerOf<DynamoDbSinkWriter>;
126
127    const SINK_NAME: &'static str = DYNAMO_DB_SINK;
128
129    async fn validate(&self) -> Result<()> {
130        risingwave_common::license::Feature::DynamoDbSink
131            .check_available()
132            .map_err(|e| anyhow::anyhow!(e))?;
133        let client = (self.config.build_client().await)
134            .context("validate DynamoDB sink error")
135            .map_err(SinkError::DynamoDb)?;
136
137        let table_name = &self.config.table;
138        let output = client
139            .describe_table()
140            .table_name(table_name)
141            .send()
142            .await
143            .map_err(|e| anyhow!(e))?;
144        let Some(table) = output.table else {
145            return Err(SinkError::DynamoDb(anyhow!(
146                "table {} not found",
147                table_name
148            )));
149        };
150        if !matches!(table.table_status(), Some(TableStatus::Active)) {
151            return Err(SinkError::DynamoDb(anyhow!(
152                "table {} is not active",
153                table_name
154            )));
155        }
156        let pk_set: HashSet<String> = self
157            .schema
158            .fields()
159            .iter()
160            .enumerate()
161            .filter(|(k, _)| self.pk_indices.contains(k))
162            .map(|(_, v)| v.name.clone())
163            .collect();
164        let key_schema = table.key_schema();
165
166        for key_element in key_schema.iter().map(|x| x.attribute_name()) {
167            if !pk_set.contains(key_element) {
168                return Err(SinkError::DynamoDb(anyhow!(
169                    "table {} key field {} not found in schema or not primary key",
170                    table_name,
171                    key_element
172                )));
173            }
174        }
175
176        Ok(())
177    }
178
179    async fn new_log_sinker(&self, _writer_param: SinkWriterParam) -> Result<Self::LogSinker> {
180        Ok(
181            DynamoDbSinkWriter::new(self.config.clone(), self.schema.clone())
182                .await?
183                .into_log_sinker(self.config.max_future_send_nums),
184        )
185    }
186}
187
188impl TryFrom<SinkParam> for DynamoDbSink {
189    type Error = SinkError;
190
191    fn try_from(param: SinkParam) -> std::result::Result<Self, Self::Error> {
192        let schema = param.schema();
193        let config = DynamoDbConfig::from_btreemap(param.properties)?;
194
195        Ok(Self {
196            config,
197            schema,
198            pk_indices: param.downstream_pk,
199        })
200    }
201}
202
203#[derive(Debug)]
204struct DynamoDbRequest {
205    inner: WriteRequest,
206    key_items: Vec<String>,
207}
208
209impl DynamoDbRequest {
210    fn extract_pk_values(&self) -> Option<Vec<AttributeValue>> {
211        let key = match (&self.inner.put_request(), &self.inner.delete_request()) {
212            (Some(put_req), None) => &put_req.item,
213            (None, Some(del_req)) => &del_req.key,
214            _ => return None,
215        };
216        let vs = key
217            .iter()
218            .filter(|(k, _)| self.key_items.contains(k))
219            .map(|(_, v)| v.clone())
220            .collect();
221        Some(vs)
222    }
223}
224
225pub struct DynamoDbSinkWriter {
226    payload_writer: DynamoDbPayloadWriter,
227    formatter: DynamoDbFormatter,
228}
229
230impl DynamoDbSinkWriter {
231    pub async fn new(config: DynamoDbConfig, schema: Schema) -> Result<Self> {
232        let client = config.build_client().await?;
233        let table_name = &config.table;
234        let output = client
235            .describe_table()
236            .table_name(table_name)
237            .send()
238            .await
239            .map_err(|e| anyhow!(e))?;
240        let Some(table) = output.table else {
241            return Err(SinkError::DynamoDb(anyhow!(
242                "table {} not found",
243                table_name
244            )));
245        };
246        let dynamodb_keys = table
247            .key_schema
248            .unwrap_or_default()
249            .into_iter()
250            .map(|k| k.attribute_name)
251            .collect();
252
253        let payload_writer = DynamoDbPayloadWriter {
254            client,
255            table: config.table.clone(),
256            dynamodb_keys,
257            max_batch_item_nums: config.max_batch_item_nums,
258        };
259
260        Ok(Self {
261            payload_writer,
262            formatter: DynamoDbFormatter { schema },
263        })
264    }
265
266    fn write_chunk_inner(&mut self, chunk: StreamChunk) -> Result<WriteChunkFuture> {
267        let mut request_items = Vec::new();
268        for (op, row) in chunk.rows() {
269            let items = self.formatter.format_row(row)?;
270            match op {
271                Op::Insert | Op::UpdateInsert => {
272                    self.payload_writer
273                        .write_one_insert(items, &mut request_items);
274                }
275                Op::Delete => {
276                    self.payload_writer
277                        .write_one_delete(items, &mut request_items);
278                }
279                Op::UpdateDelete => {}
280            }
281        }
282        Ok(self.payload_writer.write_chunk(request_items))
283    }
284}
285
286pub type DynamoDbSinkDeliveryFuture = impl TryFuture<Ok = (), Error = SinkError> + Unpin + 'static;
287
288impl AsyncTruncateSinkWriter for DynamoDbSinkWriter {
289    type DeliveryFuture = DynamoDbSinkDeliveryFuture;
290
291    #[define_opaque(DynamoDbSinkDeliveryFuture)]
292    async fn write_chunk<'a>(
293        &'a mut self,
294        chunk: StreamChunk,
295        mut add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>,
296    ) -> Result<()> {
297        let futures = self.write_chunk_inner(chunk)?;
298        add_future
299            .add_future_may_await(futures.map_ok(|_: Vec<()>| ()))
300            .await?;
301        Ok(())
302    }
303}
304
305struct DynamoDbFormatter {
306    schema: Schema,
307}
308
309impl DynamoDbFormatter {
310    fn format_row(&self, row: RowRef<'_>) -> Result<HashMap<String, AttributeValue>> {
311        row.iter()
312            .zip_eq_debug((self.schema.clone()).into_fields())
313            .map(|(scalar, field)| {
314                map_data(scalar, &field.data_type()).map(|attr| (field.name, attr))
315            })
316            .collect()
317    }
318}
319
320fn map_data(scalar_ref: Option<ScalarRefImpl<'_>>, data_type: &DataType) -> Result<AttributeValue> {
321    let Some(scalar_ref) = scalar_ref else {
322        return Ok(AttributeValue::Null(true));
323    };
324    let attr = match data_type {
325        DataType::Int16
326        | DataType::Int32
327        | DataType::Int64
328        | DataType::Int256
329        | DataType::Float32
330        | DataType::Float64
331        | DataType::Decimal
332        | DataType::Serial => AttributeValue::N(scalar_ref.to_text_with_type(data_type)),
333        // TODO: jsonb as dynamic type (https://github.com/risingwavelabs/risingwave/issues/11699)
334        DataType::Varchar
335        | DataType::Interval
336        | DataType::Date
337        | DataType::Time
338        | DataType::Timestamp
339        | DataType::Timestamptz
340        | DataType::Jsonb => AttributeValue::S(scalar_ref.to_text_with_type(data_type)),
341        DataType::Boolean => AttributeValue::Bool(scalar_ref.into_bool()),
342        DataType::Bytea => AttributeValue::B(Blob::new(scalar_ref.into_bytea())),
343        DataType::List(datatype) => {
344            let list_attr = scalar_ref
345                .into_list()
346                .iter()
347                .map(|x| map_data(x, datatype))
348                .collect::<Result<Vec<_>>>()?;
349            AttributeValue::L(list_attr)
350        }
351        DataType::Struct(st) => {
352            let mut map = HashMap::with_capacity(st.len());
353            for (sub_datum_ref, (name, data_type)) in scalar_ref
354                .into_struct()
355                .iter_fields_ref()
356                .zip_eq_debug(st.iter())
357            {
358                let attr = map_data(sub_datum_ref, data_type)?;
359                map.insert(name.to_owned(), attr);
360            }
361            AttributeValue::M(map)
362        }
363        DataType::Map(_m) => {
364            return Err(SinkError::DynamoDb(anyhow!("map is not supported yet")));
365        }
366        DataType::Vector(_) => todo!("VECTOR_PLACEHOLDER"),
367    };
368    Ok(attr)
369}
370
371mod write_chunk_future {
372    use core::result;
373    use std::collections::HashMap;
374
375    use anyhow::anyhow;
376    use aws_sdk_dynamodb as dynamodb;
377    use aws_sdk_dynamodb::client::Client;
378    use aws_smithy_runtime_api::client::orchestrator::HttpResponse;
379    use dynamodb::error::SdkError;
380    use dynamodb::operation::batch_write_item::{BatchWriteItemError, BatchWriteItemOutput};
381    use dynamodb::types::{
382        AttributeValue, DeleteRequest, PutRequest, ReturnConsumedCapacity,
383        ReturnItemCollectionMetrics, WriteRequest,
384    };
385    use futures::future::{Map, TryJoinAll};
386    use futures::prelude::Future;
387    use futures::prelude::future::{FutureExt, try_join_all};
388    use itertools::Itertools;
389    use maplit::hashmap;
390
391    use super::{DynamoDbRequest, Result, SinkError};
392
393    pub type WriteChunkFuture = TryJoinAll<
394        Map<
395            impl Future<
396                Output = result::Result<
397                    BatchWriteItemOutput,
398                    SdkError<BatchWriteItemError, HttpResponse>,
399                >,
400            >,
401            impl FnOnce(
402                result::Result<BatchWriteItemOutput, SdkError<BatchWriteItemError, HttpResponse>>,
403            ) -> Result<()>,
404        >,
405    >;
406    pub struct DynamoDbPayloadWriter {
407        pub client: Client,
408        pub table: String,
409        pub dynamodb_keys: Vec<String>,
410        pub max_batch_item_nums: usize,
411    }
412
413    impl DynamoDbPayloadWriter {
414        pub fn write_one_insert(
415            &mut self,
416            item: HashMap<String, AttributeValue>,
417            request_items: &mut Vec<DynamoDbRequest>,
418        ) {
419            let put_req = PutRequest::builder().set_item(Some(item)).build().unwrap();
420            let req = WriteRequest::builder().put_request(put_req).build();
421            self.write_one_req(req, request_items);
422        }
423
424        pub fn write_one_delete(
425            &mut self,
426            key: HashMap<String, AttributeValue>,
427            request_items: &mut Vec<DynamoDbRequest>,
428        ) {
429            let key = key
430                .into_iter()
431                .filter(|(k, _)| self.dynamodb_keys.contains(k))
432                .collect();
433            let del_req = DeleteRequest::builder().set_key(Some(key)).build().unwrap();
434            let req = WriteRequest::builder().delete_request(del_req).build();
435            self.write_one_req(req, request_items);
436        }
437
438        pub fn write_one_req(
439            &mut self,
440            req: WriteRequest,
441            request_items: &mut Vec<DynamoDbRequest>,
442        ) {
443            let r_req = DynamoDbRequest {
444                inner: req,
445                key_items: self.dynamodb_keys.clone(),
446            };
447            if let Some(v) = r_req.extract_pk_values() {
448                request_items.retain(|item| {
449                    !item
450                        .extract_pk_values()
451                        .unwrap_or_default()
452                        .iter()
453                        .all(|x| v.contains(x))
454                });
455            }
456            request_items.push(r_req);
457        }
458
459        #[define_opaque(WriteChunkFuture)]
460        pub fn write_chunk(&mut self, request_items: Vec<DynamoDbRequest>) -> WriteChunkFuture {
461            let table = self.table.clone();
462            let chunks = request_items
463                .into_iter()
464                .map(|r| r.inner)
465                .chunks(self.max_batch_item_nums);
466            let futures = chunks.into_iter().map(|chunk| {
467                let req_items = chunk.collect();
468                let reqs = hashmap! {
469                    table.clone() => req_items,
470                };
471                self.client
472                    .batch_write_item()
473                    .set_request_items(Some(reqs))
474                    .return_consumed_capacity(ReturnConsumedCapacity::None)
475                    .return_item_collection_metrics(ReturnItemCollectionMetrics::None)
476                    .send()
477                    .map(|result| {
478                        result
479                            .map_err(|e| {
480                                SinkError::DynamoDb(
481                                    anyhow!(e).context("failed to delete item from DynamoDB sink"),
482                                )
483                            })
484                            .map(|_| ())
485                    })
486            });
487            try_join_all(futures)
488        }
489    }
490}