dannycranmer commented on a change in pull request #18013:
URL: https://github.com/apache/flink/pull/18013#discussion_r769634451



##########
File path: flink-connectors/flink-connector-aws-kinesis-data-streams/pom.xml
##########
@@ -0,0 +1,130 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0";
+                xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+                xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/maven-v4_0_0.xsd";>
+
+       <modelVersion>4.0.0</modelVersion>
+
+       <parent>
+               <groupId>org.apache.flink</groupId>
+               <artifactId>flink-connectors</artifactId>
+               <version>1.15-SNAPSHOT</version>
+               <relativePath>..</relativePath>
+       </parent>
+
+       <artifactId>flink-connector-aws-kinesis-data-streams</artifactId>
+       <name>Flink : Connectors : AWS Kinesis Data Streams</name>
+       <properties>
+               <aws.sdk.version>2.17.52</aws.sdk.version>
+               <aws.kinesis-kpl.version>0.14.1</aws.kinesis-kpl.version>

Review comment:
       This is unused, we do not need KPL right? Please remove

##########
File path: 
flink-connectors/flink-connector-aws-base/src/main/java/org/apache/flink/connector/aws/util/AWSGeneralUtil.java
##########
@@ -209,6 +209,46 @@ static AwsCredentialsProvider 
getWebIdentityTokenFileCredentialsProvider(
         return webIdentityBuilder.build();
     }
 
+    public static SdkAsyncHttpClient createAsyncHttpClient(final Properties 
configProperties) {
+        final AttributeMap.Builder clientConfiguration =
+                
AttributeMap.builder().put(SdkHttpConfigurationOption.TCP_KEEPALIVE, true);
+
+        Optional.ofNullable(
+                        configProperties.getProperty(
+                                
AWSConfigConstants.HTTP_CLIENT_MAX_CONCURRENCY))
+                .map(Integer::parseInt)
+                .ifPresent(
+                        integer ->
+                                clientConfiguration.put(
+                                        
SdkHttpConfigurationOption.MAX_CONNECTIONS, integer));
+
+        Optional.ofNullable(
+                        configProperties.getProperty(
+                                
AWSConfigConstants.HTTP_CLIENT_READ_TIMEOUT_MILLIS))
+                .map(Integer::parseInt)
+                .map(Duration::ofMillis)
+                .ifPresent(
+                        timeout ->
+                                clientConfiguration.put(
+                                        
SdkHttpConfigurationOption.READ_TIMEOUT, timeout));
+
+        
Optional.ofNullable(configProperties.getProperty(AWSConfigConstants.TRUST_ALL_CERTIFICATES))
+                .map(Boolean::parseBoolean)
+                .ifPresent(
+                        bool ->
+                                clientConfiguration.put(
+                                        
SdkHttpConfigurationOption.TRUST_ALL_CERTIFICATES, bool));
+
+        
Optional.ofNullable(configProperties.getProperty(AWSConfigConstants.HTTP_PROTOCOL_VERSION))
+                .map(Protocol::valueOf)
+                .ifPresent(
+                        protocol ->
+                                clientConfiguration.put(
+                                        SdkHttpConfigurationOption.PROTOCOL, 
protocol));
+        return createAsyncHttpClient(
+                clientConfiguration.build(), 
NettyNioAsyncHttpClient.builder());
+    }
+

Review comment:
       Does not look like you added/moved the unit tests for this code?

