hlteoh37 commented on code in PR #1: URL: https://github.com/apache/flink-connector-dynamodb/pull/1#discussion_r998328370
########## flink-connector-aws-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkBuilder.java: ########## @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.dynamodb.sink; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.base.sink.AsyncSinkBaseBuilder; +import org.apache.flink.connector.base.sink.writer.ElementConverter; +import org.apache.flink.streaming.connectors.dynamodb.config.DynamoDbTablesConfig; + +import java.util.Optional; +import java.util.Properties; + +/** + * Builder to construct {@link DynamoDbSink}. + * + * <p>The following example shows the minimum setup to create a {@link DynamoDbSink} that writes + * records into DynamoDb + * + * <pre>{@code + * private static class DummyDynamoDbRequestConverter implements DynamoDbRequestConverter<String> { + * + * @Override + * public DynamoDbRequest apply(String s) { + * final Map<String, DynamoDbAttributeValue> item = new HashMap<>(); + * item.put("your-key", DynamoDbAttributeValue.builder().s(s).build()); + * return DynamoDbRequest.builder() + * .tableName("your-table-name") + * .putRequest(DynamoDbPutRequest.builder().item(item).build()) + * .build(); + * } + * } + * DynamoDbSink<String> dynamoDbSink = DynamoDbSink.<String>builder() + * .setDynamoDbRequestConverter(new DummyDynamoDbRequestConverter()) + * .build(); + * }</pre> + * + * <p>If the following parameters are not set in this builder, the following defaults will be used: + * + * <ul> + * <li>{@code maxBatchSize} will be 25 + * <li>{@code maxInFlightRequests} will be 50 + * <li>{@code maxBufferedRequests} will be 10000 + * <li>{@code maxBatchSizeInBytes} will be 16 MB i.e. {@code 16 * 1000 * 1000} + * <li>{@code maxTimeInBufferMS} will be 5000ms + * <li>{@code maxRecordSizeInBytes} will be 400 KB i.e. {@code 400 * 1000 * 1000} + * <li>{@code failOnError} will be false + * <li>{@code dynamoDbTablesConfig} will be empty meaning no records deduplication will be + * performed by the sink + * </ul> + * + * @param <InputT> type of elements that should be persisted in the destination + */ +@PublicEvolving +public class DynamoDbSinkBuilder<InputT> + extends AsyncSinkBaseBuilder<InputT, DynamoDbWriteRequest, DynamoDbSinkBuilder<InputT>> { + + private static final int DEFAULT_MAX_BATCH_SIZE = 25; + private static final int DEFAULT_MAX_IN_FLIGHT_REQUESTS = 50; Review Comment: > I am also thinking, that if we want to really support that we probably have to remove that deduplication that we are doing in case we find a duplicate key, and instead "split" the batch, to make sure we do not lose records with the same primary key IMO deduplication within a batch on the Flink DDB sink side is a useful feature, because it reduces writes to DDB. The current deduplication ensures the newest record takes precedence, which is the behaviour we see on consecutive `BatchWriteItem` requests. The difference I can see here is that the ChangeDataCapture stream off the DDB table will be different depending on whether deduping happens in Flink DDB Sink. This might be possible if the user wants to write to DDB but also track number of changes in the stream. This sounds like a relatively niche use case though, since this use case is more suited to a Streaming Destination like Kinesis. So, I think it makes sense to keep the deduping. @nirtsruya do you have a particular use case where splitting into batch would be preferable? > But if two entries with the same PK/SK get to two different in-flight batch request this is not a problem as deduplication then happens on DynamoDB side. > If the order of the duplicate/update entries matters I think batch sink can not be used anyway, as the order is mixed up by partial write failures/full retries and parallelism in the sink. Then it is up to user to dedup before the sink. As far as I can tell, the second request in DDB will take precedence and overwrite the first request. I can see Batching being very useful for Batch jobs where deduping already occured, and we just want to dump a large amt of data to DDB. For streaming use case, this situation is mitigated if user sets the `MAX_IN_FLIGHT_REQUESTS` to 1, because the `AsyncSinkWriter` will put any failed items from requests to the front of the queue, so the "older" items will be retried first. In the future, I was thinking we can implement an improvement to use `PutItem` API, and the DDB Sink can add an additional key-value with `eventTime` as the value. Whenever an item with older event time is written, we can just drop the record. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
