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


##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/config/AWSDynamoDbConfigConstants.java:
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.streaming.connectors.dynamodb.util.AWSDynamoDbUtil;
+
+import software.amazon.awssdk.core.retry.backoff.EqualJitterBackoffStrategy;
+import software.amazon.awssdk.core.retry.backoff.FixedDelayBackoffStrategy;
+import software.amazon.awssdk.core.retry.backoff.FullJitterBackoffStrategy;
+
+/** Defaults for {@link AWSDynamoDbUtil}. */
+@PublicEvolving
+public class AWSDynamoDbConfigConstants {
+
+    public static final String BASE_DYNAMODB_USER_AGENT_PREFIX_FORMAT =
+            "Apache Flink %s (%s) DynamoDB Connector";
+
+    /** Identifier for user agent prefix. */
+    public static final String DYNAMODB_CLIENT_USER_AGENT_PREFIX =
+            "aws.dynamodb.client.user-agent-prefix";
+
+    public static final String AWS_DYNAMODB_CLIENT_RETRY_PREFIX = 
"aws.dynamodb.client.retry";
+
+    /**
+     * Possible configuration values for backoff strategy on retry when 
writing to DynamoDB.
+     * Internally, a corresponding implementation of {@link
+     * software.amazon.awssdk.core.retry.backoff.BackoffStrategy} will be used.
+     */
+    @PublicEvolving
+    public enum BackoffStrategy {
+
+        /**
+         * Backoff strategy that uses a full jitter strategy for computing the 
next backoff delay. A
+         * full jitter strategy will always compute a new random delay between 
and the computed
+         * exponential backoff for each subsequent request. See example: {@link
+         * FullJitterBackoffStrategy}
+         */
+        FULL_JITTER,
+
+        /**
+         * Backoff strategy that uses equal jitter for computing the delay 
before the next retry. An
+         * equal jitter backoff strategy will first compute an exponential 
delay based on the
+         * current number of retries, base delay and max delay. The final 
computed delay before the
+         * next retry will keep half of this computed delay plus a random 
delay computed as a random
+         * number between 0 and half of the exponential delay plus one. See 
example: {@link
+         * EqualJitterBackoffStrategy}
+         */
+        EQUAL_JITTER,
+
+        /**
+         * Simple backoff strategy that always uses a fixed delay for the 
delay * before the next
+         * retry attempt. See example: {@link FixedDelayBackoffStrategy}
+         */
+        FIXED_DELAY
+    }

Review Comment:
   As a follow-up, we should generalise and pull this config into `aws-base`. 
Just a callout, nothing to do now



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSinkBuilder.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.dynamodb.sink;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.sink.AsyncSinkBaseBuilder;
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+import 
org.apache.flink.streaming.connectors.dynamodb.config.DynamoDbTablesConfig;
+
+import java.util.Optional;
+import java.util.Properties;
+
+/**
+ * Builder to construct {@link DynamoDbSink}.
+ *
+ * <p>The following example shows the minimum setup to create a {@link 
DynamoDbSink} that writes
+ * records into DynamoDb
+ *
+ * <pre>{@code
+ * private static class DummyDynamoDbRequestConverter implements 
DynamoDbRequestConverter<String> {
+ *
+ *         @Override
+ *         public DynamoDbRequest apply(String s) {
+ *             final Map<String, DynamoDbAttributeValue> item = new 
HashMap<>();
+ *             item.put("your-key", 
DynamoDbAttributeValue.builder().s(s).build());
+ *             return DynamoDbRequest.builder()
+ *                     .tableName("your-table-name")
+ *                     
.putRequest(DynamoDbPutRequest.builder().item(item).build())
+ *                     .build();
+ *         }
+ *     }
+ * DynamoDbSink<String> dynamoDbSink = DynamoDbSink.<String>builder()
+ *                                          .setDynamoDbRequestConverter(new 
DummyDynamoDbRequestConverter())
+ *                                       .build();
+ * }</pre>
+ *
+ * <p>If the following parameters are not set in this builder, the following 
defaults will be used:
+ *
+ * <ul>
+ *   <li>{@code maxBatchSize} will be 25
+ *   <li>{@code maxInFlightRequests} will be 50
+ *   <li>{@code maxBufferedRequests} will be 10000
+ *   <li>{@code maxBatchSizeInBytes} will be 16 MB i.e. {@code 16 * 1000 * 
1000}
+ *   <li>{@code maxTimeInBufferMS} will be 5000ms
+ *   <li>{@code maxRecordSizeInBytes} will be 400 KB i.e. {@code 400 * 1000 * 
1000}
+ *   <li>{@code failOnError} will be false
+ *   <li>{@code dynamoDbTablesConfig} will be empty meaning no records 
deduplication will be
+ *       performed by the sink
+ * </ul>
+ *
+ * @param <InputT> type of elements that should be persisted in the destination
+ */
+@PublicEvolving
+public class DynamoDbSinkBuilder<InputT>
+        extends AsyncSinkBaseBuilder<InputT, DynamoDbWriteRequest, 
DynamoDbSinkBuilder<InputT>> {
+
+    private static final int DEFAULT_MAX_BATCH_SIZE = 25;
+    private static final int DEFAULT_MAX_IN_FLIGHT_REQUESTS = 50;
+    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 DynamoDbTablesConfig dynamoDbTablesConfig;
+    private Properties dynamodbClientProperties;
+
+    private ElementConverter<InputT, DynamoDbWriteRequest> elementConverter;
+
+    public DynamoDbSinkBuilder<InputT> setDynamoDbProperties(Properties 
properties) {
+        this.dynamodbClientProperties = properties;
+        return this;
+    }
+
+    /**
+     * @param elementConverter the {@link ElementConverter} to be used for the 
sink
+     * @return {@link DynamoDbSinkBuilder} itself
+     */

Review Comment:
   nit: Since this comment does not add anything above the method signature, 
consider removing, as per the [coding 
guidelines](https://flink.apache.org/contributing/code-style-and-quality-common.html#comments)



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSink.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.dynamodb.sink;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.sink.AsyncSinkBase;
+import org.apache.flink.connector.base.sink.writer.BufferedRequestState;
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import 
org.apache.flink.streaming.connectors.dynamodb.config.DynamoDbTablesConfig;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A DynamoDB Sink that performs async requests against a destination stream 
using the buffering

Review Comment:
   ```suggestion
    * A DynamoDB Sink that performs async requests against a destination table 
using the buffering
   ```



##########
flink-connector-dynamodb/src/main/java/org/apache/flink/streaming/connectors/dynamodb/sink/DynamoDbSink.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.dynamodb.sink;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.sink.AsyncSinkBase;
+import org.apache.flink.connector.base.sink.writer.BufferedRequestState;
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import 
org.apache.flink.streaming.connectors.dynamodb.config.DynamoDbTablesConfig;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Properties;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A DynamoDB Sink that performs async requests against a destination stream 
using the buffering
+ * protocol specified in {@link AsyncSinkBase}.
+ *
+ * <p>The sink internally uses a {@link
+ * software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient} to 
communicate with the AWS
+ * endpoint.
+ *
+ * <p>The behaviour of the buffering may be specified by providing 
configuration during the sink
+ * build time.
+ *
+ * <ul>
+ *   <li>{@code maxBatchSize}: the maximum size of a batch of entries that may 
be written to
+ *       DynamoDb.
+ *   <li>{@code maxInFlightRequests}: the maximum number of in flight requests 
that may exist, if
+ *       any more in flight requests need to be initiated once the maximum has 
been reached, then it
+ *       will be blocked until some have completed
+ *   <li>{@code maxBufferedRequests}: the maximum number of elements held in 
the buffer, requests to
+ *       add elements will be blocked while the number of elements in the 
buffer is at the maximum
+ *   <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 DynamoDbTablesConfig dynamoDbTablesConfig;
+    private final boolean failOnError;
+
+    protected DynamoDbSink(
+            ElementConverter<InputT, DynamoDbWriteRequest> elementConverter,
+            int maxBatchSize,
+            int maxInFlightRequests,
+            int maxBufferedRequests,
+            long maxBatchSizeInBytes,
+            long maxTimeInBufferMS,
+            long maxRecordSizeInBytes,
+            boolean failOnError,
+            DynamoDbTablesConfig dynamoDbTablesConfig,
+            Properties dynamoDbClientProperties) {
+        super(
+                elementConverter,
+                maxBatchSize,
+                maxInFlightRequests,
+                maxBufferedRequests,
+                maxBatchSizeInBytes,
+                maxTimeInBufferMS,
+                maxRecordSizeInBytes);
+        checkNotNull(dynamoDbTablesConfig, "Missing dynamoDbTablesConfig");
+        checkNotNull(dynamoDbClientProperties, "Missing 
dynamoDbClientProperties");
+        this.failOnError = failOnError;
+        this.dynamoDbTablesConfig = dynamoDbTablesConfig;
+        this.dynamoDbClientProperties = dynamoDbClientProperties;
+    }
+
+    /**
+     * Create a {@link DynamoDbSinkBuilder} to construct a new {@link 
DynamoDbSink}.
+     *
+     * @param <InputT> type of incoming records
+     * @return {@link DynamoDbSinkBuilder}
+     */
+    public static <InputT> DynamoDbSinkBuilder<InputT> builder() {
+        return new DynamoDbSinkBuilder<>();
+    }
+
+    @Internal
+    @Override
+    public StatefulSinkWriter<InputT, 
BufferedRequestState<DynamoDbWriteRequest>> createWriter(
+            InitContext context) throws IOException {
+        return new DynamoDbSinkWriter<>(

Review Comment:
   nit: Can simplify with:
   ```
   return restoreWriter(context, Collections.emptyList());
   ```



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