##########
File path: 
flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/connector/kinesis/sink/KinesisDataStreamsSink.java
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.connector.kinesis.sink;
+
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.connector.sink.SinkWriter;
+import org.apache.flink.connector.base.sink.AsyncSinkBase;
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.util.Preconditions;
+
+import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.Properties;
+
+/**
+ * A Kinesis Data Streams (KDS) 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.kinesis.KinesisAsyncClient}
+ * 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 sent to KDS
+ *   <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 flushOnBufferSizeInBytes}: if the total size in bytes of all 
elements in the buffer
+ *       reaches this value, then a flush will occur the next time any 
elements are added to the
+ *       buffer
+ *   <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 failOnError}: when an exception is encountered while 
persisting to Kinesis Data
+ *       Streams, the job will fail immediately if failOnError is set

Review comment:
       This comment is outdated, please update

##########
File path: 
flink-connectors/flink-connector-aws-kinesis-data-streams/src/test/java/org/apache/flink/connectors/kinesis/testutils/KinesaliteContainer.java
##########
@@ -45,33 +52,28 @@
 public class KinesaliteContainer extends GenericContainer<KinesaliteContainer> 
{
     private static final String ACCESS_KEY = "access key";
     private static final String SECRET_KEY = "secret key";
+    private static final int PORT = 4567;
+    private static final Region REGION = Region.US_EAST_1;
+    private static final String URL_FORMAT = "https://%s:%s";;
 
     public KinesaliteContainer(DockerImageName imageName) {
         super(imageName);
 
-        withEnv(ACCESS_KEY_ENV_VAR, ACCESS_KEY);
-        withEnv(SECRET_KEY_ENV_VAR, ACCESS_KEY);
-        withExposedPorts(4567);
+        System.setProperty(SdkSystemSetting.CBOR_ENABLED.property(), "false");

Review comment:
       Same as above, can this be set on client instead?

##########
File path: 
flink-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java
##########
@@ -22,9 +22,9 @@
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.MemorySize;
 import org.apache.flink.configuration.TaskManagerOptions;
+import org.apache.flink.connector.aws.config.AWSConfigConstants;
 import org.apache.flink.runtime.testutils.MiniClusterResource;
 import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
-import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants;
 import 
org.apache.flink.streaming.connectors.kinesis.testutils.ExactlyOnceValidatingConsumerThread;
 import 
org.apache.flink.streaming.connectors.kinesis.testutils.KinesisEventsGeneratorProducerThread;
 import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil;

Review comment:
       I assume these class swaps are not actually required and you just did it 
for completeness (seems this way reading the code)? If they are required, this 
is not backwards compatible, please confirm

##########
File path: 
flink-connectors/flink-connector-aws-kinesis-data-streams/src/test/java/org/apache/flink/connector/kinesis/sink/examples/SinkIntoKinesis.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.connector.kinesis.sink.examples;
+
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+import org.apache.flink.connector.kinesis.sink.KinesisDataStreamsSink;
+import 
org.apache.flink.connector.kinesis.sink.KinesisDataStreamsSinkElementConverter;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry;
+
+/**
+ * An example application demonstrating how to use the {@link 
KinesisDataStreamsSink} to sink into
+ * KDS.
+ *
+ * <p>The {@link KinesisAsyncClient} 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.
+ */
+public class SinkIntoKinesis {
+
+    private static final String JSON_PAYLOAD_TEMPLATE = "{\"data\": \"%s\"}";
+
+    private static final ElementConverter<String, PutRecordsRequestEntry> 
elementConverter =
+            KinesisDataStreamsSinkElementConverter.<String>builder()
+                    .setSerializationSchema(new SimpleStringSchema())
+                    .setPartitionKeyGenerator(element -> 
String.valueOf(element.hashCode()))
+                    .build();
+
+    public static void main(String[] args) throws Exception {
+        final StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+        env.enableCheckpointing(10_000);
+
+        DataStream<String> fromGen =
+                env.fromSequence(1, 10_000_000L)
+                        .map(Object::toString)
+                        .returns(String.class)
+                        .map(data -> String.format(JSON_PAYLOAD_TEMPLATE, 
data));

Review comment:
       nit: This serialisation is error prone. How about serialising a Map 
instead?

##########
File path: flink-connectors/flink-connector-aws-kinesis-data-streams/pom.xml
##########
@@ -0,0 +1,130 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0";
+                xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
+                xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/maven-v4_0_0.xsd";>
+
+       <modelVersion>4.0.0</modelVersion>
+
+       <parent>
+               <groupId>org.apache.flink</groupId>
+               <artifactId>flink-connectors</artifactId>
+               <version>1.15-SNAPSHOT</version>
+               <relativePath>..</relativePath>
+       </parent>
+
+       <artifactId>flink-connector-aws-kinesis-data-streams</artifactId>
+       <name>Flink : Connectors : AWS Kinesis Data Streams</name>
+       <properties>
+               <aws.sdk.version>2.17.52</aws.sdk.version>
+               <aws.kinesis-kpl.version>0.14.1</aws.kinesis-kpl.version>
+       </properties>
+
+       <packaging>jar</packaging>
+
+       <dependencies>
+               <dependency>
+                       <groupId>org.apache.flink</groupId>
+                       <artifactId>flink-streaming-java</artifactId>
+                       <version>${project.version}</version>
+                       <scope>provided</scope>
+               </dependency>
+
+               <dependency>
+                       <groupId>org.apache.flink</groupId>
+                       <artifactId>flink-connector-base</artifactId>
+                       <version>${project.version}</version>
+               </dependency>
+
+               <dependency>
+                       <groupId>org.apache.flink</groupId>
+                       <artifactId>flink-connector-aws-base</artifactId>
+                       <version>${project.version}</version>
+               </dependency>
+
+               <dependency>
+                       <groupId>software.amazon.awssdk</groupId>
+                       <artifactId>sdk-core</artifactId>
+                       <version>${aws.sdk.version}</version>
+               </dependency>
+
+               <dependency>
+                       <groupId>software.amazon.awssdk</groupId>
+                       <artifactId>kinesis</artifactId>
+                       <version>${aws.sdk.version}</version>
+               </dependency>
+
+               <dependency>
+                       <groupId>software.amazon.awssdk</groupId>
+                       <artifactId>netty-nio-client</artifactId>
+                       <version>${aws.sdk.version}</version>
+               </dependency>
+
+               <dependency>
+                       <groupId>software.amazon.awssdk</groupId>
+                       <artifactId>sts</artifactId>
+                       <version>${aws.sdk.version}</version>
+               </dependency>
+
+               <!-- Test dependencies -->
+               <dependency>
+                       <groupId>org.apache.flink</groupId>
+                       <artifactId>flink-test-utils</artifactId>
+                       <version>${project.version}</version>
+                       <scope>test</scope>
+               </dependency>
+
+               <dependency>
+                       <groupId>org.apache.flink</groupId>
+                       <artifactId>flink-connector-aws-base</artifactId>
+                       <version>${project.version}</version>
+                       <type>test-jar</type>
+                       <scope>test</scope>
+               </dependency>
+
+               <dependency>
+                       <groupId>org.testcontainers</groupId>
+                       <artifactId>testcontainers</artifactId>
+                       <scope>test</scope>
+               </dependency>
+       </dependencies>
+
+       <build>
+               <plugins>
+                       <plugin>
+                               <groupId>org.apache.maven.plugins</groupId>
+                               <artifactId>maven-jar-plugin</artifactId>
+                               <executions>
+                                       <execution>
+                                               <goals>
+                                                       <goal>test-jar</goal>
+                                               </goals>
+                                       </execution>
+                               </executions>
+                       </plugin>
+
+                       <plugin>
+                               <groupId>org.apache.maven.plugins</groupId>
+                               <artifactId>maven-surefire-plugin</artifactId>
+                               <version>2.22.1</version>
+                       </plugin>

Review comment:
       nit: Is this required? Why are we setting version here? It should 
probably be inherited? 

##########
File path: 
flink-connectors/flink-connector-aws-kinesis-data-streams/src/test/java/org/apache/flink/connector/kinesis/sink/KinesisDataStreamsSinkITCase.java
##########
@@ -0,0 +1,357 @@
+/*
+ * 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.connector.kinesis.sink;
+
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+import org.apache.flink.api.common.time.Deadline;
+import org.apache.flink.connector.base.sink.writer.ElementConverter;
+import org.apache.flink.connectors.kinesis.testutils.KinesaliteContainer;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import 
org.apache.flink.streaming.api.functions.source.datagen.DataGeneratorSource;
+import org.apache.flink.streaming.api.functions.source.datagen.RandomGenerator;
+import org.apache.flink.util.DockerImageVersions;
+import org.apache.flink.util.TestLogger;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.rnorth.ducttape.ratelimits.RateLimiter;
+import org.rnorth.ducttape.ratelimits.RateLimiterBuilder;
+import org.testcontainers.containers.Network;
+import org.testcontainers.utility.DockerImageName;
+import software.amazon.awssdk.core.SdkSystemSetting;
+import software.amazon.awssdk.services.kinesis.KinesisAsyncClient;
+import software.amazon.awssdk.services.kinesis.model.CreateStreamRequest;
+import software.amazon.awssdk.services.kinesis.model.DescribeStreamRequest;
+import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest;
+import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest;
+import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry;
+import software.amazon.awssdk.services.kinesis.model.ShardIteratorType;
+import software.amazon.awssdk.services.kinesis.model.StreamStatus;
+
+import java.time.Duration;
+import java.util.Properties;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static 
org.apache.flink.connector.aws.config.AWSConfigConstants.AWS_ACCESS_KEY_ID;
+import static 
org.apache.flink.connector.aws.config.AWSConfigConstants.AWS_ENDPOINT;
+import static 
org.apache.flink.connector.aws.config.AWSConfigConstants.AWS_REGION;
+import static 
org.apache.flink.connector.aws.config.AWSConfigConstants.AWS_SECRET_ACCESS_KEY;
+import static 
org.apache.flink.connector.aws.config.AWSConfigConstants.HTTP_PROTOCOL_VERSION;
+import static 
org.apache.flink.connector.aws.config.AWSConfigConstants.TRUST_ALL_CERTIFICATES;
+
+/** IT cases for using Kinesis Data Streams Sink based on Kinesalite. */
+public class KinesisDataStreamsSinkITCase extends TestLogger {
+
+    private static final String DEFAULT_FIRST_SHARD_NAME = 
"shardId-000000000000";
+
+    private final ElementConverter<String, PutRecordsRequestEntry> 
elementConverter =
+            KinesisDataStreamsSinkElementConverter.<String>builder()
+                    .setSerializationSchema(new SimpleStringSchema())
+                    .setPartitionKeyGenerator(element -> 
String.valueOf(element.hashCode()))
+                    .build();
+
+    private final ElementConverter<String, PutRecordsRequestEntry>
+            partitionKeyTooLongElementConverter =
+                    KinesisDataStreamsSinkElementConverter.<String>builder()
+                            .setSerializationSchema(new SimpleStringSchema())
+                            .setPartitionKeyGenerator(element -> element)
+                            .build();
+
+    @ClassRule
+    public static final KinesaliteContainer KINESALITE =
+            new 
KinesaliteContainer(DockerImageName.parse(DockerImageVersions.KINESALITE))
+                    .withNetwork(Network.newNetwork())
+                    .withNetworkAliases("kinesalite");
+
+    private StreamExecutionEnvironment env;
+    private KinesisAsyncClient kinesisClient;
+
+    @Before
+    public void setUp() throws Exception {
+        System.setProperty(SdkSystemSetting.CBOR_ENABLED.property(), "false");

Review comment:
       Is there a way to set this on the client instead of a System property? 
System properties could interfere with other tests, and I see you are not 
"unsetting" it at the end of the test




-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to