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


##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkBuilder.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.List;
+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 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(
+ *                       WriteRequest.builder()
+ *                           .putRequest(PutRequest.builder()
+ *                               .item(item)
+ *                               .build())
+ *                           .build()
+ *                   );
+ *         }
+ *     }
+ * DynamoDbSink<String> dynamoDbSink = DynamoDbSink.<String>builder()
+ *                                          .setElementConverter(new 
DummyDynamoDbElementConverter())
+ *                                          
.setDestinationTableName("your-table-name")
+ *                                       .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}
+ *   <li>{@code failOnError} will be false
+ *   <li>{@code destinationTableName} destination table for the sink
+ *   <li>{@code overwriteByPKeys} will be empty meaning no records 
deduplication will be performed

Review Comment:
   This comment is outdated



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkBuilder.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.List;
+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 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(
+ *                       WriteRequest.builder()
+ *                           .putRequest(PutRequest.builder()
+ *                               .item(item)
+ *                               .build())
+ *                           .build()
+ *                   );
+ *         }
+ *     }
+ * DynamoDbSink<String> dynamoDbSink = DynamoDbSink.<String>builder()
+ *                                          .setElementConverter(new 
DummyDynamoDbElementConverter())
+ *                                          
.setDestinationTableName("your-table-name")
+ *                                       .build();

