CrynetLogistics commented on a change in pull request #17345: URL: https://github.com/apache/flink/pull/17345#discussion_r740973594
########## File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/unified/KinesisDataStreamsSink.java ########## @@ -0,0 +1,130 @@ +/* + * 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.unified; + +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); + Preconditions.checkNotNull( + streamName, "The stream name must not be null when initializing the KDS Sink."); + Preconditions.checkArgument( + !streamName.isEmpty(), + "The stream name must be set when initializing the KDS Sink."); + this.failOnError = failOnError; Review comment: This was added to fail the job immediately if there is any exception encountered by the Sink. If not, the request batch will be readded to the buffer via the `requestResult` consumer. ########## File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/unified/KinesisDataStreamsSink.java ########## @@ -0,0 +1,130 @@ +/* + * 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.unified; + +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); + Preconditions.checkNotNull( + streamName, "The stream name must not be null when initializing the KDS Sink."); + Preconditions.checkArgument( + !streamName.isEmpty(), + "The stream name must be set when initializing the KDS Sink."); + this.failOnError = failOnError; Review comment: This will fail the job immediately if there is any exception encountered by the Sink. If not, the request batch will be readded to the buffer via the `requestResult` consumer. ########## File path: flink-connectors/flink-connector-aws/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</artifactId> + <name>Flink : Connectors : AWS</name> + <properties> + <aws.sdkv2.version>2.17.52</aws.sdkv2.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-table-api-java-bridge</artifactId> + <version>${project.version}</version> + <scope>provided</scope> + <optional>true</optional> + </dependency> + + <!-- Amazon AWS SDK v2.x dependencies --> + <dependency> + <groupId>software.amazon.awssdk</groupId> + <artifactId>kinesis</artifactId> + <version>${aws.sdkv2.version}</version> + </dependency> + + <dependency> + <groupId>com.amazonaws</groupId> + <artifactId>aws-java-sdk-core</artifactId> + <version>1.12.7</version> Review comment: I think so :-) Thank you. ########## File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/unified/KinesisDataStreamsSinkBuilder.java ########## @@ -0,0 +1,110 @@ +/* + * 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.unified; + +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) + * .setStreamName("your_stream_name") + * .build(); + * }</pre> + * + * <p>If the following parameters are not set in this builder, the following defaults will be used: + * + * <ul> + * <li>{@code maxBatchSize} will be 200 + * <li>{@code maxInFlightRequests} will be 16 + * <li>{@code maxBufferedRequests} will be 10000 + * <li>{@code flushOnBufferSizeInBytes} will be 64MB i.e. {@code 64 * 1024 * 1024} + * <li>{@code maxTimeInBufferMS} will be 5000ms + * <li>{@code failOnError} will be false + * </ul> + * + * @param <InputT> type of elements that should be persisted in the destination + */ +@PublicEvolving +public class KinesisDataStreamsSinkBuilder<InputT> + extends AsyncSinkBaseBuilder< + InputT, PutRecordsRequestEntry, KinesisDataStreamsSinkBuilder<InputT>> { + + private static final int DEFAULT_MAX_BATCH_SIZE = 200; + private static final int DEFAULT_MAX_IN_FLIGHT_REQUESTS = 16; + private static final int DEFAULT_MAX_BUFFERED_REQUESTS = 10000; + private static final long DEFAULT_FLUSH_ON_BUFFER_SIZE_IN_B = 64 * 1024 * 1024; + private static final long DEFAULT_MAX_TIME_IN_BUFFER_MS = 5000; + private static final boolean DEFAULT_FAIL_ON_ERROR = false; + + private Boolean failOnError; + private String streamName; + private Properties kinesisClientProperties; + + KinesisDataStreamsSinkBuilder() {} + + /** + * Sets the name of the KDS stream that the sink will connect to. There is no default for this + * parameter, therefore, this must be provided at sink creation time otherwise the build will + * fail. + * + * @param streamName the name of the stream + * @return {@link KinesisDataStreamsSinkBuilder} itself + */ + public KinesisDataStreamsSinkBuilder<InputT> setStreamName(String streamName) { + this.streamName = streamName; + return this; + } + + public KinesisDataStreamsSinkBuilder<InputT> setFailOnError(boolean failOnError) { + this.failOnError = failOnError; + return this; + } + + public KinesisDataStreamsSinkBuilder<InputT> setKinesisClientProperties( + Properties kinesisClientProperties) { + this.kinesisClientProperties = kinesisClientProperties; + return this; + } + + public KinesisDataStreamsSink<InputT> build() { Review comment: It definitely should, I appreciate that. ########## File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/unified/KinesisDataStreamsSinkWriter.java ########## @@ -0,0 +1,218 @@ +/* + * 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.unified; + +import org.apache.flink.annotation.Internal; +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.unified.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. + */ +@Internal +public class KinesisDataStreamsSinkWriter<InputT> + extends AsyncSinkWriter<InputT, PutRecordsRequestEntry> { + + private static final String TOTAL_FULLY_SUCCESSFUL_FLUSHES_METRIC = + "totalFullySuccessfulFlushes"; + private static final String TOTAL_PARTIALLY_SUCCESSFUL_FLUSHES_METRIC = + "totalPartiallySuccessfulFlushes"; + private static final String TOTAL_FULLY_FAILED_FLUSHES_METRIC = "totalFullyFailedFlushes"; + private transient Counter totalFullySuccessfulFlushesCounter; + private transient Counter totalPartiallySuccessfulFlushesCounter; + private transient Counter totalFullyFailedFlushesCounter; + private transient Counter numRecordsOutErrorsCounter; Review comment: I previously thought fields in `SinkWriter` had to be serialisable... but it's the `Sink` only. :-) ########## File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/unified/KinesisDataStreamsSinkBuilder.java ########## @@ -0,0 +1,110 @@ +/* + * 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.unified; + +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) + * .setStreamName("your_stream_name") + * .build(); + * }</pre> + * + * <p>If the following parameters are not set in this builder, the following defaults will be used: + * + * <ul> + * <li>{@code maxBatchSize} will be 200 + * <li>{@code maxInFlightRequests} will be 16 + * <li>{@code maxBufferedRequests} will be 10000 + * <li>{@code flushOnBufferSizeInBytes} will be 64MB i.e. {@code 64 * 1024 * 1024} + * <li>{@code maxTimeInBufferMS} will be 5000ms + * <li>{@code failOnError} will be false + * </ul> + * + * @param <InputT> type of elements that should be persisted in the destination + */ +@PublicEvolving +public class KinesisDataStreamsSinkBuilder<InputT> + extends AsyncSinkBaseBuilder< + InputT, PutRecordsRequestEntry, KinesisDataStreamsSinkBuilder<InputT>> { + + private static final int DEFAULT_MAX_BATCH_SIZE = 200; + private static final int DEFAULT_MAX_IN_FLIGHT_REQUESTS = 16; + private static final int DEFAULT_MAX_BUFFERED_REQUESTS = 10000; + private static final long DEFAULT_FLUSH_ON_BUFFER_SIZE_IN_B = 64 * 1024 * 1024; + private static final long DEFAULT_MAX_TIME_IN_BUFFER_MS = 5000; + private static final boolean DEFAULT_FAIL_ON_ERROR = false; + + private Boolean failOnError; + private String streamName; + private Properties kinesisClientProperties; + + KinesisDataStreamsSinkBuilder() {} + + /** + * Sets the name of the KDS stream that the sink will connect to. There is no default for this + * parameter, therefore, this must be provided at sink creation time otherwise the build will + * fail. + * + * @param streamName the name of the stream + * @return {@link KinesisDataStreamsSinkBuilder} itself + */ + public KinesisDataStreamsSinkBuilder<InputT> setStreamName(String streamName) { + this.streamName = streamName; + return this; + } + + public KinesisDataStreamsSinkBuilder<InputT> setFailOnError(boolean failOnError) { + this.failOnError = failOnError; + return this; + } + + public KinesisDataStreamsSinkBuilder<InputT> setKinesisClientProperties( + Properties kinesisClientProperties) { + this.kinesisClientProperties = kinesisClientProperties; + return this; + } + + public KinesisDataStreamsSink<InputT> build() { Review comment: Oh actually, I would have to add a type parameter to the base class... do you think it will confuse? I imagine most people will know to `build()`... ########## File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/unified/KinesisDataStreamsSinkWriter.java ########## @@ -0,0 +1,218 @@ +/* + * 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.unified; + +import org.apache.flink.annotation.Internal; +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.unified.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. + */ +@Internal +public class KinesisDataStreamsSinkWriter<InputT> + extends AsyncSinkWriter<InputT, PutRecordsRequestEntry> { + + private static final String TOTAL_FULLY_SUCCESSFUL_FLUSHES_METRIC = + "totalFullySuccessfulFlushes"; + private static final String TOTAL_PARTIALLY_SUCCESSFUL_FLUSHES_METRIC = + "totalPartiallySuccessfulFlushes"; + private static final String TOTAL_FULLY_FAILED_FLUSHES_METRIC = "totalFullyFailedFlushes"; + private transient Counter totalFullySuccessfulFlushesCounter; + private transient Counter totalPartiallySuccessfulFlushesCounter; + private transient Counter totalFullyFailedFlushesCounter; + private transient 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(); + initMetricsGroup(); + 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, + Consumer<Exception> exceptionConsumer) { + + 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) { + LOG.warn( + "KDS Sink failed to persist {} entries to KDS", + requestEntries.size(), + err); + totalFullyFailedFlushesCounter.inc(); + numRecordsOutErrorsCounter.inc(requestEntries.size()); + + if (isRetryable(err, exceptionConsumer)) { + requestResult.accept(requestEntries); + } Review comment: So I retry elements from a partially failed one on line 178, where I build up the elements that have failed in a batch where some have succeeded and add those back to the buffer. ########## File path: flink-connectors/flink-connector-aws/src/test/resources/profile ########## @@ -0,0 +1,7 @@ +[default] +aws_access_key_id=11111111111111111111 +aws_secret_access_key=wJalrXUtnFEMI/K7MDENG/bPxRfiCY1111111111 + +[foo] +aws_access_key_id=22222222222222222222 Review comment: This file contains dummy credentials for the AwsV2Util test. ########## File path: flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/sink/writer/AsyncSinkWriter.java ########## @@ -236,8 +273,16 @@ private void flush() { "Mark in-flight request as completed and requeue %d request entries", failedRequestEntries.size()); + Consumer<Exception> fatalExceptionCons = + exception -> + mailboxExecutor.execute( + () -> { + throw exception; + }, + "A fatal exception occurred in the sink that cannot be recovered from or should not be retried."); + inFlightRequestsCount++; - submitRequestEntries(batch, requestResult); + submitRequestEntries(batch, requestResult, fatalExceptionCons); Review comment: Sound good to me ! ########## File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/unified/KinesisDataStreamsSink.java ########## @@ -0,0 +1,130 @@ +/* + * 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.unified; Review comment: Cool. I've chosen `org.apache.flink.connector.kinesis.sink` to match the Kafka unified sink, which is `org.apache.flink.connector.kafka.sink`. Thanks for the suggestion! ########## File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/unified/util/AWSConfigConstants.java ########## @@ -0,0 +1,167 @@ +/* + * 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.unified.util; + +import org.apache.flink.annotation.PublicEvolving; + +import com.amazonaws.auth.AWSCredentialsProvider; + +/** Configuration keys for AWS service usage. */ +@PublicEvolving +public class AWSConfigConstants { + + /** + * Possible configuration values for the type of credential provider to use when accessing AWS + * Kinesis. Internally, a corresponding implementation of {@link AWSCredentialsProvider} will be + * used. + */ + public enum CredentialProvider { + + /** + * Look for the environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY to create + * AWS credentials. + */ + ENV_VAR, + + /** + * Look for Java system properties aws.accessKeyId and aws.secretKey to create AWS + * credentials. + */ + SYS_PROP, + + /** Use a AWS credentials profile file to create the AWS credentials. */ + PROFILE, + + /** + * Simply create AWS credentials by supplying the AWS access key ID and AWS secret key in + * the configuration properties. + */ + BASIC, + + /** + * Create AWS credentials by assuming a role. The credentials for assuming the role must be + * supplied. * + */ + ASSUME_ROLE, + + /** + * Use AWS WebIdentityToken in order to assume a role. A token file and role details can be + * supplied as configuration or environment variables. * + */ + WEB_IDENTITY_TOKEN, + + /** + * A credentials provider chain will be used that searches for credentials in this order: + * ENV_VARS, SYS_PROPS, WEB_IDENTITY_TOKEN, PROFILE in the AWS instance metadata. * + */ + AUTO, + } + + /** The AWS region of the Kinesis streams to be pulled ("us-east-1" is used if not set). */ + public static final String AWS_REGION = "aws.region"; + + /** + * The credential provider type to use when AWS credentials are required (BASIC is used if not + * set). + */ + public static final String AWS_CREDENTIALS_PROVIDER = "aws.credentials.provider"; + + /** The AWS access key ID to use when setting credentials provider type to BASIC. */ + public static final String AWS_ACCESS_KEY_ID = accessKeyId(AWS_CREDENTIALS_PROVIDER); + + /** The AWS secret key to use when setting credentials provider type to BASIC. */ + public static final String AWS_SECRET_ACCESS_KEY = secretKey(AWS_CREDENTIALS_PROVIDER); + + /** Optional configuration for profile path if credential provider type is set to be PROFILE. */ + public static final String AWS_PROFILE_PATH = profilePath(AWS_CREDENTIALS_PROVIDER); + + /** Optional configuration for profile name if credential provider type is set to be PROFILE. */ + public static final String AWS_PROFILE_NAME = profileName(AWS_CREDENTIALS_PROVIDER); + + /** + * The role ARN to use when credential provider type is set to ASSUME_ROLE or + * WEB_IDENTITY_TOKEN. + */ + public static final String AWS_ROLE_ARN = roleArn(AWS_CREDENTIALS_PROVIDER); Review comment: Thank you, I will remove the unused fields. But I wanted to ask about this actually, there is some duplication here with `AWSConfigConstants` and `AwsV2Util` of `flink-connector-kinesis`. I can't use the same trick we used for `KinesaliteContainer` since it's not a test class (for fear of breaking existing customer code). ########## File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/unified/KinesisDataStreamsSinkWriter.java ########## @@ -0,0 +1,218 @@ +/* + * 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.unified; + +import org.apache.flink.annotation.Internal; +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.unified.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. + */ +@Internal +public class KinesisDataStreamsSinkWriter<InputT> + extends AsyncSinkWriter<InputT, PutRecordsRequestEntry> { + + private static final String TOTAL_FULLY_SUCCESSFUL_FLUSHES_METRIC = + "totalFullySuccessfulFlushes"; + private static final String TOTAL_PARTIALLY_SUCCESSFUL_FLUSHES_METRIC = + "totalPartiallySuccessfulFlushes"; + private static final String TOTAL_FULLY_FAILED_FLUSHES_METRIC = "totalFullyFailedFlushes"; Review comment: Sounds good, I'll remove my custom ones and implement `setCurrentSendTimeGauge` (implementation similar to the one in `ElasticsearchWriter`) and `numRecordsOutCounter`/`numBytesOutCounter` that are also in the `KafkaWriter`. ########## File path: flink-connectors/flink-connector-aws/src/main/java/org/apache/flink/streaming/connectors/kinesis/unified/KinesisDataStreamsSinkWriter.java ########## @@ -0,0 +1,218 @@ +/* + * 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.unified; + +import org.apache.flink.annotation.Internal; +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.unified.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. + */ +@Internal +public class KinesisDataStreamsSinkWriter<InputT> + extends AsyncSinkWriter<InputT, PutRecordsRequestEntry> { + + private static final String TOTAL_FULLY_SUCCESSFUL_FLUSHES_METRIC = + "totalFullySuccessfulFlushes"; + private static final String TOTAL_PARTIALLY_SUCCESSFUL_FLUSHES_METRIC = + "totalPartiallySuccessfulFlushes"; Review comment: See comment above. -- 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]
