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

feat(connector): add DynamoDB sink #16670

Open
wants to merge 7 commits into
base: main
Choose a base branch
from
Open

Conversation

jetjinser
Copy link
Contributor

@jetjinser jetjinser commented May 9, 2024

I hereby agree to the terms of the RisingWave Labs, Inc. Contributor License Agreement.

What's changed and what's your intention?

1:1 mapping DynamoDB table sink.

Checklist

  • I have written necessary rustdoc comments
  • I have added necessary unit tests and integration tests
  • I have added test labels as necessary. See details.
  • I have added fuzzing tests or opened an issue to track them. (Optional, recommended for new SQL features Sqlsmith: Sql feature generation #7934).
  • My PR contains breaking changes. (If it deprecates some features, please create a tracking issue to remove them in the future).
  • All checks passed in ./risedev check (or alias, ./risedev c)
  • My PR changes performance-critical code. (Please run macro/micro-benchmarks and show the results.)
  • My PR contains critical fixes that are necessary to be merged into the latest release. (Please check out the details)

Documentation

  • My PR needs documentation updates. (Please use the Release note section below to summarize the impact on users)

Release note

If this PR includes changes that directly affect users or other significant modifications relevant to the community, kindly draft a release note to provide a concise summary of these changes. Please prioritize highlighting the impact these changes will have on users.

new sink:

CREATE TABLE IF NOT EXISTS movies (
    year integer,
    title varchar,
    description varchar,
    primary key (year, title)
);

CREATE SINK
  dyn_sink
FROM
  movies
WITH
(
  connector = 'dynamodb',
  table = 'Movies',
  primary_key = 'year,title',
  endpoint = 'http://localhost:8000',
  region = 'us',
  access_key = 'ac',
  secret_key = 'sk'
);

@jetjinser jetjinser added type/feature user-facing-changes Contains changes that are visible to users labels May 9, 2024
@jetjinser jetjinser marked this pull request as ready for review May 10, 2024 17:14
@jetjinser jetjinser requested a review from a team as a code owner May 10, 2024 17:14
Comment on lines 291 to 292
| ScalarRefImpl::Struct(_)
| ScalarRefImpl::Jsonb(_)) => AttributeValue::S(string.to_text()),
Copy link
Contributor

Choose a reason for hiding this comment

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

  • struct may be better mapped to DynamoDB Map
  • jsonb can be mapped as string for now, and latter extended to a dynamic recursive one similar to #11699. It is your call whether the latter dynamic one shall be the default here instead. Just bringing the alternative option to attention.

Copy link
Contributor Author

@jetjinser jetjinser May 13, 2024

Choose a reason for hiding this comment

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

  • I'm not sure, the struct here must be named, since in create sink?
  • Does this mean providing an TimestampHandlingMode-like option for jsonb format to DynamoDB sink?

Copy link
Contributor

Choose a reason for hiding this comment

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

You can consider struct as a list of named key values pairs, right?

struct <
  id varchar,
  name varchar
>
Map<String, Value> {
  "id": "123",
  "name": "jinser"
}

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, I think the current implementation is like this

let Some(scalar_ref) = scalar_ref else {
return Ok(AttributeValue::Null(true));
};
let attr = match (data_type, scalar_ref) {
Copy link
Contributor

Choose a reason for hiding this comment

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

It seems that matching the data_type only could make code a bit cleaner

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'm worried that the DataType is different from the ScalarRefImpl variant, and just matching data_type would mean requiring scalar_ref.into_foo() (which could panic in some case). Do you mean that this function map_data_type is only used here and does not need to consider possible the tow varant different situations?

Copy link
Contributor

@fuyufjh fuyufjh May 14, 2024

Choose a reason for hiding this comment

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

I think each DataType must have only one possible corresponding ScalarRefImpl, so no need to match ScalarRefImpl.

async fn write_chunk_inner(&mut self, chunk: StreamChunk) -> Result<()> {
for (op, row) in chunk.rows() {
match op {
Op::Insert | Op::UpdateInsert => {
Copy link
Contributor

@fuyufjh fuyufjh May 14, 2024

Choose a reason for hiding this comment

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

Have you ever tested an update event?

In self.payload_writer.write_chunk(), you inserted all the events before delete, which effectively means the UpdateInsert will happen before UpdateDelete. IIUC, the rows will be deleted rather than being updated.

Here, I prefer to handle UpdateInsert and UpdateDelete as a whole i.e. convert them into a single put KV into DynamoDB. There is no reason not doing so.

But keep in mind that this won't really solve the problem. Please carefully consider both of the following case:

  • An Insert comes after Delete on the same key
  • An Delete comes after Insert on the same key

Perhaps the current implementation of self.payload_writer.write_chunk() won't work.

Comment on lines 716 to 729
#[serde(rename = "aws.region")]
pub stream_region: String,
#[serde(rename = "aws.endpoint")]
pub endpoint: Option<String>,
#[serde(rename = "aws.credentials.access_key_id")]
pub credentials_access_key: Option<String>,
#[serde(rename = "aws.credentials.secret_access_key")]
pub credentials_secret_access_key: Option<String>,
#[serde(rename = "aws.credentials.session_token")]
pub session_token: Option<String>,
#[serde(rename = "aws.credentials.role.arn")]
pub assume_role_arn: Option<String>,
#[serde(rename = "aws.credentials.role.external_id")]
pub assume_role_external_id: Option<String>,
Copy link
Contributor

Choose a reason for hiding this comment

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

You can just put AwsAuthProps here.

@lmatz
Copy link
Contributor

lmatz commented May 20, 2024

DynamoDB can be deployed locally via docker image: https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/DynamoDBLocal.html
worth adding an integration test

@jetjinser
Copy link
Contributor Author

DynamoDB can be deployed locally via docker image: https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/DynamoDBLocal.html worth adding an integration test

Should I add it to this PR? Or should I open another PR?

@jetjinser
Copy link
Contributor Author

@xiangjinwu @fuyufjh @yuhao-su I've made the changes, please take a review when you have time 😃

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
type/feature user-facing-changes Contains changes that are visible to users
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

6 participants