hlteoh37 commented on code in PR #1:
URL: 
https://github.com/apache/flink-connector-dynamodb/pull/1#discussion_r996321025


##########
flink-connector-aws-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSink.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.sink.AsyncSinkBase;
+import org.apache.flink.connector.base.sink.writer.BufferedRequestState;
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import 
org.apache.flink.streaming.connectors.dynamodb.config.DynamoDbTablesConfig;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A DynamoDB Sink that performs async requests against a destination stream 
using the buffering
+ * protocol specified in {@link AsyncSinkBase}.
+ *
+ * <p>The sink internally uses a {@link
+ * software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient} to 
communicate with the AWS
+ * endpoint.
+ *
+ * <p>The behaviour of the buffering may be specified by providing 
configuration during the sink
+ * build time.
+ *
+ * <ul>
+ *   <li>{@code maxBatchSize}: the maximum size of a batch of entries that may 
be written to
+ *       DynamoDb.
+ *   <li>{@code maxInFlightRequests}: the maximum number of in flight requests 
that may exist, if
+ *       any more in flight requests need to be initiated once the maximum has 
been reached, then it
+ *       will be blocked until some have completed
+ *   <li>{@code maxBufferedRequests}: the maximum number of elements held in 
the buffer, requests to
+ *       add elements will be blocked while the number of elements in the 
buffer is at the maximum

Review Comment:
   ```suggestion
    *   <li>{@code maxBufferedRequests}: the maximum number of elements held in 
the buffer, requests to
    *       sink will backpressure while the number of elements in the buffer 
is at the maximum
   ```
   
   Nit: This wording might be clearer from a Flink user's POV