Review Comment:
   This comment is now outdated



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbWriteRequest.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.util.Preconditions;
+
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+
+import java.io.Serializable;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents a single Write Request to DynamoDb. Contains the item to be 
written as well as the
+ * type of the Write Request (PUT/DELETE)
+ */
+@PublicEvolving
+public class DynamoDbWriteRequest implements Serializable {

Review Comment:
   
https://flink.apache.org/contributing/code-style-and-quality-java.html#java-serialization
   
   > Serializable classes must define a Serial Version UID:
   > private static final long serialVersionUID = 1L;
   
   



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbWriteRequest.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.util.Preconditions;
+
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+
+import java.io.Serializable;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Represents a single Write Request to DynamoDb. Contains the item to be 
written as well as the
+ * type of the Write Request (PUT/DELETE)
+ */
+@PublicEvolving
+public class DynamoDbWriteRequest implements Serializable {
+
+    private final Map<String, AttributeValue> item;
+    private final DynamoDbWriteRequestType type;
+
+    private DynamoDbWriteRequest(Map<String, AttributeValue> item, 
DynamoDbWriteRequestType type) {
+        this.item = item;
+        this.type = type;
+    }
+
+    public Map<String, AttributeValue> getItem() {
+        return item;
+    }
+
+    public DynamoDbWriteRequestType getType() {
+        return type;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        DynamoDbWriteRequest that = (DynamoDbWriteRequest) o;
+        return item.equals(that.item) && type == that.type;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(item, type);
+    }

Review Comment:
   Why is this needed?



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/TableRequestsContainer.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.streaming.connectors.dynamodb.util.PrimaryKeyBuilder;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList;
+
+import software.amazon.awssdk.services.dynamodb.model.WriteRequest;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Container to accumulate requests in batches per table. De-duplicates batch 
request entities as
+ * per PrimaryKey definition. DynamoDB Batch API rejects the whole batch 
request if the request
+ * contains at least two items with identical hash and range keys (which 
essentially is two put
+ * operations).
+ */
+@Internal
+class TableRequestsContainer {
+
+    private final Map<String, WriteRequest> container;
+
+    private List<String> overwriteByPKeys;

Review Comment:
   Please update field name to match the other one `overwriteByPartitionKeys`



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkWriter.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.util.AWSDynamoDbUtil;
+import 
org.apache.flink.streaming.connectors.dynamodb.util.DynamoDbExceptionUtils;
+import org.apache.flink.util.CollectionUtil;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+
+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.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.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 numRecordsSendErrorsCounter;
+
+    /* The sink writer metric group */
+    private final SinkWriterMetricGroup metrics;
+
+    private final DynamoDbAsyncClient client;
+    private final boolean failOnError;
+    private final String tableName;
+
+    private List<String> overwriteByPKeys;

Review Comment:
   Can we update name to `overwriteByPartitionKeys` to match the other classes?



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkWriter.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.util.AWSDynamoDbUtil;
+import 
org.apache.flink.streaming.connectors.dynamodb.util.DynamoDbExceptionUtils;
+import org.apache.flink.util.CollectionUtil;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+
+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.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.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 numRecordsSendErrorsCounter;
+
+    /* The sink writer metric group */
+    private final SinkWriterMetricGroup metrics;
+
+    private final DynamoDbAsyncClient client;
+    private final boolean failOnError;
+    private final String tableName;
+
+    private List<String> overwriteByPKeys;
+
+    public DynamoDbSinkWriter(
+            ElementConverter<InputT, DynamoDbWriteRequest> elementConverter,
+            InitContext context,
+            int maxBatchSize,
+            int maxInFlightRequests,
+            int maxBufferedRequests,
+            long maxBatchSizeInBytes,
+            long maxTimeInBufferMS,
+            long maxRecordSizeInBytes,
+            boolean failOnError,
+            String tableName,
+            List<String> overwriteByPKeys,
+            Properties dynamoDbClientProperties,
+            Collection<BufferedRequestState<DynamoDbWriteRequest>> states) {
+        super(
+                elementConverter,
+                context,
+                maxBatchSize,
+                maxInFlightRequests,
+                maxBufferedRequests,
+                maxBatchSizeInBytes,
+                maxTimeInBufferMS,
+                maxRecordSizeInBytes,
+                states);
+        this.failOnError = failOnError;
+        this.tableName = tableName;
+        this.overwriteByPKeys = overwriteByPKeys;
+        this.metrics = context.metricGroup();
+        this.numRecordsSendErrorsCounter = 
metrics.getNumRecordsSendErrorsCounter();
+        this.client = AWSDynamoDbUtil.createClient(dynamoDbClientProperties);
+    }
+
+    @Override
+    protected void submitRequestEntries(
+            List<DynamoDbWriteRequest> requestEntries,
+            Consumer<List<DynamoDbWriteRequest>> requestResultConsumer) {
+
+        TableRequestsContainer container = new 
TableRequestsContainer(overwriteByPKeys);
+        for (DynamoDbWriteRequest requestEntry : requestEntries) {
+            container.put(convertToWriteRequest(requestEntry));
+        }
+
+        CompletableFuture<BatchWriteItemResponse> future =
+                client.batchWriteItem(
+                        BatchWriteItemRequest.builder()
+                                .requestItems(
+                                        ImmutableMap.of(tableName, 
container.getRequestItems()))
+                                .build());
+
+        future.whenComplete(
+                (response, err) -> {
+                    if (err != null) {
+                        handleFullyFailedRequest(err, requestEntries, 
requestResultConsumer);
+                    } else if 
(!CollectionUtil.isNullOrEmpty(response.unprocessedItems())) {
+                        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 writeRequest : 
response.unprocessedItems().get(tableName)) {
+                unprocessed.add(convertToDynamoDbWriteRequest(writeRequest));
+            }
+        }
+
+        LOG.warn(
+                "DynamoDB Sink failed to persist {} entries. Adding these 
entries back to retry.",
+                unprocessed.size());
+        numRecordsSendErrorsCounter.inc(unprocessed.size());
+
+        requestResult.accept(unprocessed);
+    }
+
+    private void handleFullyFailedRequest(
+            Throwable err,
+            List<DynamoDbWriteRequest> requestEntries,
+            Consumer<List<DynamoDbWriteRequest>> requestResult) {
+        LOG.warn(
+                "DynamoDB Sink failed to persist {} entries. Adding these 
entries back to retry.",
+                requestEntries.size(),
+                err);
+        numRecordsSendErrorsCounter.inc(requestEntries.size());
+
+        if (DynamoDbExceptionUtils.isNotRetryableException(err.getCause())) {
+            getFatalExceptionCons()
+                    .accept(
+                            new DynamoDbSinkException(
+                                    "Encountered non-recoverable exception", 
err));
+        } else if (failOnError) {
+            getFatalExceptionCons()
+                    .accept(new 
DynamoDbSinkException.DynamoDbSinkFailFastException(err));
+        } else {
+            requestResult.accept(requestEntries);
+        }
+    }
+
+    @Override
+    protected long getSizeInBytes(DynamoDbWriteRequest requestEntry) {
+        // dynamodb calculates item size as a sum of all attributes and all 
values, but doing so on
+        // every operation may be too expensive, so this is just an estimate
+        return 
requestEntry.getItem().toString().getBytes(StandardCharsets.UTF_8).length;
+    }
+
+    private WriteRequest convertToWriteRequest(DynamoDbWriteRequest 
dynamoDbWriteRequest) {
+        if (dynamoDbWriteRequest.getType() == DynamoDbWriteRequestType.PUT) {
+            return WriteRequest.builder()
+                    
.putRequest(PutRequest.builder().item(dynamoDbWriteRequest.getItem()).build())
+                    .build();
+        } else if (dynamoDbWriteRequest.getType() == 
DynamoDbWriteRequestType.DELETE) {
+            return WriteRequest.builder()
+                    .deleteRequest(
+                            
DeleteRequest.builder().key(dynamoDbWriteRequest.getItem()).build())
+                    .build();
+        } else {
+            throw new IllegalArgumentException("");
+        }
+    }
+
+    private DynamoDbWriteRequest convertToDynamoDbWriteRequest(WriteRequest 
writeRequest) {
+        if (writeRequest.putRequest() != null) {
+            return DynamoDbWriteRequest.build()
+                    .setItem(writeRequest.putRequest().item())
+                    .setType(DynamoDbWriteRequestType.PUT)
+                    .build();
+        } else if (writeRequest.deleteRequest() != null) {
+            return DynamoDbWriteRequest.build()
+                    .setItem(writeRequest.deleteRequest().key())
+                    .setType(DynamoDbWriteRequestType.DELETE)
+                    .build();
+        } else {
+            throw new IllegalArgumentException("");

Review Comment:
   Same as above



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkWriter.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.util.AWSDynamoDbUtil;
+import 
org.apache.flink.streaming.connectors.dynamodb.util.DynamoDbExceptionUtils;
+import org.apache.flink.util.CollectionUtil;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+
+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.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.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 numRecordsSendErrorsCounter;
+
+    /* The sink writer metric group */
+    private final SinkWriterMetricGroup metrics;
+
+    private final DynamoDbAsyncClient client;
+    private final boolean failOnError;
+    private final String tableName;
+
+    private List<String> overwriteByPKeys;
+
+    public DynamoDbSinkWriter(
+            ElementConverter<InputT, DynamoDbWriteRequest> elementConverter,
+            InitContext context,
+            int maxBatchSize,
+            int maxInFlightRequests,
+            int maxBufferedRequests,
+            long maxBatchSizeInBytes,
+            long maxTimeInBufferMS,
+            long maxRecordSizeInBytes,
+            boolean failOnError,
+            String tableName,
+            List<String> overwriteByPKeys,
+            Properties dynamoDbClientProperties,
+            Collection<BufferedRequestState<DynamoDbWriteRequest>> states) {
+        super(
+                elementConverter,
+                context,
+                maxBatchSize,
+                maxInFlightRequests,
+                maxBufferedRequests,
+                maxBatchSizeInBytes,
+                maxTimeInBufferMS,
+                maxRecordSizeInBytes,
+                states);
+        this.failOnError = failOnError;
+        this.tableName = tableName;
+        this.overwriteByPKeys = overwriteByPKeys;
+        this.metrics = context.metricGroup();
+        this.numRecordsSendErrorsCounter = 
metrics.getNumRecordsSendErrorsCounter();
+        this.client = AWSDynamoDbUtil.createClient(dynamoDbClientProperties);
+    }
+
+    @Override
+    protected void submitRequestEntries(
+            List<DynamoDbWriteRequest> requestEntries,
+            Consumer<List<DynamoDbWriteRequest>> requestResultConsumer) {
+
+        TableRequestsContainer container = new 
TableRequestsContainer(overwriteByPKeys);
+        for (DynamoDbWriteRequest requestEntry : requestEntries) {
+            container.put(convertToWriteRequest(requestEntry));
+        }
+
+        CompletableFuture<BatchWriteItemResponse> future =
+                client.batchWriteItem(
+                        BatchWriteItemRequest.builder()
+                                .requestItems(
+                                        ImmutableMap.of(tableName, 
container.getRequestItems()))
+                                .build());

Review Comment:
   As an optimization, we could completely skip `PrimaryKeyBuilder.build` when  
`CollectionUtil.isNullOrEmpty(overwriteByPKeys)`. This would remove the UUID 
generation and map overhead. Something like this:
   
   ```
           final List<WriteRequest> writeRequests;
           
           if (CollectionUtil.isNullOrEmpty(overwriteByPKeys)) {
               writeRequests = new ArrayList<>();
               for (DynamoDbWriteRequest requestEntry : requestEntries) {
                   writeRequests.put(convertToWriteRequest(requestEntry));
               }
           } else {
               TableRequestsContainer container = new 
TableRequestsContainer(overwriteByPKeys);
               for (DynamoDbWriteRequest requestEntry : requestEntries) {
                   container.put(convertToWriteRequest(requestEntry));
               }
   
               writeRequests = container.getRequestItems();
           }
   
           CompletableFuture<BatchWriteItemResponse> future =
                   client.batchWriteItem(
                           BatchWriteItemRequest.builder()
                                   .requestItems(ImmutableMap.of(tableName, 
writeRequests))
                                   .build());
   
   ```



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkWriter.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.util.AWSDynamoDbUtil;
+import 
org.apache.flink.streaming.connectors.dynamodb.util.DynamoDbExceptionUtils;
+import org.apache.flink.util.CollectionUtil;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+
+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.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.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 numRecordsSendErrorsCounter;
+
+    /* The sink writer metric group */
+    private final SinkWriterMetricGroup metrics;
+
+    private final DynamoDbAsyncClient client;
+    private final boolean failOnError;
+    private final String tableName;
+
+    private List<String> overwriteByPKeys;
+
+    public DynamoDbSinkWriter(
+            ElementConverter<InputT, DynamoDbWriteRequest> elementConverter,
+            InitContext context,
+            int maxBatchSize,
+            int maxInFlightRequests,
+            int maxBufferedRequests,
+            long maxBatchSizeInBytes,
+            long maxTimeInBufferMS,
+            long maxRecordSizeInBytes,
+            boolean failOnError,
+            String tableName,
+            List<String> overwriteByPKeys,
+            Properties dynamoDbClientProperties,
+            Collection<BufferedRequestState<DynamoDbWriteRequest>> states) {
+        super(
+                elementConverter,
+                context,
+                maxBatchSize,
+                maxInFlightRequests,
+                maxBufferedRequests,
+                maxBatchSizeInBytes,
+                maxTimeInBufferMS,
+                maxRecordSizeInBytes,
+                states);
+        this.failOnError = failOnError;
+        this.tableName = tableName;
+        this.overwriteByPKeys = overwriteByPKeys;
+        this.metrics = context.metricGroup();
+        this.numRecordsSendErrorsCounter = 
metrics.getNumRecordsSendErrorsCounter();
+        this.client = AWSDynamoDbUtil.createClient(dynamoDbClientProperties);
+    }
+
+    @Override
+    protected void submitRequestEntries(
+            List<DynamoDbWriteRequest> requestEntries,
+            Consumer<List<DynamoDbWriteRequest>> requestResultConsumer) {
+
+        TableRequestsContainer container = new 
TableRequestsContainer(overwriteByPKeys);
+        for (DynamoDbWriteRequest requestEntry : requestEntries) {
+            container.put(convertToWriteRequest(requestEntry));
+        }
+
+        CompletableFuture<BatchWriteItemResponse> future =
+                client.batchWriteItem(
+                        BatchWriteItemRequest.builder()
+                                .requestItems(
+                                        ImmutableMap.of(tableName, 
container.getRequestItems()))
+                                .build());
+
+        future.whenComplete(
+                (response, err) -> {
+                    if (err != null) {
+                        handleFullyFailedRequest(err, requestEntries, 
requestResultConsumer);
+                    } else if 
(!CollectionUtil.isNullOrEmpty(response.unprocessedItems())) {
+                        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 writeRequest : 
response.unprocessedItems().get(tableName)) {
+                unprocessed.add(convertToDynamoDbWriteRequest(writeRequest));
+            }
+        }
+
+        LOG.warn(
+                "DynamoDB Sink failed to persist {} entries. Adding these 
entries back to retry.",
+                unprocessed.size());
+        numRecordsSendErrorsCounter.inc(unprocessed.size());
+
+        requestResult.accept(unprocessed);
+    }
+
+    private void handleFullyFailedRequest(
+            Throwable err,
+            List<DynamoDbWriteRequest> requestEntries,
+            Consumer<List<DynamoDbWriteRequest>> requestResult) {
+        LOG.warn(
+                "DynamoDB Sink failed to persist {} entries. Adding these 
entries back to retry.",
+                requestEntries.size(),
+                err);
+        numRecordsSendErrorsCounter.inc(requestEntries.size());
+
+        if (DynamoDbExceptionUtils.isNotRetryableException(err.getCause())) {
+            getFatalExceptionCons()
+                    .accept(
+                            new DynamoDbSinkException(
+                                    "Encountered non-recoverable exception", 
err));
+        } else if (failOnError) {
+            getFatalExceptionCons()
+                    .accept(new 
DynamoDbSinkException.DynamoDbSinkFailFastException(err));
+        } else {
+            requestResult.accept(requestEntries);
+        }
+    }
+
+    @Override
+    protected long getSizeInBytes(DynamoDbWriteRequest requestEntry) {
+        // dynamodb calculates item size as a sum of all attributes and all 
values, but doing so on
+        // every operation may be too expensive, so this is just an estimate
+        return 
requestEntry.getItem().toString().getBytes(StandardCharsets.UTF_8).length;
+    }
+
+    private WriteRequest convertToWriteRequest(DynamoDbWriteRequest 
dynamoDbWriteRequest) {
+        if (dynamoDbWriteRequest.getType() == DynamoDbWriteRequestType.PUT) {
+            return WriteRequest.builder()
+                    
.putRequest(PutRequest.builder().item(dynamoDbWriteRequest.getItem()).build())
+                    .build();
+        } else if (dynamoDbWriteRequest.getType() == 
DynamoDbWriteRequestType.DELETE) {
+            return WriteRequest.builder()
+                    .deleteRequest(
+                            
DeleteRequest.builder().key(dynamoDbWriteRequest.getItem()).build())
+                    .build();
+        } else {
+            throw new IllegalArgumentException("");

Review Comment:
   Can we add a message to the exception? Given this should not happen we can 
quite explicitly say the `convertToWriteRequest` method must need updating



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