dannycranmer commented on a change in pull request #17345: URL: https://github.com/apache/flink/pull/17345#discussion_r749336832
########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSGeneralUtil.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.kinesis.util; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants.CredentialProvider; + +import java.util.Properties; + +/** Some utilities specific to Amazon Web Service. */ +@Internal +public class AWSGeneralUtil { Review comment: The name of this class seems too wide, can it be narrowed to `AWSKinesisDataStreamsUtil` ? ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/pom.xml ########## @@ -0,0 +1,134 @@ +<?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> Review comment: Looks like this module has 2 package formats: - `org/apache/flink/streaming/connectors/kinesis` - `org/apache/flink/connector/kinesis` Is this for backwards compatibility? ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java ########## @@ -18,23 +18,21 @@ package org.apache.flink.streaming.connectors.kinesis.config; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; -import org.apache.flink.streaming.connectors.kinesis.internals.ShardConsumer; import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber; -import com.amazonaws.services.kinesis.model.ShardIteratorType; +import software.amazon.awssdk.http.Protocol; import java.time.Duration; /** - * Optional consumer specific configuration keys and default values for {@link + * Optional consumer specific configuration keys and default values for {@code * FlinkKinesisConsumer}. */ @PublicEvolving public class ConsumerConfigConstants extends AWSConfigConstants { Review comment: Why did you move this class? This is for the `Consumer`, once this is moved it is difficult to remove it in a backwards compatible way, so we need to make sure it is 100% needed ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/connector/kinesis/sink/KinesisDataStreamsSinkWriter.java ########## @@ -0,0 +1,204 @@ +/* + * 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.connector.sink.Sink; +import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter; +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.kinesis.util.AwsV2Util; + +import com.amazonaws.ClientConfiguration; +import com.amazonaws.ClientConfigurationFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.http.async.SdkAsyncHttpClient; +import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.PutRecordsRequest; +import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry; +import software.amazon.awssdk.services.kinesis.model.PutRecordsResponse; +import software.amazon.awssdk.services.kinesis.model.PutRecordsResultEntry; +import software.amazon.awssdk.services.kinesis.model.ResourceNotFoundException; + +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.concurrent.CompletionException; +import java.util.function.Consumer; + +/** + * Sink writer created by {@link KinesisDataStreamsSink} to write to Kinesis Data Streams. More + * details on the operation of this sink writer may be found in the doc for {@link + * KinesisDataStreamsSink}. More details on the internals of this sink writer may be found in {@link + * AsyncSinkWriter}. + * + * <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. + */ +class KinesisDataStreamsSinkWriter<InputT> extends AsyncSinkWriter<InputT, PutRecordsRequestEntry> { + + /* A counter for the total number of records that have encountered an error during put */ + private final Counter numRecordsOutErrorsCounter; + + /* Name of the stream in Kinesis Data Streams */ + private final String streamName; + + /* The sink writer metric group */ + private final SinkWriterMetricGroup metrics; + + /* The asynchronous Kinesis client - construction is by kinesisClientProperties */ + private final KinesisAsyncClient client; + + /* Flag to whether fatally fail any time we encounter an exception when persisting records */ + private final boolean failOnError; + + private static final Logger LOG = LoggerFactory.getLogger(KinesisDataStreamsSinkWriter.class); + + KinesisDataStreamsSinkWriter( + ElementConverter<InputT, PutRecordsRequestEntry> elementConverter, + Sink.InitContext context, + int maxBatchSize, + int maxInFlightRequests, + int maxBufferedRequests, + long flushOnBufferSizeInBytes, + long maxTimeInBufferMS, + boolean failOnError, + String streamName, + Properties kinesisClientProperties) { + super( + elementConverter, + context, + maxBatchSize, + maxInFlightRequests, + maxBufferedRequests, + flushOnBufferSizeInBytes, + maxTimeInBufferMS); + this.failOnError = failOnError; + this.streamName = streamName; + this.metrics = context.metricGroup(); + this.numRecordsOutErrorsCounter = metrics.getNumRecordsOutErrorsCounter(); + this.client = buildClient(kinesisClientProperties); + } + + private KinesisAsyncClient buildClient(Properties kinesisClientProperties) { + final ClientConfiguration clientConfiguration = + new ClientConfigurationFactory().getConfig(); + clientConfiguration.setUseTcpKeepAlive(true); Review comment: Where are you populating ClientConfig from the properties map? In the legacy source it is done using `AWSUtil.setAwsClientConfigProperties(awsClientConfig, configProps);` Am I right in thinking this block is the only reason to keep the dependency on AWS SDK v1? We should consider refactoring this out to drop the dependency ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/pom.xml ########## @@ -0,0 +1,134 @@ +<?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>1.12.7</aws.sdk.version> + <aws.sdkv2.version>2.17.52</aws.sdkv2.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> + + <!-- Amazon AWS SDK v1.x dependencies --> + <dependency> + <groupId>com.amazonaws</groupId> + <artifactId>aws-java-sdk-core</artifactId> + <version>${aws.sdk.version}</version> + </dependency> + + <dependency> + <groupId>com.amazonaws</groupId> + <artifactId>aws-java-sdk-kinesis</artifactId> + <version>${aws.sdk.version}</version> + </dependency> + Review comment: This is not required as compile scope. It is used only in the tests, which we could refactor to use SDK v2 and remove this dependency all together ########## 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() + .serializationSchema(new SimpleStringSchema()) + .partitionKeyGenerator(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)); + + KinesisDataStreamsSink<String> kdsSink = + KinesisDataStreamsSink.<String>builder() + .setElementConverter(elementConverter) Review comment: This still feels like we are exposing too much of the underlying implementation. Why do we need to surface the `ElementConverter` interface here? Can we simplify it to: ``` KinesisDataStreamsSink<String> kdsSink = KinesisDataStreamsSink.<String>builder() .setStreamName("your_stream_name") .setSerializationSchema(new SimpleStringSchema()) .setPartitionKeyGenerator(element -> String.valueOf(element.hashCode())) .build(); ``` ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSGeneralUtil.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.kinesis.util; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants.CredentialProvider; + +import java.util.Properties; + +/** Some utilities specific to Amazon Web Service. */ +@Internal +public class AWSGeneralUtil { + /** Used for formatting Flink-specific user agent string when creating Kinesis client. */ + private static final String USER_AGENT_FORMAT = "Apache Flink %s (%s) Kinesis Connector"; Review comment: Is there anyway to differentiate the legacy and new source/sink via user agent? ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java ########## @@ -18,23 +18,21 @@ package org.apache.flink.streaming.connectors.kinesis.config; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; -import org.apache.flink.streaming.connectors.kinesis.internals.ShardConsumer; import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber; -import com.amazonaws.services.kinesis.model.ShardIteratorType; +import software.amazon.awssdk.http.Protocol; import java.time.Duration; /** - * Optional consumer specific configuration keys and default values for {@link + * Optional consumer specific configuration keys and default values for {@code * FlinkKinesisConsumer}. */ @PublicEvolving public class ConsumerConfigConstants extends AWSConfigConstants { Review comment: Infact I just checked and this is not needed, same for: - `SentinelSequenceNumber` - `SequenceNumber` ########## File path: flink-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java ########## @@ -60,10 +59,7 @@ /** Some utilities specific to Amazon Web Service. */ @Internal -public class AWSUtil { - /** Used for formatting Flink-specific user agent string when creating Kinesis client. */ - private static final String USER_AGENT_FORMAT = "Apache Flink %s (%s) Kinesis Connector"; - +public class AWSUtil extends AWSGeneralUtil { Review comment: Is this here for backwards compatibility? Util classes extending util classes seems a bit smelly ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSGeneralUtil.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.kinesis.util; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.util.EnvironmentInformation; +import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants.CredentialProvider; + +import java.util.Properties; + +/** Some utilities specific to Amazon Web Service. */ +@Internal +public class AWSGeneralUtil { Review comment: File name inconsistent with `AwsV2Util` .. `AWS` vs `Aws` ########## 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() + .serializationSchema(new SimpleStringSchema()) + .partitionKeyGenerator(element -> String.valueOf(element.hashCode())) Review comment: Builder `setters` do not have verb `set` like they do for the sink builder. Inconsistent ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesaliteContainer.java ########## @@ -112,26 +120,65 @@ private AmazonKinesis getClient(String endPoint) { .build(); } + public KinesisAsyncClient getV2Client() throws URISyntaxException { + return KinesisAsyncClient.builder() + .endpointOverride(new URI(getHostEndpointUrl())) + .region(REGION) + .credentialsProvider( + () -> AwsBasicCredentials.create(getAccessKey(), getSecretKey())) + .httpClient(buildSdkAsyncHttpClient()) + .build(); + } + + private void tryStartContainer() { Review comment: nit: Why call it `try`? ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/streaming/connectors/kinesis/config/ConsumerConfigConstants.java ########## @@ -18,23 +18,21 @@ package org.apache.flink.streaming.connectors.kinesis.config; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.streaming.connectors.kinesis.FlinkKinesisConsumer; -import org.apache.flink.streaming.connectors.kinesis.internals.ShardConsumer; import org.apache.flink.streaming.connectors.kinesis.model.SentinelSequenceNumber; -import com.amazonaws.services.kinesis.model.ShardIteratorType; +import software.amazon.awssdk.http.Protocol; import java.time.Duration; /** - * Optional consumer specific configuration keys and default values for {@link + * Optional consumer specific configuration keys and default values for {@code * FlinkKinesisConsumer}. */ @PublicEvolving public class ConsumerConfigConstants extends AWSConfigConstants { Review comment: We should remove these methods from `AwsV2Util` back to the legacy module: - `isUsingEfoRecordPublisher` - `isEagerEfoRegistrationType` - `isLazyEfoRegistrationType` - `isNoneEfoRegistrationType` Then you need to refactor these configs, since they are now being used for sink too, and the naming is not good: - `EFO_HTTP_CLIENT_MAX_CONCURRENCY` - `EFO_HTTP_CLIENT_READ_TIMEOUT_MILLIS` ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/pom.xml ########## @@ -0,0 +1,134 @@ +<?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>1.12.7</aws.sdk.version> + <aws.sdkv2.version>2.17.52</aws.sdkv2.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> + + <!-- Amazon AWS SDK v1.x dependencies --> + <dependency> + <groupId>com.amazonaws</groupId> + <artifactId>aws-java-sdk-core</artifactId> + <version>${aws.sdk.version}</version> + </dependency> + + <dependency> + <groupId>com.amazonaws</groupId> + <artifactId>aws-java-sdk-kinesis</artifactId> + <version>${aws.sdk.version}</version> + </dependency> + + <!-- Amazon AWS SDK v2.x dependencies --> + <dependency> + <groupId>software.amazon.awssdk</groupId> + <artifactId>kinesis</artifactId> + <version>${aws.sdkv2.version}</version> + </dependency> + + <dependency> + <groupId>software.amazon.awssdk</groupId> + <artifactId>netty-nio-client</artifactId> + <version>${aws.sdkv2.version}</version> + </dependency> + + <dependency> + <groupId>software.amazon.awssdk</groupId> + <artifactId>sts</artifactId> + <version>${aws.sdkv2.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.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> + <configuration> + <systemPropertyVariables> + <!-- Required for Kinesalite. --> + <!-- Including shaded and non-shaded conf to support test running from Maven and IntelliJ --> + <com.amazonaws.sdk.disableCbor>true</com.amazonaws.sdk.disableCbor> + <com.amazonaws.sdk.disableCertChecking>true</com.amazonaws.sdk.disableCertChecking> + <org.apache.flink.kinesis.shaded.com.amazonaws.sdk.disableCbor>true</org.apache.flink.kinesis.shaded.com.amazonaws.sdk.disableCbor> + <org.apache.flink.kinesis.shaded.com.amazonaws.sdk.disableCertChecking>true</org.apache.flink.kinesis.shaded.com.amazonaws.sdk.disableCertChecking> Review comment: We should refactor the tests to use SDK v2 and drop the V1 dependency -- 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]
