fapaul commented on a change in pull request #17345: URL: https://github.com/apache/flink/pull/17345#discussion_r747447497
########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/test/java/org/apache/flink/connector/kinesis/sink/KinesisDataStreamsSinkITCase.java ########## @@ -0,0 +1,345 @@ +/* + * 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.connector.base.sink.writer.ElementConverter; +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.streaming.connectors.kinesis.testutils.KinesaliteContainer; +import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.TestLogger; + +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +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.DescribeStreamResponse; +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.util.Properties; +import java.util.concurrent.ExecutionException; + +import static org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants.AWS_ACCESS_KEY_ID; +import static org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants.AWS_ENDPOINT; +import static org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants.AWS_REGION; +import static org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants.AWS_SECRET_ACCESS_KEY; +import static org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants.HTTP_PROTOCOL_VERSION; +import static org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants.TRUST_ALL_CERTIFICATES; +import static org.junit.Assert.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; Review comment: Does this work, mixing junit 4 and junit 5 concepts? ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/connector/kinesis/sink/KinesisDataStreamsSink.java ########## @@ -0,0 +1,131 @@ +/* + * 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.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 + * </ul> + * + * <p>Please see the writer implementation in {@link KinesisDataStreamsSinkWriter} + * + * @param <InputT> Type of the elements handled by this sink + */ +@PublicEvolving +public class KinesisDataStreamsSink<InputT> extends AsyncSinkBase<InputT, PutRecordsRequestEntry> { + + private final boolean failOnError; + private final String streamName; + private final Properties kinesisClientProperties; + + KinesisDataStreamsSink( + ElementConverter<InputT, PutRecordsRequestEntry> elementConverter, + Integer maxBatchSize, + Integer maxInFlightRequests, + Integer maxBufferedRequests, + Long flushOnBufferSizeInBytes, + Long maxTimeInBufferMS, + boolean failOnError, + String streamName, + Properties kinesisClientProperties) { + super( + elementConverter, + maxBatchSize, + maxInFlightRequests, + maxBufferedRequests, + flushOnBufferSizeInBytes, + maxTimeInBufferMS); + this.streamName = + Preconditions.checkNotNull( + streamName, + "The stream name must not be null when initializing the KDS Sink."); + Preconditions.checkArgument( + !this.streamName.isEmpty(), + "The stream name must be set when initializing the KDS Sink."); + this.failOnError = failOnError; + this.kinesisClientProperties = kinesisClientProperties; Review comment: Nit: check not null ########## 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); + + final SdkAsyncHttpClient httpClient = + AwsV2Util.createHttpClient( + clientConfiguration, + NettyNioAsyncHttpClient.builder(), + kinesisClientProperties); + + return AwsV2Util.createKinesisAsyncClient( + kinesisClientProperties, clientConfiguration, httpClient); + } + + @Override + protected void submitRequestEntries( + List<PutRecordsRequestEntry> requestEntries, + Consumer<Collection<PutRecordsRequestEntry>> requestResult) { + + PutRecordsRequest batchRequest = + PutRecordsRequest.builder().records(requestEntries).streamName(streamName).build(); + + LOG.trace("Request to submit {} entries to KDS using KDS Sink.", requestEntries.size()); + + CompletableFuture<PutRecordsResponse> future = client.putRecords(batchRequest); + + future.whenComplete( + (response, err) -> { + if (err != null) { + handleFullyFailedRequest( + err, requestEntries, requestResult, getFatalExceptionCons()); + } else if (response.failedRecordCount() > 0) { + handlePartiallyFailedRequest( + response, requestEntries, requestResult, getFatalExceptionCons()); + } else { + requestResult.accept(Collections.emptyList()); + } + }); + } + + @Override + protected long getSizeInBytes(PutRecordsRequestEntry requestEntry) { + return requestEntry.data().asByteArrayUnsafe().length; + } + + private void handleFullyFailedRequest( + Throwable err, + List<PutRecordsRequestEntry> requestEntries, + Consumer<Collection<PutRecordsRequestEntry>> requestResult, + Consumer<Exception> exceptionConsumer) { + LOG.warn("KDS Sink failed to persist {} entries to KDS", requestEntries.size(), err); + numRecordsOutErrorsCounter.inc(requestEntries.size()); + + if (isRetryable(err, exceptionConsumer)) { + requestResult.accept(requestEntries); + } + } + + private void handlePartiallyFailedRequest( + PutRecordsResponse response, + List<PutRecordsRequestEntry> requestEntries, + Consumer<Collection<PutRecordsRequestEntry>> requestResult, + Consumer<Exception> exceptionConsumer) { Review comment: Nit: the parameter is not necessary because you can access the instance method `getFatalExceptionCons` directly. ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/connector/kinesis/sink/KinesisDataStreamsSinkBuilder.java ########## @@ -0,0 +1,117 @@ +/* + * 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.PublicEvolving; +import org.apache.flink.connector.base.sink.AsyncSinkBaseBuilder; + +import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry; + +import java.util.Properties; + +/** + * Builder to construct {@link KinesisDataStreamsSink}. + * + * <p>The following example shows the minimum setup to create a {@link KinesisDataStreamsSink} that + * writes String values to a Kinesis Data Streams stream named your_stream_here. + * + * <pre>{@code + * KinesisDataStreamsSink<String> kdsSink = + * KinesisDataStreamsSink.<String>builder() + * .setElementConverter(elementConverter) Review comment: Nit: I know we haven't been a good example so far but maybe you can replace `elementConverter` with a proper lambda statement to make the example theoretically compilable. ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigGeneralUtil.java ########## @@ -0,0 +1,598 @@ +/* + * 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.streaming.connectors.kinesis.config.AWSConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants.CredentialProvider; +import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.EFORegistrationType; +import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.InitialPosition; +import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.RecordPublisherType; +import org.apache.flink.streaming.connectors.kinesis.config.ProducerConfigConstants; + +import com.amazonaws.regions.Regions; + +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; + +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_STREAM_TIMESTAMP_DATE_FORMAT; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.STREAM_INITIAL_TIMESTAMP; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.STREAM_TIMESTAMP_DATE_FORMAT; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Utilities for Flink Kinesis connector configuration. */ +@Internal +public class KinesisConfigGeneralUtil { + + /** Maximum number of items to pack into an PutRecords request. */ + protected static final String COLLECTION_MAX_COUNT = "CollectionMaxCount"; Review comment: I am not sure I like this pattern of using `protected static` it works but it feels very artificial. Can we replace the inheritance here with just having config classes and sharing the constant? ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/pom.xml ########## @@ -0,0 +1,139 @@ +<?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> + + <dependency> + <groupId>com.amazonaws</groupId> + <artifactId>aws-java-sdk-sts</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> Review comment: Nit: space between the dependencies ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/connector/kinesis/sink/KinesisDataStreamsSinkBuilder.java ########## @@ -0,0 +1,117 @@ +/* + * 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.PublicEvolving; +import org.apache.flink.connector.base.sink.AsyncSinkBaseBuilder; + +import software.amazon.awssdk.services.kinesis.model.PutRecordsRequestEntry; + +import java.util.Properties; + +/** + * Builder to construct {@link KinesisDataStreamsSink}. + * + * <p>The following example shows the minimum setup to create a {@link KinesisDataStreamsSink} that + * writes String values to a Kinesis Data Streams stream named your_stream_here. + * + * <pre>{@code + * KinesisDataStreamsSink<String> kdsSink = + * KinesisDataStreamsSink.<String>builder() + * .setElementConverter(elementConverter) Review comment: I am also wondering whether it is really possible to use any `ElementConverter` or should the sink enforce to always use the `KinesisDataStreamsSinkElementConverter` ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AwsV2Util.java ########## @@ -110,6 +114,20 @@ public static SdkAsyncHttpClient createHttpClient( .map(Duration::ofMillis) .orElse(DEFAULT_EFO_HTTP_CLIENT_READ_TIMEOUT); + boolean trustAllCerts = + Optional.ofNullable( + consumerConfig.getProperty( + AWSConfigConstants.TRUST_ALL_CERTIFICATES)) + .map(Boolean::parseBoolean) + .orElse(DEFAULT_TRUST_ALL_CERTIFICATES); + + Protocol httpProtocolVersion = + Optional.ofNullable( + consumerConfig.getProperty( + AWSConfigConstants.HTTP_PROTOCOL_VERSION)) + .map(Protocol::valueOf) + .orElse(DEFAULT_HTTP_PROTOCOL); Review comment: Why is this change needed? To me, it is not obvious why the sink needs it. Can you put this on a separate commit? ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/test/java/org/apache/flink/streaming/connectors/kinesis/testutils/KinesaliteContainer.java ########## @@ -45,23 +56,18 @@ 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 Region REGION = Region.US_EAST_1; public KinesaliteContainer(DockerImageName imageName) { super(imageName); + System.setProperty("com.amazonaws.sdk.disableCertChecking", "true"); Review comment: Is this still needed although we are setting the property through maven? ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/AwsV2UtilTest.java ########## @@ -258,7 +263,12 @@ public void testCreateNettyHttpClientWithDefaults() { AwsV2Util.createHttpClient(clientConfiguration, builder, new Properties()); - verify(builder).build(); + AttributeMap defaultCustomAttribute = Review comment: Why do we need this change? ########## 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); + + final SdkAsyncHttpClient httpClient = + AwsV2Util.createHttpClient( + clientConfiguration, + NettyNioAsyncHttpClient.builder(), + kinesisClientProperties); + + return AwsV2Util.createKinesisAsyncClient( + kinesisClientProperties, clientConfiguration, httpClient); + } + + @Override + protected void submitRequestEntries( + List<PutRecordsRequestEntry> requestEntries, + Consumer<Collection<PutRecordsRequestEntry>> requestResult) { + + PutRecordsRequest batchRequest = + PutRecordsRequest.builder().records(requestEntries).streamName(streamName).build(); + + LOG.trace("Request to submit {} entries to KDS using KDS Sink.", requestEntries.size()); + + CompletableFuture<PutRecordsResponse> future = client.putRecords(batchRequest); + + future.whenComplete( + (response, err) -> { + if (err != null) { + handleFullyFailedRequest( + err, requestEntries, requestResult, getFatalExceptionCons()); + } else if (response.failedRecordCount() > 0) { + handlePartiallyFailedRequest( + response, requestEntries, requestResult, getFatalExceptionCons()); + } else { + requestResult.accept(Collections.emptyList()); + } + }); + } + + @Override + protected long getSizeInBytes(PutRecordsRequestEntry requestEntry) { + return requestEntry.data().asByteArrayUnsafe().length; + } + + private void handleFullyFailedRequest( + Throwable err, + List<PutRecordsRequestEntry> requestEntries, + Consumer<Collection<PutRecordsRequestEntry>> requestResult, + Consumer<Exception> exceptionConsumer) { Review comment: Nit: the parameter is not necessary because you can access the instance method getFatalExceptionCons directly. ########## 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,72 @@ +/* + * 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.connector.base.sink.writer.ElementConverter; +import org.apache.flink.connector.kinesis.sink.KinesisDataStreamsSink; +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 software.amazon.awssdk.core.SdkBytes; +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 = + (element, context) -> + PutRecordsRequestEntry.builder() + .data(SdkBytes.fromUtf8String(element)) + .partitionKey(String.valueOf(element.hashCode())) + .build(); + + public static void main(String[] args) throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(10_000); + + DataGeneratorSource<String> src = + new DataGeneratorSource<>(RandomGenerator.stringGenerator(10), 10, 10_000_000L); Review comment: I'd rather not expose the `DataGeneratorSource` it is marked as experimental. You can use `env.fromElement` or `env.fromSequence` ########## File path: flink-connectors/flink-connector-aws-kinesis-data-streams/src/test/java/org/apache/flink/streaming/connectors/kinesis/util/KinesisConfigGeneralUtilTest.java ########## @@ -0,0 +1,912 @@ +/* + * 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.streaming.connectors.kinesis.config.AWSConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants; +import org.apache.flink.streaming.connectors.kinesis.testutils.GeneralTestUtils; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Date; +import java.util.List; +import java.util.Properties; +import java.util.stream.Collectors; + +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.DEFAULT_STREAM_TIMESTAMP_DATE_FORMAT; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.EFO_HTTP_CLIENT_MAX_CONCURRENCY; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.STREAM_INITIAL_TIMESTAMP; +import static org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants.STREAM_TIMESTAMP_DATE_FORMAT; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** Tests for KinesisConfigGeneralUtil. */ +@RunWith(PowerMockRunner.class) Review comment: I suppose you copied this class? -- 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]