##########
flink-connector-aws-dynamodb/src/test/java/org/apache/flink/streaming/connectors/dynamodb/sink/TableRequestsContainerTest.java:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.streaming.connectors.dynamodb.config.DynamoDbTablesConfig;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.PutRequest;
+import software.amazon.awssdk.services.dynamodb.model.WriteRequest;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Tests for {@link TableRequestsContainer}. */
+public class TableRequestsContainerTest {
+    @Test
+    public void testRequestNotDeduplicatedWhenNoTableConfig() {
+        new Scenario()
+                .witItem("table", ImmutableMap.of("pk", "1", "sk", "1"))
+                .witItem("table", ImmutableMap.of("pk", "1", "sk", "1"))
+                .withDescription("number of requests without deduplication 
configuration")
+                .withExpectedNumberOfItems(2)
+                .runScenario();
+    }
+
+    @Test
+    public void testDeduplicatedOnCompositeKey() {
+        new Scenario()
+                .withTableConfig("tableOne", "pk", "sk")
+                .witItem("tableOne", ImmutableMap.of("pk", "1", "sk", "1"))
+                // to be deduplicated
+                .witItem("tableOne", ImmutableMap.of("pk", "1", "sk", "1"))

Review Comment:
   Should we also validate that the newer item takes precedence?



##########
flink-connector-aws-dynamodb/src/test/java/org/apache/flink/streaming/connectors/dynamodb/util/AWSDynamoDbUtilTest.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.util;
+
+import org.apache.flink.connector.aws.util.TestUtil;
+import 
org.apache.flink.streaming.connectors.dynamodb.config.AWSDynamoDbConfigConstants;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration;
+import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption;
+import software.amazon.awssdk.core.retry.RetryPolicy;
+import software.amazon.awssdk.core.retry.backoff.BackoffStrategy;
+import software.amazon.awssdk.core.retry.backoff.EqualJitterBackoffStrategy;
+import software.amazon.awssdk.core.retry.backoff.FixedDelayBackoffStrategy;
+import software.amazon.awssdk.core.retry.backoff.FullJitterBackoffStrategy;
+
+import java.time.Duration;
+import java.util.Properties;
+
+/** Tests for {@link AWSDynamoDbUtil}. */
+public class AWSDynamoDbUtilTest {
+
+    @Test
+    public void testHasRetryConfiguration() {
+        Properties properties =
+                TestUtil.properties(
+                        
AWSDynamoDbConfigConstants.THROTTLING_BACKOFF_STRATEGY, "FULL_JITTER");
+
+        
Assertions.assertThat(AWSDynamoDbUtil.hasRetryConfiguration(properties))
+                .as("Should have retry configuration")
+                .isTrue();
+    }
+
+    @Test
+    public void testConstructsFullJitterBackoffStrategy() {
+        Properties properties = new Properties();
+        properties.put(AWSDynamoDbConfigConstants.BACKOFF_STRATEGY, 
"FULL_JITTER");
+        properties.put(AWSDynamoDbConfigConstants.FULL_JITTER_BASE_DELAY_MS, 
"100");
+        
properties.put(AWSDynamoDbConfigConstants.FULL_JITTER_MAX_BACKOFF_TIME_MS, 
"10");
+
+        BackoffStrategy backoffStrategy =
+                AWSDynamoDbUtil.getBackoffStrategy(
+                        properties, 
AWSDynamoDbConfigConstants.BACKOFF_STRATEGY);
+
+        
Assertions.assertThat(backoffStrategy).isInstanceOf(FullJitterBackoffStrategy.class);
+        Assertions.assertThat(((FullJitterBackoffStrategy) 
backoffStrategy).toBuilder().baseDelay())
+                .isEqualTo(Duration.ofMillis(100));
+        Assertions.assertThat(
+                        ((FullJitterBackoffStrategy) 
backoffStrategy).toBuilder().maxBackoffTime())
+                .isEqualTo(Duration.ofMillis(10));
+    }
+
+    @Test
+    public void testConstructsEqualJitterBackoffStrategy() {
+        Properties properties = new Properties();
+        properties.put(AWSDynamoDbConfigConstants.BACKOFF_STRATEGY, 
"EQUAL_JITTER");
+        properties.put(AWSDynamoDbConfigConstants.EQUAL_JITTER_BASE_DELAY_MS, 
"10");
+        
properties.put(AWSDynamoDbConfigConstants.EQUAL_JITTER_MAX_BACKOFF_TIME_MS, 
"100");
+
+        BackoffStrategy backoffStrategy =
+                AWSDynamoDbUtil.getBackoffStrategy(
+                        properties, 
AWSDynamoDbConfigConstants.BACKOFF_STRATEGY);
+
+        
Assertions.assertThat(backoffStrategy).isInstanceOf(EqualJitterBackoffStrategy.class);
+        Assertions.assertThat(
+                        ((EqualJitterBackoffStrategy) 
backoffStrategy).toBuilder().baseDelay())
+                .isEqualTo(Duration.ofMillis(10));
+        Assertions.assertThat(
+                        ((EqualJitterBackoffStrategy) 
backoffStrategy).toBuilder().maxBackoffTime())
+                .isEqualTo(Duration.ofMillis(100));
+    }
+
+    @Test
+    public void testConstructsFixedDelayBackoffStrategy() {
+        Properties properties = new Properties();
+        properties.put(AWSDynamoDbConfigConstants.THROTTLING_BACKOFF_STRATEGY, 
"FIXED_DELAY");
+        properties.put(AWSDynamoDbConfigConstants.FIXED_DELAY_BACKOFF_MS, 
"10");
+
+        BackoffStrategy backoffStrategy =
+                AWSDynamoDbUtil.getBackoffStrategy(
+                        properties, 
AWSDynamoDbConfigConstants.THROTTLING_BACKOFF_STRATEGY);
+
+        
Assertions.assertThat(backoffStrategy).isInstanceOf(FixedDelayBackoffStrategy.class);
+        Assertions.assertThat(backoffStrategy)
+                
.isEqualTo(FixedDelayBackoffStrategy.create(Duration.ofMillis(10)));
+    }
+
+    @Test
+    public void testConstructsRetryPolicy() {
+        Properties properties = new Properties();
+
+        properties.put(AWSDynamoDbConfigConstants.BACKOFF_STRATEGY, 
"FIXED_DELAY");
+        properties.put(AWSDynamoDbConfigConstants.THROTTLING_BACKOFF_STRATEGY, 
"FIXED_DELAY");
+
+        int expectedNumRetries = 100;
+        properties.put(
+                AWSDynamoDbConfigConstants.NUMBER_RETRIES, 
String.valueOf(expectedNumRetries));
+        int expectedDurationMs = 10;
+        BackoffStrategy expectedStrategy =
+                
FixedDelayBackoffStrategy.create(Duration.ofMillis(expectedDurationMs));
+        properties.put(
+                AWSDynamoDbConfigConstants.FIXED_DELAY_BACKOFF_MS,
+                String.valueOf(expectedDurationMs));
+
+        RetryPolicy policy = AWSDynamoDbUtil.getRetryPolicy(properties);
+
+        
Assertions.assertThat(policy.toBuilder().numRetries()).isEqualTo(expectedNumRetries);
+        
Assertions.assertThat(policy.toBuilder().backoffStrategy()).isEqualTo(expectedStrategy);
+        Assertions.assertThat(policy.toBuilder().throttlingBackoffStrategy())
+                .isEqualTo(expectedStrategy);
+    }
+
+    @Test
+    public void testConstructsOverrideConfigurationWithDefaults() {
+        Properties properties = new Properties();
+
+        ClientOverrideConfiguration configuration =
+                AWSDynamoDbUtil.getOverrideConfiguration(properties);
+
+        Assertions.assertThat(
+                        getAdvancedOption(configuration, 
SdkAdvancedClientOption.USER_AGENT_PREFIX))
+                
.isEqualTo(AWSDynamoDbUtil.getFlinkUserAgentPrefix(properties));
+
+        
Assertions.assertThat(configuration.retryPolicy().isPresent()).isTrue();

Review Comment:
   nit: For a better message when tests fail
   ```suggestion
           Assertions.assertThat(configuration.retryPolicy()).isEmpty();
   ```



##########
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>

Review Comment:
   ```suggestion
    * <pre>{@code
    * private static class DummyDynamoDbElementConverter implements 
ElementConverter<String, DynamoDbWriteRequest> {
    *
    *         @Override
    *         public DynamoDbWriteRequest apply(String s) {
    *             final Map<String, AttributeValue> item = new HashMap<>();
    *             item.put("your-key", AttributeValue.builder().s(s).build());
    *             return new DynamoDbWriteRequest(
    *                       "your-table-name",
    *                       WriteRequest.builder()
    *                           .putRequest(PutRequest.builder()
    *                               .item(item)
    *                               .build())
    *                           .build()
    *                   );
    *         }
    *     }
    * DynamoDbSink<String> dynamoDbSink = DynamoDbSink.<String>builder()
    *                                          .setElementConverter(new 
DummyDynamoDbElementConverter())
    *                                       .build();
    * }</pre>
   ```
   
   Updating to reflect latest builder code



##########
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}

Review Comment:
   ```suggestion
    *   <li>{@code maxRecordSizeInBytes} will be 400 KB i.e. {@code 400 * 1000}
   ```



##########
flink-connector-aws-dynamodb/src/test/java/org/apache/flink/streaming/connectors/dynamodb/sink/TableRequestsContainerTest.java:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.streaming.connectors.dynamodb.config.DynamoDbTablesConfig;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.PutRequest;
+import software.amazon.awssdk.services.dynamodb.model.WriteRequest;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Tests for {@link TableRequestsContainer}. */
+public class TableRequestsContainerTest {
+    @Test
+    public void testRequestNotDeduplicatedWhenNoTableConfig() {
+        new Scenario()
+                .witItem("table", ImmutableMap.of("pk", "1", "sk", "1"))
+                .witItem("table", ImmutableMap.of("pk", "1", "sk", "1"))
+                .withDescription("number of requests without deduplication 
configuration")
+                .withExpectedNumberOfItems(2)
+                .runScenario();
+    }
+
+    @Test
+    public void testDeduplicatedOnCompositeKey() {
+        new Scenario()
+                .withTableConfig("tableOne", "pk", "sk")
+                .witItem("tableOne", ImmutableMap.of("pk", "1", "sk", "1"))

Review Comment:
   nit: `withItem`



##########
flink-connector-aws-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkWriter.java:
##########
@@ -0,0 +1,170 @@
+/*
+ * 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.Internal;
+import org.apache.flink.api.connector.sink2.Sink.InitContext;
+import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter;
+import org.apache.flink.connector.base.sink.writer.BufferedRequestState;
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import 
org.apache.flink.streaming.connectors.dynamodb.config.DynamoDbTablesConfig;
+import org.apache.flink.streaming.connectors.dynamodb.util.AWSDynamoDbUtil;
+import 
org.apache.flink.streaming.connectors.dynamodb.util.DynamoDbExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
+import software.amazon.awssdk.services.dynamodb.model.BatchWriteItemRequest;
+import software.amazon.awssdk.services.dynamodb.model.BatchWriteItemResponse;
+import software.amazon.awssdk.services.dynamodb.model.WriteRequest;
+
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+
+/**
+ * Sink writer created by {@link DynamoDbSink} to write to DynamoDB. More 
details on the operation
+ * of this sink writer may be found in the doc for {@link DynamoDbSink}. More 
details on the
+ * internals of this sink writer may be found in {@link AsyncSinkWriter}.
+ *
+ * <p>The {@link DynamoDbAsyncClient} used here may be configured in the 
standard way for the AWS
+ * SDK 2.x. e.g. the provision of {@code AWS_REGION}, {@code 
AWS_ACCESS_KEY_ID} and {@code
+ * AWS_SECRET_ACCESS_KEY} through environment variables etc.
+ */
+@Internal
+class DynamoDbSinkWriter<InputT> extends AsyncSinkWriter<InputT, 
DynamoDbWriteRequest> {
+    private static final Logger LOG = 
LoggerFactory.getLogger(DynamoDbSinkWriter.class);
+
+    /* A counter for the total number of records that have encountered an 
error during put */
+    private final Counter numRecordsOutErrorsCounter;
+
+    /* The sink writer metric group */
+    private final SinkWriterMetricGroup metrics;
+
+    private final DynamoDbTablesConfig tablesConfig;
+    private final DynamoDbAsyncClient client;
+    private final boolean failOnError;
+
+    public DynamoDbSinkWriter(
+            ElementConverter<InputT, DynamoDbWriteRequest> elementConverter,
+            InitContext context,
+            int maxBatchSize,
+            int maxInFlightRequests,
+            int maxBufferedRequests,
+            long maxBatchSizeInBytes,
+            long maxTimeInBufferMS,
+            long maxRecordSizeInBytes,
+            boolean failOnError,
+            DynamoDbTablesConfig tablesConfig,
+            Properties dynamoDbClientProperties,
+            Collection<BufferedRequestState<DynamoDbWriteRequest>> states) {
+        super(
+                elementConverter,
+                context,
+                maxBatchSize,
+                maxInFlightRequests,
+                maxBufferedRequests,
+                maxBatchSizeInBytes,
+                maxTimeInBufferMS,
+                maxRecordSizeInBytes,
+                states);
+        this.failOnError = failOnError;
+        this.tablesConfig = tablesConfig;
+        this.metrics = context.metricGroup();
+        this.numRecordsOutErrorsCounter = 
metrics.getNumRecordsOutErrorsCounter();
+        this.client = AWSDynamoDbUtil.createClient(dynamoDbClientProperties);
+    }
+
+    @Override
+    protected void submitRequestEntries(
+            List<DynamoDbWriteRequest> requestEntries,
+            Consumer<List<DynamoDbWriteRequest>> requestResultConsumer) {
+
+        TableRequestsContainer container = new 
TableRequestsContainer(tablesConfig);
+        requestEntries.forEach(container::put);
+
+        CompletableFuture<BatchWriteItemResponse> future =
+                client.batchWriteItem(
+                        BatchWriteItemRequest.builder()
+                                .requestItems(container.getRequestItems())
+                                .build());
+
+        future.whenComplete(
+                (response, err) -> {
+                    if (err != null) {
+                        handleFullyFailedRequest(err, requestEntries, 
requestResultConsumer);
+                    } else if (response.unprocessedItems() != null
+                            && !response.unprocessedItems().isEmpty()) {
+                        handlePartiallyUnprocessedRequest(response, 
requestResultConsumer);
+                    } else {
+                        requestResultConsumer.accept(Collections.emptyList());
+                    }
+                });
+    }
+
+    private void handlePartiallyUnprocessedRequest(
+            BatchWriteItemResponse response, 
Consumer<List<DynamoDbWriteRequest>> requestResult) {
+        List<DynamoDbWriteRequest> unprocessed = new ArrayList<>();
+
+        for (String tableName : response.unprocessedItems().keySet()) {
+            for (WriteRequest request : 
response.unprocessedItems().get(tableName)) {
+                unprocessed.add(new DynamoDbWriteRequest(tableName, request));
+            }
+        }
+
+        LOG.warn("DynamoDB Sink failed to persist {} entries", 
unprocessed.size());

Review Comment:
   Nit: Might help to also mention these entries will be retried. (same below 
for `handleFullyFailedRequest()`



##########
flink-connector-aws-dynamodb/src/test/java/org/apache/flink/streaming/connectors/dynamodb/sink/TableRequestsContainerTest.java:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.streaming.connectors.dynamodb.config.DynamoDbTablesConfig;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.PutRequest;
+import software.amazon.awssdk.services.dynamodb.model.WriteRequest;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Tests for {@link TableRequestsContainer}. */
+public class TableRequestsContainerTest {
+    @Test
+    public void testRequestNotDeduplicatedWhenNoTableConfig() {
+        new Scenario()
+                .witItem("table", ImmutableMap.of("pk", "1", "sk", "1"))
+                .witItem("table", ImmutableMap.of("pk", "1", "sk", "1"))
+                .withDescription("number of requests without deduplication 
configuration")
+                .withExpectedNumberOfItems(2)
+                .runScenario();
+    }
+
+    @Test
+    public void testDeduplicatedOnCompositeKey() {
+        new Scenario()
+                .withTableConfig("tableOne", "pk", "sk")
+                .witItem("tableOne", ImmutableMap.of("pk", "1", "sk", "1"))
+                // to be deduplicated
+                .witItem("tableOne", ImmutableMap.of("pk", "1", "sk", "1"))
+                .witItem("tableOne", ImmutableMap.of("pk", "2", "sk", "1"))
+                .withDescription("total number of requests after 
deduplication")
+                .withExpectedNumberOfItems(2)
+                .runScenario();
+    }
+
+    @Test
+    public void testDeduplicatedOnPartitionKey() {
+        new Scenario()
+                .withTableConfig("tableOne", "pk")
+                .witItem("tableOne", ImmutableMap.of("pk", "1", "sk", "1"))
+                // to be deduplicated
+                .witItem("tableOne", ImmutableMap.of("pk", "1", "sk", "1"))
+                .witItem("tableOne", ImmutableMap.of("pk", "2", "sk", "1"))
+                .withDescription(
+                        "total number of requests after deduplication on 
partition key only")
+                .withExpectedNumberOfItems(2)
+                .runScenario();
+    }
+
+    @Test
+    public void testNotDeduplicatedWhenOnDifferentTable() {
+        new Scenario()
+                .withTableConfig("tableOne", "pk", "sk")
+                .withTableConfig("tableTwo", "pk", "sk")
+                .witItem("tableOne", ImmutableMap.of("pk", "1", "sk", "1"))
+                .witItem("tableOne", ImmutableMap.of("pk", "2", "sk", "1"))
+                .witItem("tableTwo", ImmutableMap.of("pk", "1", "sk", "1"))
+                .withDescription(
+                        "requests should not be deduplecated if belong to 
different tables")

Review Comment:
   ```suggestion
                           "requests should not be deduplicated if belong to 
different tables")
   ```



##########
flink-connector-aws-dynamodb/src/test/java/org/apache/flink/streaming/connectors/dynamodb/sink/TableRequestsContainerTest.java:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.streaming.connectors.dynamodb.config.DynamoDbTablesConfig;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.PutRequest;
+import software.amazon.awssdk.services.dynamodb.model.WriteRequest;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Tests for {@link TableRequestsContainer}. */
+public class TableRequestsContainerTest {
+    @Test
+    public void testRequestNotDeduplicatedWhenNoTableConfig() {
+        new Scenario()
+                .witItem("table", ImmutableMap.of("pk", "1", "sk", "1"))
+                .witItem("table", ImmutableMap.of("pk", "1", "sk", "1"))
+                .withDescription("number of requests without deduplication 
configuration")
+                .withExpectedNumberOfItems(2)
+                .runScenario();
+    }
+
+    @Test
+    public void testDeduplicatedOnCompositeKey() {
+        new Scenario()
+                .withTableConfig("tableOne", "pk", "sk")
+                .witItem("tableOne", ImmutableMap.of("pk", "1", "sk", "1"))
+                // to be deduplicated
+                .witItem("tableOne", ImmutableMap.of("pk", "1", "sk", "1"))
+                .witItem("tableOne", ImmutableMap.of("pk", "2", "sk", "1"))
+                .withDescription("total number of requests after 
deduplication")
+                .withExpectedNumberOfItems(2)
+                .runScenario();
+    }
+
+    @Test
+    public void testDeduplicatedOnPartitionKey() {
+        new Scenario()
+                .withTableConfig("tableOne", "pk")
+                .witItem("tableOne", ImmutableMap.of("pk", "1", "sk", "1"))
+                // to be deduplicated
+                .witItem("tableOne", ImmutableMap.of("pk", "1", "sk", "1"))
+                .witItem("tableOne", ImmutableMap.of("pk", "2", "sk", "1"))
+                .withDescription(
+                        "total number of requests after deduplication on 
partition key only")
+                .withExpectedNumberOfItems(2)
+                .runScenario();
+    }
+
+    @Test
+    public void testNotDeduplicatedWhenOnDifferentTable() {
+        new Scenario()
+                .withTableConfig("tableOne", "pk", "sk")
+                .withTableConfig("tableTwo", "pk", "sk")
+                .witItem("tableOne", ImmutableMap.of("pk", "1", "sk", "1"))
+                .witItem("tableOne", ImmutableMap.of("pk", "2", "sk", "1"))
+                .witItem("tableTwo", ImmutableMap.of("pk", "1", "sk", "1"))
+                .withDescription(
+                        "requests should not be deduplecated if belong to 
different tables")
+                .withExpectedNumberOfItems(3)
+                .runScenario();
+    }
+
+    private class Scenario {

Review Comment:
   Liking this Scenario class! Tests are super easy to read



##########
flink-connector-aws-dynamodb/src/test/java/org/apache/flink/streaming/connectors/dynamodb/sink/key/PrimaryKeyTest.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.key;
+
+import 
org.apache.flink.streaming.connectors.dynamodb.config.DynamoDbTablesConfig;
+import 
org.apache.flink.streaming.connectors.dynamodb.sink.InvalidRequestException;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+import software.amazon.awssdk.core.SdkBytes;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.DeleteRequest;
+import software.amazon.awssdk.services.dynamodb.model.PutRequest;
+import software.amazon.awssdk.services.dynamodb.model.WriteRequest;
+
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+/** Unit tests for {@link PrimaryKey}. */
+public class PrimaryKeyTest {
+
+    private static final String PARTITION_KEY_NAME = "part_key_name";
+    private static final String SORT_KEY_NAME = "sort_key_name";
+
+    private ImmutableMap<String, AttributeValue> createItemValues() {
+        return ImmutableMap.of(
+                PARTITION_KEY_NAME,
+                AttributeValue.builder()
+                        .s("123")
+                        .n("456")
+                        .b(SdkBytes.fromString("789", StandardCharsets.UTF_8))
+                        .build(),
+                SORT_KEY_NAME,
+                AttributeValue.builder().s("101112").build(),
+                "some_item",
+                AttributeValue.builder().bool(false).build());
+    }
+
+    public WriteRequest createPutItemRequest(Map<String, AttributeValue> 
itemValues) {
+        return WriteRequest.builder()
+                .putRequest(PutRequest.builder().item(itemValues).build())
+                .build();
+    }
+
+    public WriteRequest createDeleteItemRequest(Map<String, AttributeValue> 
itemValues) {
+        return WriteRequest.builder()
+                .deleteRequest(DeleteRequest.builder().key(itemValues).build())
+                .build();
+    }
+
+    @Test
+    public void testPartitionKeysOfTwoDifferentRequestsEqual() {

Review Comment:
   should we also test the case where `config` is `null`?



-- 
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]


Reply via email to