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


##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSink.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A DynamoDB Sink that performs async requests against a destination table 
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
+ *       sink will backpressure while the number of elements in the buffer is 
at the maximum
+ *   <li>{@code maxBatchSizeInBytes}: the maximum size of a batch of entries 
that may be written to
+ *       DynamoDb measured in bytes
+ *   <li>{@code maxTimeInBufferMS}: the maximum amount of time an entry is 
allowed to live in the
+ *       buffer, if any element reaches this age, the entire buffer will be 
flushed immediately
+ *   <li>{@code maxRecordSizeInBytes}: the maximum size of a record the sink 
will accept into the
+ *       buffer, a record of size larger than this will be rejected when 
passed to the sink
+ *   <li>{@code failOnError}: when an exception is encountered while 
persisting to DynamoDb, the job
+ *       will fail immediately if failOnError is set
+ *   <li>{@code dynamoDbTablesConfig}: if provided for the table, the DynamoDb 
sink will attempt to
+ *       deduplicate records with the same primary and/or secondary keys in 
the same batch request.
+ *       Only the latest record with the same combination of key attributes is 
preserved in the
+ *       request.
+ * </ul>
+ *
+ * <p>Please see the writer implementation in {@link DynamoDbSinkWriter}
+ *
+ * @param <InputT> Type of the elements handled by this sink
+ */
+@PublicEvolving
+public class DynamoDbSink<InputT> extends AsyncSinkBase<InputT, 
DynamoDbWriteRequest> {
+
+    private final Properties dynamoDbClientProperties;
+    private final boolean failOnError;
+    private final String tableName;
+    private List<String> overwriteByPKeys;
+
+    protected DynamoDbSink(
+            ElementConverter<InputT, DynamoDbWriteRequest> elementConverter,
+            int maxBatchSize,
+            int maxInFlightRequests,
+            int maxBufferedRequests,
+            long maxBatchSizeInBytes,
+            long maxTimeInBufferMS,
+            long maxRecordSizeInBytes,
+            boolean failOnError,
+            String tableName,
+            List<String> overwriteByPKeys,
+            Properties dynamoDbClientProperties) {
+        super(
+                elementConverter,
+                maxBatchSize,
+                maxInFlightRequests,
+                maxBufferedRequests,
+                maxBatchSizeInBytes,
+                maxTimeInBufferMS,
+                maxRecordSizeInBytes);
+        checkNotNull(tableName, "Destination table name must be set.");

Review Comment:
   How about checking for empty string too?



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSink.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A DynamoDB Sink that performs async requests against a destination table 
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
+ *       sink will backpressure while the number of elements in the buffer is 
at the maximum
+ *   <li>{@code maxBatchSizeInBytes}: the maximum size of a batch of entries 
that may be written to
+ *       DynamoDb measured in bytes
+ *   <li>{@code maxTimeInBufferMS}: the maximum amount of time an entry is 
allowed to live in the
+ *       buffer, if any element reaches this age, the entire buffer will be 
flushed immediately
+ *   <li>{@code maxRecordSizeInBytes}: the maximum size of a record the sink 
will accept into the
+ *       buffer, a record of size larger than this will be rejected when 
passed to the sink
+ *   <li>{@code failOnError}: when an exception is encountered while 
persisting to DynamoDb, the job
+ *       will fail immediately if failOnError is set
+ *   <li>{@code dynamoDbTablesConfig}: if provided for the table, the DynamoDb 
sink will attempt to
+ *       deduplicate records with the same primary and/or secondary keys in 
the same batch request.
+ *       Only the latest record with the same combination of key attributes is 
preserved in the
+ *       request.
+ * </ul>
+ *
+ * <p>Please see the writer implementation in {@link DynamoDbSinkWriter}
+ *
+ * @param <InputT> Type of the elements handled by this sink
+ */
+@PublicEvolving
+public class DynamoDbSink<InputT> extends AsyncSinkBase<InputT, 
DynamoDbWriteRequest> {
+
+    private final Properties dynamoDbClientProperties;
+    private final boolean failOnError;
+    private final String tableName;
+    private List<String> overwriteByPKeys;
+
+    protected DynamoDbSink(
+            ElementConverter<InputT, DynamoDbWriteRequest> elementConverter,
+            int maxBatchSize,
+            int maxInFlightRequests,
+            int maxBufferedRequests,
+            long maxBatchSizeInBytes,
+            long maxTimeInBufferMS,
+            long maxRecordSizeInBytes,
+            boolean failOnError,
+            String tableName,
+            List<String> overwriteByPKeys,

Review Comment:
   Should we null check this too?



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkWriter.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.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.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 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.numRecordsOutErrorsCounter = 
metrics.getNumRecordsOutErrorsCounter();
+        this.client = AWSDynamoDbUtil.createClient(dynamoDbClientProperties);
+    }
+
+    @Override
+    protected void submitRequestEntries(
+            List<DynamoDbWriteRequest> requestEntries,
+            Consumer<List<DynamoDbWriteRequest>> requestResultConsumer) {
+
+        TableRequestsContainer container = new 
TableRequestsContainer(overwriteByPKeys);
+        requestEntries.forEach(container::put);
+
+        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 (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 (WriteRequest request : 
response.unprocessedItems().get(tableName)) {
+            unprocessed.add(new DynamoDbWriteRequest(request));
+        }
+
+        LOG.warn(
+                "DynamoDB Sink failed to persist {} entries. Adding these 
entries back to retry.",
+                unprocessed.size());
+        numRecordsOutErrorsCounter.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);
+        numRecordsOutErrorsCounter.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.getWriteRequest().toString().getBytes(StandardCharsets.UTF_8).length;

Review Comment:
   > requestEntry.getWriteRequest().toString()
   
   What does this return? It might also be expensive to call this, depending on 
the implementation. I agree though this is not an easy thing to compute here, I 
had the same issue with the CloudWatch metrics sink



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkWriter.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.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.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 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.numRecordsOutErrorsCounter = 
metrics.getNumRecordsOutErrorsCounter();
+        this.client = AWSDynamoDbUtil.createClient(dynamoDbClientProperties);
+    }
+
+    @Override
+    protected void submitRequestEntries(
+            List<DynamoDbWriteRequest> requestEntries,
+            Consumer<List<DynamoDbWriteRequest>> requestResultConsumer) {
+
+        TableRequestsContainer container = new 
TableRequestsContainer(overwriteByPKeys);
+        requestEntries.forEach(container::put);

Review Comment:
   nit: From reading this it is not clear what this does. We could rename the 
class to increase readability.
   



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkWriter.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.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.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 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.numRecordsOutErrorsCounter = 
metrics.getNumRecordsOutErrorsCounter();
+        this.client = AWSDynamoDbUtil.createClient(dynamoDbClientProperties);
+    }
+
+    @Override
+    protected void submitRequestEntries(
+            List<DynamoDbWriteRequest> requestEntries,
+            Consumer<List<DynamoDbWriteRequest>> requestResultConsumer) {
+
+        TableRequestsContainer container = new 
TableRequestsContainer(overwriteByPKeys);
+        requestEntries.forEach(container::put);
+
+        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 (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 (WriteRequest request : 
response.unprocessedItems().get(tableName)) {
+            unprocessed.add(new DynamoDbWriteRequest(request));
+        }
+
+        LOG.warn(
+                "DynamoDB Sink failed to persist {} entries. Adding these 
entries back to retry.",
+                unprocessed.size());
+        numRecordsOutErrorsCounter.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);
+        numRecordsOutErrorsCounter.inc(requestEntries.size());
+
+        if (DynamoDbExceptionUtils.isNotRetryableException(err.getCause())) {

Review Comment:
   Unfortunately the async sink base does not perform ANY error classification. 
This means we also need to handle other types of non-recoverable errors. We 
implemented some base helpers for this, please see how it is done in KDS/KDF: 
https://github.com/apache/flink/blob/f53dd03fd5fe87a366de0da1fdef4ad88d5b20af/flink-connectors/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/sink/KinesisStreamsSinkWriter.java#L73



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkWriter.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.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.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 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.numRecordsOutErrorsCounter = 
metrics.getNumRecordsOutErrorsCounter();
+        this.client = AWSDynamoDbUtil.createClient(dynamoDbClientProperties);
+    }
+
+    @Override
+    protected void submitRequestEntries(
+            List<DynamoDbWriteRequest> requestEntries,
+            Consumer<List<DynamoDbWriteRequest>> requestResultConsumer) {
+
+        TableRequestsContainer container = new 
TableRequestsContainer(overwriteByPKeys);
+        requestEntries.forEach(container::put);
+
+        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 (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 (WriteRequest request : 
response.unprocessedItems().get(tableName)) {
+            unprocessed.add(new DynamoDbWriteRequest(request));
+        }
+
+        LOG.warn(
+                "DynamoDB Sink failed to persist {} entries. Adding these 
entries back to retry.",
+                unprocessed.size());
+        numRecordsOutErrorsCounter.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);
+        numRecordsOutErrorsCounter.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.getWriteRequest().toString().getBytes(StandardCharsets.UTF_8).length;
+    }

Review Comment:
   You need to implement `close()` and close both the DDB client and Http client



##########
flink-connector-dynamodb/src/test/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkBuilderTest.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+
+import java.util.Map;
+
+/** Tests for {@link DynamoDbSinkBuilder}. */
+public class DynamoDbSinkBuilderTest {
+
+    @Test

Review Comment:
   We should be using junit 5 (jupiter) and Asseert4J for new code as per the 
[coding 
guide](https://flink.apache.org/contributing/code-style-and-quality-common.html#tooling).
 Please migrate tests



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbWriteRequest.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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 software.amazon.awssdk.services.dynamodb.model.WriteRequest;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+/**
+ * Represents a single DynamoDb {@link WriteRequest}. TODO remove this class, 
replace with interface
+ * to support batch and non-batch modes
+ */
+@Internal
+public class DynamoDbWriteRequest implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final WriteRequest writeRequest;
+
+    public DynamoDbWriteRequest(WriteRequest writeRequest) {
+        this.writeRequest = writeRequest;
+    }
+
+    public WriteRequest getWriteRequest() {
+        return writeRequest;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        DynamoDbWriteRequest that = (DynamoDbWriteRequest) o;
+        return Objects.equals(writeRequest, that.writeRequest);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(writeRequest);
+    }

Review Comment:
   Why do we need these?



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkWriter.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.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.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 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.numRecordsOutErrorsCounter = 
metrics.getNumRecordsOutErrorsCounter();
+        this.client = AWSDynamoDbUtil.createClient(dynamoDbClientProperties);

Review Comment:
   We *could* remove the whole retry policy thing from the initial connector 
and add support when we implement 
https://issues.apache.org/jira/browse/FLINK-29683. This way we do not need to 
maintain 2 sets of keys too.



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSink.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A DynamoDB Sink that performs async requests against a destination table 
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
+ *       sink will backpressure while the number of elements in the buffer is 
at the maximum
+ *   <li>{@code maxBatchSizeInBytes}: the maximum size of a batch of entries 
that may be written to
+ *       DynamoDb measured in bytes
+ *   <li>{@code maxTimeInBufferMS}: the maximum amount of time an entry is 
allowed to live in the
+ *       buffer, if any element reaches this age, the entire buffer will be 
flushed immediately
+ *   <li>{@code maxRecordSizeInBytes}: the maximum size of a record the sink 
will accept into the
+ *       buffer, a record of size larger than this will be rejected when 
passed to the sink
+ *   <li>{@code failOnError}: when an exception is encountered while 
persisting to DynamoDb, the job
+ *       will fail immediately if failOnError is set
+ *   <li>{@code dynamoDbTablesConfig}: if provided for the table, the DynamoDb 
sink will attempt to
+ *       deduplicate records with the same primary and/or secondary keys in 
the same batch request.
+ *       Only the latest record with the same combination of key attributes is 
preserved in the
+ *       request.
+ * </ul>
+ *
+ * <p>Please see the writer implementation in {@link DynamoDbSinkWriter}
+ *
+ * @param <InputT> Type of the elements handled by this sink
+ */
+@PublicEvolving
+public class DynamoDbSink<InputT> extends AsyncSinkBase<InputT, 
DynamoDbWriteRequest> {
+
+    private final Properties dynamoDbClientProperties;
+    private final boolean failOnError;
+    private final String tableName;
+    private List<String> overwriteByPKeys;

Review Comment:
   This can be `final` and can we expand the `P` abbreviation to make it more 
readable? 



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/util/AWSDynamoDbUtil.java:
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.connector.aws.config.AWSConfigConstants;
+import org.apache.flink.connector.aws.util.AWSGeneralUtil;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+import 
org.apache.flink.streaming.connectors.dynamodb.config.AWSDynamoDbConfigConstants;
+
+import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration;
+import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption;
+import software.amazon.awssdk.core.client.config.SdkClientConfiguration;
+import software.amazon.awssdk.core.client.config.SdkClientOption;
+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 software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient;
+import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClientBuilder;
+
+import java.net.URI;
+import java.time.Duration;
+import java.util.Optional;
+import java.util.Properties;
+
+/** Some utilities specific to Amazon DynamoDB. */
+@Internal
+public class AWSDynamoDbUtil extends AWSGeneralUtil {

Review Comment:
   We do not need to extend `AWSGeneralUtil`. This was done in Kinesis for 
backwards compatibility



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/util/DynamoDbSerializationUtil.java:
##########
@@ -0,0 +1,214 @@
+/*
+ * 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.annotation.Internal;
+
+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.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Serialization Utils for DynamoDb {@link AttributeValue}. This class is 
currently not
+ * serializable, see <a 
href="https://github.com/aws/aws-sdk-java-v2/issues/3143";>open issue</a>
+ */
+@Internal
+public class DynamoDbSerializationUtil {
+
+    public static void serializeWriteRequest(WriteRequest writeRequest, 
DataOutputStream out)
+            throws IOException {
+        if (writeRequest.putRequest() != null) {
+            out.writeUTF(DynamoDbWriteRequestType.PUT.name());
+            Map<String, AttributeValue> item = 
writeRequest.putRequest().item();
+            serializeItem(item, out);
+        } else if (writeRequest.deleteRequest() != null) {
+            out.writeUTF(DynamoDbWriteRequestType.DELETE.name());
+            Map<String, AttributeValue> key = 
writeRequest.deleteRequest().key();
+            serializeItem(key, out);
+        } else {
+            throw new IllegalArgumentException("Empty write request");
+        }
+    }
+
+    public static WriteRequest deserializeWriteRequest(DataInputStream in) 
throws IOException {
+        String writeRequestType = in.readUTF();
+        DynamoDbWriteRequestType dynamoDbWriteRequestType =
+                DynamoDbWriteRequestType.valueOf(writeRequestType);
+        switch (dynamoDbWriteRequestType) {
+            case PUT:
+                return WriteRequest.builder()
+                        
.putRequest(PutRequest.builder().item(deserializeItem(in)).build())
+                        .build();
+            case DELETE:
+                return WriteRequest.builder()
+                        
.deleteRequest(DeleteRequest.builder().key(deserializeItem(in)).build())
+                        .build();
+            default:
+                throw new IllegalArgumentException(
+                        "Invalid write request type " + writeRequestType);
+        }
+    }
+
+    public static void serializeItem(Map<String, AttributeValue> item, 
DataOutputStream out)
+            throws IOException {
+        out.writeInt(item.size());
+        for (Map.Entry<String, AttributeValue> entry : item.entrySet()) {
+            out.writeUTF(entry.getKey());
+            AttributeValue value = entry.getValue();
+            serializeAttributeValue(value, out);
+        }
+    }
+
+    public static void serializeAttributeValue(AttributeValue value, 
DataOutputStream out)
+            throws IOException {
+        if (value.nul() != null) {
+            out.writeUTF(DynamoDbType.NULL.name());

Review Comment:
   We can reduce state size here by using an enum ordinal or similar shortened 
key



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkBuilder.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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
+ *       by the batch 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;
+    private static final int DEFAULT_MAX_BUFFERED_REQUESTS = 10000;
+    private static final long DEFAULT_MAX_BATCH_SIZE_IN_B = 16 * 1000 * 1000;
+    private static final long DEFAULT_MAX_TIME_IN_BUFFER_MS = 5000;
+    private static final long DEFAULT_MAX_RECORD_SIZE_IN_B = 400 * 1000;
+    private static final boolean DEFAULT_FAIL_ON_ERROR = false;
+
+    private boolean failOnError;
+    private Properties dynamodbClientProperties;
+
+    private ElementConverter<InputT, DynamoDbWriteRequest> elementConverter;
+    private String tableName;
+
+    private List<String> overwriteByPKeys;
+
+    public DynamoDbSinkBuilder<InputT> setDynamoDbProperties(Properties 
properties) {
+        this.dynamodbClientProperties = properties;
+        return this;
+    }
+
+    public DynamoDbSinkBuilder<InputT> setElementConverter(
+            ElementConverter<InputT, DynamoDbWriteRequest> elementConverter) {
+        this.elementConverter = elementConverter;
+        return this;
+    }
+
+    /** Destination DynamoDB table name for the sink to write to. */
+    public DynamoDbSinkBuilder<InputT> setDestinationTableName(String 
tableName) {
+        this.tableName = tableName;
+        return this;
+    }
+
+    /**
+     * @param overwriteByPKeys provide partition key and (optionally) sort key 
name if you want to
+     *     bypass no duplication limitation of single batch write request. 
Batching DynamoDB sink
+     *     will drop request items in the buffer if their primary 
keys(composite) values are the
+     *     same as newly added one.

Review Comment:
   > and (optionally) sort key name 
   
   It is not clear from this comment or the method signature how I provide a 
sort key. If it is element 0 and 1, could we split to individual fields instead?
   



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkWriter.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.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.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 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.numRecordsOutErrorsCounter = 
metrics.getNumRecordsOutErrorsCounter();
+        this.client = AWSDynamoDbUtil.createClient(dynamoDbClientProperties);
+    }
+
+    @Override
+    protected void submitRequestEntries(
+            List<DynamoDbWriteRequest> requestEntries,
+            Consumer<List<DynamoDbWriteRequest>> requestResultConsumer) {
+
+        TableRequestsContainer container = new 
TableRequestsContainer(overwriteByPKeys);
+        requestEntries.forEach(container::put);
+
+        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 (response.unprocessedItems() != null
+                            && !response.unprocessedItems().isEmpty()) {

Review Comment:
   nit: You can use `CollectionUtil::isNullOrEmpty`



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/util/PrimaryKeyBuilder.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.annotation.Internal;
+import 
org.apache.flink.streaming.connectors.dynamodb.sink.InvalidRequestException;
+import org.apache.flink.util.CollectionUtil;
+
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.WriteRequest;
+
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+/** Helper to construct primary(composite) key for a DynamoDB request. */
+@Internal
+public class PrimaryKeyBuilder {
+
+    public static String build(List<String> pKeys, WriteRequest request) {
+
+        if (CollectionUtil.isNullOrEmpty(pKeys)) {
+            // fake key, because no dynamodb partition key or sort key 
provided. Using UUID should be safe
+            // here, as we have at most 25 elements per batch
+            return UUID.randomUUID().toString();
+        } else {
+            Map<String, AttributeValue> requestItems = 
getRequestItems(request);
+
+            StringBuilder builder = new StringBuilder();
+            for (String keyName : pKeys) {
+                AttributeValue value = requestItems.get(keyName);
+
+                if (value == null) {
+                    throw new InvalidRequestException(
+                            "Request " + request.toString() + " does not 
contain pKey " + keyName);
+                }
+
+                builder.append(getKeyValue(value));
+            }
+
+            return builder.toString();
+        }
+    }
+
+    /**
+     * Returns string value of a partition key attribute. Each primary key 
attribute must be defined
+     * as type String, Number, or binary as per DynamoDB specification.
+     */
+    private static String getKeyValue(AttributeValue value) {
+        StringBuilder builder = new StringBuilder();
+
+        if (value.n() != null) {
+            builder.append(value.n());
+        }
+
+        if (value.s() != null) {
+            builder.append(value.s());
+        }
+
+        if (value.b() != null) {
+            builder.append(value.b().asUtf8String());
+        }
+
+        return builder.toString();

Review Comment:
   Do we want to validate that no unsupported types were used, or is this done 
elsewhere?



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/util/PrimaryKeyBuilder.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.annotation.Internal;
+import 
org.apache.flink.streaming.connectors.dynamodb.sink.InvalidRequestException;
+import org.apache.flink.util.CollectionUtil;
+
+import software.amazon.awssdk.services.dynamodb.model.AttributeValue;
+import software.amazon.awssdk.services.dynamodb.model.WriteRequest;
+
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+/** Helper to construct primary(composite) key for a DynamoDB request. */
+@Internal
+public class PrimaryKeyBuilder {
+
+    public static String build(List<String> pKeys, WriteRequest request) {
+
+        if (CollectionUtil.isNullOrEmpty(pKeys)) {
+            // fake key, because no dynamodb partition key or sort key 
provided. Using UUID should be safe
+            // here, as we have at most 25 elements per batch
+            return UUID.randomUUID().toString();
+        } else {
+            Map<String, AttributeValue> requestItems = 
getRequestItems(request);
+
+            StringBuilder builder = new StringBuilder();
+            for (String keyName : pKeys) {
+                AttributeValue value = requestItems.get(keyName);
+
+                if (value == null) {
+                    throw new InvalidRequestException(
+                            "Request " + request.toString() + " does not 
contain pKey " + keyName);
+                }
+
+                builder.append(getKeyValue(value));

Review Comment:
   We should add a separator. Consider `["ab", "cd"]`  and `["a", "bcd"]` would 
both result in the same composite key



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/util/DynamoDbSerializationUtil.java:
##########
@@ -0,0 +1,214 @@
+/*
+ * 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.annotation.Internal;
+
+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.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Serialization Utils for DynamoDb {@link AttributeValue}. This class is 
currently not
+ * serializable, see <a 
href="https://github.com/aws/aws-sdk-java-v2/issues/3143";>open issue</a>
+ */
+@Internal
+public class DynamoDbSerializationUtil {
+
+    public static void serializeWriteRequest(WriteRequest writeRequest, 
DataOutputStream out)
+            throws IOException {
+        if (writeRequest.putRequest() != null) {
+            out.writeUTF(DynamoDbWriteRequestType.PUT.name());
+            Map<String, AttributeValue> item = 
writeRequest.putRequest().item();
+            serializeItem(item, out);
+        } else if (writeRequest.deleteRequest() != null) {
+            out.writeUTF(DynamoDbWriteRequestType.DELETE.name());
+            Map<String, AttributeValue> key = 
writeRequest.deleteRequest().key();
+            serializeItem(key, out);
+        } else {
+            throw new IllegalArgumentException("Empty write request");
+        }
+    }
+
+    public static WriteRequest deserializeWriteRequest(DataInputStream in) 
throws IOException {
+        String writeRequestType = in.readUTF();
+        DynamoDbWriteRequestType dynamoDbWriteRequestType =
+                DynamoDbWriteRequestType.valueOf(writeRequestType);
+        switch (dynamoDbWriteRequestType) {
+            case PUT:
+                return WriteRequest.builder()
+                        
.putRequest(PutRequest.builder().item(deserializeItem(in)).build())
+                        .build();
+            case DELETE:
+                return WriteRequest.builder()
+                        
.deleteRequest(DeleteRequest.builder().key(deserializeItem(in)).build())
+                        .build();
+            default:
+                throw new IllegalArgumentException(
+                        "Invalid write request type " + writeRequestType);
+        }
+    }
+
+    public static void serializeItem(Map<String, AttributeValue> item, 
DataOutputStream out)

Review Comment:
   Can the rest of the methods below here be private?



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkWriter.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.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.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 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.numRecordsOutErrorsCounter = 
metrics.getNumRecordsOutErrorsCounter();
+        this.client = AWSDynamoDbUtil.createClient(dynamoDbClientProperties);

Review Comment:
   There is a bunch of duplication in the `AWSDynamoDbUtil` with base classes. 
Consider doing this, and deleting `AWSDynamoDbUtil`:
   
   ```
   AWSAsyncSinkUtil.createAwsAsyncClient(
                   dynamoDbClientProperties,
                   AWSGeneralUtil.createAsyncHttpClient(properties),
                   DynamoDbAsyncClient.builder(),
                   USER_AGENT,
                   USER_AGENT_KEY);
   ```



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