hlteoh37 commented on code in PR #141: URL: https://github.com/apache/flink-connector-aws/pull/141#discussion_r1625636303
########## docs/content.zh/docs/connectors/datastream/sqs.md: ########## @@ -0,0 +1,134 @@ +--- +title: DynamoDB +weight: 5 +type: docs +aliases: +- /zh/dev/connectors/dynamodb.html Review Comment: These are wrong and should be changed to `sqs.html` ########## docs/content.zh/docs/connectors/datastream/sqs.md: ########## @@ -0,0 +1,134 @@ +--- +title: DynamoDB +weight: 5 +type: docs +aliases: +- /zh/dev/connectors/dynamodb.html +--- +<!-- +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. +--> + +# Amazon SQS Sink + +The SQS sink writes to [Amazon SQS](https://aws.amazon.com/sqs) using the [AWS v2 SDK for Java](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/home.html). Follow the instructions from the [Amazon SQS Developer Guide](https://docs.aws.amazon.com/AWSSimpleQueueService/latest/SQSDeveloperGuide/welcome.html) +to setup a SQS. + +To use the connector, add the following Maven dependency to your project: + +{{< connector_artifact flink-connector-sqs sqs >}} + +{{< tabs "ec24a4ae-6a47-11ed-a1eb-0242ac120002" >}} +{{< tab "Java" >}} +```java +Properties sinkProperties = new Properties(); +// Required +sinkProperties.put(AWSConfigConstants.AWS_REGION, "eu-west-1"); Review Comment: Is there a reason we need to specify this? Can we figure this out via the SQS URL? ########## flink-connector-aws/flink-connector-sqs/src/main/java/org.apache.flink.connector.sqs/sink/SqsConfigConstants.java: ########## @@ -0,0 +1,31 @@ +/* + * 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.sqs.sink; + +import org.apache.flink.annotation.PublicEvolving; + +/** Defaults for {@link SqsSinkWriter}. */ +@PublicEvolving +public class SqsConfigConstants { + + public static final String BASE_SQS_USER_AGENT_PREFIX_FORMAT = Review Comment: Could we use `ConfigOption` instead of strings here? Example: https://github.com/apache/flink-connector-aws/blob/38aafb44d3a8200e4ff41d87e0780338f40da258/flink-connector-aws/flink-connector-aws-kinesis-streams/src/main/java/org/apache/flink/connector/kinesis/source/config/KinesisStreamsSourceConfigConstants.java#L41-L46 ########## docs/content.zh/docs/connectors/datastream/sqs.md: ########## @@ -0,0 +1,134 @@ +--- +title: DynamoDB +weight: 5 +type: docs +aliases: +- /zh/dev/connectors/dynamodb.html +--- +<!-- +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. +--> + +# Amazon SQS Sink + +The SQS sink writes to [Amazon SQS](https://aws.amazon.com/sqs) using the [AWS v2 SDK for Java](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/home.html). Follow the instructions from the [Amazon SQS Developer Guide](https://docs.aws.amazon.com/AWSSimpleQueueService/latest/SQSDeveloperGuide/welcome.html) +to setup a SQS. Review Comment: nit: setup an SQS message queue. ########## flink-connector-aws/flink-connector-sqs/src/main/java/org.apache.flink.connector.sqs/sink/SqsSinkWriter.java: ########## @@ -0,0 +1,257 @@ +/* + * 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.sqs.sink; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.connector.aws.sink.throwable.AWSExceptionHandler; +import org.apache.flink.connector.aws.util.AWSClientUtil; +import org.apache.flink.connector.aws.util.AWSGeneralUtil; +import org.apache.flink.connector.base.sink.throwable.FatalExceptionClassifier; +import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter; +import org.apache.flink.connector.base.sink.writer.BufferedRequestState; +import org.apache.flink.connector.base.sink.writer.ElementConverter; +import org.apache.flink.connector.base.sink.writer.config.AsyncSinkWriterConfiguration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.http.async.SdkAsyncHttpClient; +import software.amazon.awssdk.services.sqs.SqsAsyncClient; +import software.amazon.awssdk.services.sqs.model.BatchResultErrorEntry; +import software.amazon.awssdk.services.sqs.model.SendMessageBatchRequest; +import software.amazon.awssdk.services.sqs.model.SendMessageBatchRequestEntry; +import software.amazon.awssdk.services.sqs.model.SendMessageBatchResponse; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; + +import static org.apache.flink.connector.aws.util.AWSCredentialFatalExceptionClassifiers.getInvalidCredentialsExceptionClassifier; +import static org.apache.flink.connector.aws.util.AWSCredentialFatalExceptionClassifiers.getSdkClientMisconfiguredExceptionClassifier; +import static org.apache.flink.connector.base.sink.writer.AsyncSinkFatalExceptionClassifiers.getInterruptedExceptionClassifier; + +/** + * Sink writer created by {@link SqsSink} to write to SQS. More details on the operation of this + * sink writer may be found in the doc for {@link SqsSink}. More details on the internals of this + * sink writer may be found in {@link AsyncSinkWriter}. + * + * <p>The {@link SqsAsyncClient} 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 +class SqsSinkWriter<InputT> extends AsyncSinkWriter<InputT, SendMessageBatchRequestEntry> { + + private static final Logger LOG = LoggerFactory.getLogger(SqsSinkWriter.class); + + private static SdkAsyncHttpClient createHttpClient(Properties sqsClientProperties) { + return AWSGeneralUtil.createAsyncHttpClient(sqsClientProperties); + } + + private static SqsAsyncClient createSqsClient( + Properties sqsClientProperties, SdkAsyncHttpClient httpClient) { + AWSGeneralUtil.validateAwsCredentials(sqsClientProperties); + return AWSClientUtil.createAwsAsyncClient( + sqsClientProperties, + httpClient, + SqsAsyncClient.builder(), + SqsConfigConstants.BASE_SQS_USER_AGENT_PREFIX_FORMAT, + SqsConfigConstants.SQS_CLIENT_USER_AGENT_PREFIX); + } + + private static final AWSExceptionHandler SQS_EXCEPTION_HANDLER = + AWSExceptionHandler.withClassifier( + FatalExceptionClassifier.createChain( + getInterruptedExceptionClassifier(), + getInvalidCredentialsExceptionClassifier(), + SqsExceptionClassifiers.getResourceNotFoundExceptionClassifier(), + SqsExceptionClassifiers.getAccessDeniedExceptionClassifier(), + SqsExceptionClassifiers.getNotAuthorizedExceptionClassifier(), + getSdkClientMisconfiguredExceptionClassifier())); + + private final Counter numRecordsOutErrorsCounter; + + /* Url of SQS */ + private final String sqsUrl; + + /* The sink writer metric group */ + private final SinkWriterMetricGroup metrics; + + /* The asynchronous http client */ + private SdkAsyncHttpClient httpClient; + + /* The asynchronous SQS client */ + private SqsAsyncClient sqsAsyncClient; + + /* Flag to whether fatally fail any time we encounter an exception when persisting records */ + private final boolean failOnError; + + SqsSinkWriter( + ElementConverter<InputT, SendMessageBatchRequestEntry> elementConverter, + Sink.InitContext context, + int maxBatchSize, + int maxInFlightRequests, + int maxBufferedRequests, + long maxBatchSizeInBytes, + long maxTimeInBufferMS, + long maxRecordSizeInBytes, + boolean failOnError, + String sqsUrl, + Properties sqsClientProperties, + Collection<BufferedRequestState<SendMessageBatchRequestEntry>> initialStates) { + super( + elementConverter, + context, + AsyncSinkWriterConfiguration.builder() + .setMaxBatchSize(maxBatchSize) + .setMaxBatchSizeInBytes(maxBatchSizeInBytes) + .setMaxInFlightRequests(maxInFlightRequests) + .setMaxBufferedRequests(maxBufferedRequests) + .setMaxTimeInBufferMS(maxTimeInBufferMS) + .setMaxRecordSizeInBytes(maxRecordSizeInBytes) + .build(), + initialStates); + this.failOnError = failOnError; + this.sqsUrl = sqsUrl; + this.metrics = context.metricGroup(); + this.numRecordsOutErrorsCounter = metrics.getNumRecordsOutErrorsCounter(); + this.httpClient = createHttpClient(sqsClientProperties); + this.sqsAsyncClient = createSqsClient(sqsClientProperties, httpClient); + } + + @Override + protected void submitRequestEntries( + List<SendMessageBatchRequestEntry> requestEntries, + Consumer<List<SendMessageBatchRequestEntry>> requestResult) { + + final SendMessageBatchRequest batchRequest = + SendMessageBatchRequest.builder().entries(requestEntries).queueUrl(sqsUrl).build(); + + CompletableFuture<SendMessageBatchResponse> future = + sqsAsyncClient.sendMessageBatch(batchRequest); + + future.whenComplete( + (response, err) -> { + if (err != null) { + handleFullyFailedRequest(err, requestEntries, requestResult); + } else if (response.failed() != null && response.failed().size() > 0) { + handlePartiallyFailedRequest(response, requestEntries, requestResult); + } else { + requestResult.accept(Collections.emptyList()); + } + }); + } + + @Override + protected long getSizeInBytes(SendMessageBatchRequestEntry requestEntry) { + return requestEntry.messageBody().getBytes(StandardCharsets.UTF_8).length; + } + + @Override + public void close() { + AWSGeneralUtil.closeResources(httpClient, sqsAsyncClient); + } + + private void handleFullyFailedRequest( + Throwable err, + List<SendMessageBatchRequestEntry> requestEntries, + Consumer<List<SendMessageBatchRequestEntry>> requestResult) { + + numRecordsOutErrorsCounter.inc(requestEntries.size()); + boolean isFatal = SQS_EXCEPTION_HANDLER.consumeIfFatal(err, getFatalExceptionCons()); + if (isFatal) { + return; + } + + if (failOnError) { + getFatalExceptionCons().accept(new SqsSinkException.SqsFailFastSinkException(err)); + return; + } + + LOG.warn( + "SQS Sink failed to write and will retry {} entries to SQS first request was {}", + requestEntries.size(), + requestEntries.get(0).toString(), + err); + requestResult.accept(requestEntries); + } + + private void handlePartiallyFailedRequest( + SendMessageBatchResponse response, + List<SendMessageBatchRequestEntry> requestEntries, + Consumer<List<SendMessageBatchRequestEntry>> requestResult) { + + if (response.failed() != null) { + LOG.warn( + "handlePartiallyFailedRequest: SQS Sink failed to write and will retry {} entries to SQS", + response.failed().size()); + numRecordsOutErrorsCounter.inc(response.failed().size()); + } + + if (failOnError) { + getFatalExceptionCons().accept(new SqsSinkException.SqsFailFastSinkException()); + return; + } + + if (response.failed() != null) { + final List<SendMessageBatchRequestEntry> failedRequestEntries = + new ArrayList<>(response.failed().size()); + + for (final BatchResultErrorEntry failedEntry : response.failed()) { + final Optional<SendMessageBatchRequestEntry> retryEntry = + getFailedRecord(requestEntries, failedEntry.id()); + if (retryEntry.isPresent()) { + failedRequestEntries.add(retryEntry.get()); + } + } + + requestResult.accept(failedRequestEntries); + } + } + + private Optional<SendMessageBatchRequestEntry> getFailedRecord( + List<SendMessageBatchRequestEntry> requestEntries, String selectedId) { + for (SendMessageBatchRequestEntry entry : requestEntries) { + if (entry.id().equals(selectedId)) { + return Optional.of(entry); + } + } + return Optional.empty(); + } + + @Internal + @VisibleForTesting + void setSqsAsyncClient(final SqsAsyncClient sqsAsyncClient) { + this.sqsAsyncClient = sqsAsyncClient; + } + + @Internal + @VisibleForTesting + void setSdkAsyncHttpClient(final SdkAsyncHttpClient httpClient) { + this.httpClient = httpClient; + } Review Comment: This is an antipattern, could we instead use a `ClientProvider` as shown, so we can test it without `@VisibleForTesting` instead? Example: https://github.com/apache/flink-connector-aws/blob/38aafb44d3a8200e4ff41d87e0780338f40da258/flink-connector-aws/flink-connector-dynamodb/src/main/java/org/apache/flink/connector/dynamodb/sink/DynamoDbSinkWriter.java#L139 ########## docs/content.zh/docs/connectors/datastream/sqs.md: ########## @@ -0,0 +1,134 @@ +--- +title: DynamoDB Review Comment: Typo - this should be SQS ########## flink-connector-aws-e2e-tests/flink-connector-aws-sqs-e2e-tests/pom.xml: ########## @@ -0,0 +1,122 @@ +<?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/xsd/maven-4.0.0.xsd"> + + <parent> + <artifactId>flink-connector-aws-e2e-tests-parent</artifactId> + <groupId>org.apache.flink</groupId> + <version>4.3-SNAPSHOT</version> + </parent> + + <modelVersion>4.0.0</modelVersion> + + <artifactId>flink-connector-aws-sqs-e2e-tests</artifactId> + <name>Flink : Connectors : AWS : E2E Tests : Amazon SQS</name> + <packaging>jar</packaging> + + <dependencies> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-streaming-java</artifactId> + <version>${flink.version}</version> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-connector-sqs</artifactId> + <version>${project.version}</version> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-connector-aws-base</artifactId> + <version>${project.version}</version> + <scope>test</scope> + <type>test-jar</type> + <exclusions> + <exclusion> + <groupId>com.typesafe.netty</groupId> + <artifactId>netty-reactive-streams-http</artifactId> + </exclusion> + </exclusions> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-connector-sqs</artifactId> + <version>${project.version}</version> + <scope>test</scope> + <type>test-jar</type> + <exclusions> + <exclusion> + <groupId>com.typesafe.netty</groupId> + <artifactId>netty-reactive-streams-http</artifactId> + </exclusion> + </exclusions> + </dependency> + + <!-- Other third-party dependencies --> + <dependency> + <groupId>com.google.guava</groupId> + <artifactId>guava</artifactId> + </dependency> Review Comment: Should we scope to `test`? ########## flink-connector-aws-e2e-tests/flink-connector-aws-sqs-e2e-tests/pom.xml: ########## @@ -0,0 +1,122 @@ +<?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/xsd/maven-4.0.0.xsd"> + + <parent> + <artifactId>flink-connector-aws-e2e-tests-parent</artifactId> + <groupId>org.apache.flink</groupId> + <version>4.3-SNAPSHOT</version> + </parent> + + <modelVersion>4.0.0</modelVersion> + + <artifactId>flink-connector-aws-sqs-e2e-tests</artifactId> + <name>Flink : Connectors : AWS : E2E Tests : Amazon SQS</name> + <packaging>jar</packaging> + + <dependencies> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-streaming-java</artifactId> + <version>${flink.version}</version> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-connector-sqs</artifactId> + <version>${project.version}</version> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-connector-aws-base</artifactId> + <version>${project.version}</version> + <scope>test</scope> + <type>test-jar</type> + <exclusions> + <exclusion> + <groupId>com.typesafe.netty</groupId> + <artifactId>netty-reactive-streams-http</artifactId> + </exclusion> + </exclusions> Review Comment: Why do we need this exclusion? ########## docs/content/docs/connectors/datastream/sqs.md: ########## @@ -0,0 +1,134 @@ +--- +title: DynamoDB +weight: 5 +type: docs +aliases: +- /zh/dev/connectors/dynamodb.html Review Comment: Same as above - should be SQS instead of DynamoDB ########## docs/content.zh/docs/connectors/datastream/sqs.md: ########## @@ -0,0 +1,134 @@ +--- +title: DynamoDB +weight: 5 +type: docs +aliases: +- /zh/dev/connectors/dynamodb.html +--- +<!-- +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. +--> + +# Amazon SQS Sink + +The SQS sink writes to [Amazon SQS](https://aws.amazon.com/sqs) using the [AWS v2 SDK for Java](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/home.html). Follow the instructions from the [Amazon SQS Developer Guide](https://docs.aws.amazon.com/AWSSimpleQueueService/latest/SQSDeveloperGuide/welcome.html) +to setup a SQS. + +To use the connector, add the following Maven dependency to your project: + +{{< connector_artifact flink-connector-sqs sqs >}} + +{{< tabs "ec24a4ae-6a47-11ed-a1eb-0242ac120002" >}} +{{< tab "Java" >}} +```java +Properties sinkProperties = new Properties(); +// Required +sinkProperties.put(AWSConfigConstants.AWS_REGION, "eu-west-1"); +// Optional, provide via alternative routes e.g. environment variables +sinkProperties.put(AWSConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id"); +sinkProperties.put(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key"); + +// Optional, use following if you want to provide access via AssumeRole +sinkProperties.setProperty(AWSConfigConstants.AWS_CREDENTIALS_PROVIDER, "ASSUME_ROLE"); +sinkProperties.setProperty(AWSConfigConstants.AWS_ROLE_ARN, "replace-this-with-IAMRole-arn"); +sinkProperties.setProperty(AWSConfigConstants.AWS_ROLE_SESSION_NAME, "any-session-name-string"); + +SqsSink<String> sqsSink = +SqsSink.<String>builder() + .setSerializationSchema(new SimpleStringSchema()) // Required + .setSqsUrl("https://sqs.us-east-1.amazonaws.com/xxxx/test-sqs") // Required + .setSqsClientProperties(sinkProperties) // Required + .setFailOnError(false) // Optional + .setMaxBatchSize(10) // Optional + .setMaxInFlightRequests(50) // Optional + .setMaxBufferedRequests(1_000) // Optional + .setMaxBatchSizeInBytes(256 * 1024) // Optional + .setMaxTimeInBufferMS(5000) // Optional + .setMaxRecordSizeInBytes(256 * 1024) // Optional + .build(); + +flinkStream.sinkTo(sqsSink) + +``` +{{< /tab >}} +{{< tab "Scala" >}} +```scala +val sinkProperties = new Properties() +// Required +sinkProperties.put(AWSConfigConstants.AWS_REGION, "eu-west-1") +// Optional, provide via alternative routes e.g. environment variables +sinkProperties.put(AWSConfigConstants.AWS_ACCESS_KEY_ID, "aws_access_key_id") +sinkProperties.put(AWSConfigConstants.AWS_SECRET_ACCESS_KEY, "aws_secret_access_key") + +val SqsSink<String> sqsSink = + SqsSink.<String>builder() + .setSerializationSchema(new SimpleStringSchema()) // Required + .setSqsUrl("https://sqs.us-east-1.amazonaws.com/xxxx/test-sqs") // Required + .setSqsClientProperties(sinkProperties) // Required + .setFailOnError(false) // Optional + .setMaxBatchSize(10) // Optional + .setMaxInFlightRequests(50) // Optional + .setMaxBufferedRequests(1_000) // Optional + .setMaxBatchSizeInBytes(256 * 1024) // Optional + .setMaxTimeInBufferMS(5000) // Optional + .setMaxRecordSizeInBytes(256 * 1024) // Optional + .build(); + + +flinkStream.sinkTo(sqsSink) +``` +{{< /tab >}} +{{< /tabs >}} + +## Configurations + +Flink's SQS sink is created by using the static builder `SqsSink.<String>builder()`. + +1. __setSqsClientProperties(Properties sinkProperties)__ + * Required. + * Supplies credentials, region and other parameters to the SQS client. +2. __setSerializationSchema(SerializationSchema<InputType> serializationSchema)__ + * Required. + * Supplies a serialization schema to the Sink. This schema is used to serialize elements before sending to SQS. +3. __setSqsUrl(String sqsUrl)__ + * Required. + * Url of the SQS to sink to. +4. _setFailOnError(boolean failOnError)_ + * Optional. Default: `false`. + * Whether failed requests to write records to SQS are treated as fatal exceptions in the sink. Review Comment: nit: Might be a better description if we say "causes a Flink job restart" ########## flink-connector-aws/flink-connector-sqs/src/main/java/org.apache.flink.connector.sqs/sink/SqsSinkBuilder.java: ########## @@ -0,0 +1,161 @@ +/* + * 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.sqs.sink; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.base.sink.AsyncSinkBaseBuilder; + +import software.amazon.awssdk.http.Protocol; +import software.amazon.awssdk.services.sqs.model.SendMessageBatchRequestEntry; + +import java.util.Optional; +import java.util.Properties; + +import static org.apache.flink.connector.aws.config.AWSConfigConstants.HTTP_PROTOCOL_VERSION; +import static software.amazon.awssdk.http.Protocol.HTTP1_1; + +/** + * Builder to construct {@link SqsSink}. + * + * <p>The following example shows the minimum setup to create a {@link SqsSink} that + * writes String values to a SQS named sqsUrl. + * + * <pre>{@code + * Properties sinkProperties = new Properties(); + * sinkProperties.put(AWSConfigConstants.AWS_REGION, "eu-west-1"); + * + * SqsSink<String> sqsSink = + * SqsSink.<String>builder() + * .setElementConverter(elementConverter) + * .setSqsUrl("sqsUrl") + * .setSqsClientProperties(sinkProperties) + * .setSerializationSchema(new SimpleStringSchema()) + * .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 10 + * <li>{@code maxInFlightRequests} will be 50 + * <li>{@code maxBufferedRequests} will be 5000 + * <li>{@code maxBatchSizeInBytes} will be 256 KB i.e. {@code 256 * 1000} + * <li>{@code maxTimeInBufferMs} will be 5000ms + * <li>{@code maxRecordSizeInBytes} will be 256 KB i.e. {@code 256 * 1000} + * <li>{@code failOnError} will be false + * </ul> + * + * @param <InputT> type of elements that should be persisted in the destination + */ +@PublicEvolving +public class SqsSinkBuilder<InputT> + extends AsyncSinkBaseBuilder<InputT, SendMessageBatchRequestEntry, SqsSinkBuilder<InputT>> { + + private static final int DEFAULT_MAX_BATCH_SIZE = 10; Review Comment: Do we know if request will be rejected if batch contains > 10 messages? If so, can we add a validation on the builder to ensure customer doesn't set max > 10? ########## flink-connector-aws-e2e-tests/flink-connector-aws-sqs-e2e-tests/pom.xml: ########## @@ -0,0 +1,122 @@ +<?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/xsd/maven-4.0.0.xsd"> + + <parent> + <artifactId>flink-connector-aws-e2e-tests-parent</artifactId> + <groupId>org.apache.flink</groupId> + <version>4.3-SNAPSHOT</version> + </parent> + + <modelVersion>4.0.0</modelVersion> + + <artifactId>flink-connector-aws-sqs-e2e-tests</artifactId> + <name>Flink : Connectors : AWS : E2E Tests : Amazon SQS</name> + <packaging>jar</packaging> + + <dependencies> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-streaming-java</artifactId> + <version>${flink.version}</version> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-connector-sqs</artifactId> + <version>${project.version}</version> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-connector-aws-base</artifactId> + <version>${project.version}</version> + <scope>test</scope> + <type>test-jar</type> + <exclusions> + <exclusion> + <groupId>com.typesafe.netty</groupId> + <artifactId>netty-reactive-streams-http</artifactId> + </exclusion> + </exclusions> + </dependency> + + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-connector-sqs</artifactId> + <version>${project.version}</version> + <scope>test</scope> + <type>test-jar</type> + <exclusions> + <exclusion> + <groupId>com.typesafe.netty</groupId> + <artifactId>netty-reactive-streams-http</artifactId> + </exclusion> + </exclusions> Review Comment: Why do we need this exclusion? ########## flink-connector-aws/flink-connector-sqs/src/main/java/org.apache.flink.connector.sqs/sink/SqsSinkWriter.java: ########## @@ -0,0 +1,257 @@ +/* + * 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.sqs.sink; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.connector.aws.sink.throwable.AWSExceptionHandler; +import org.apache.flink.connector.aws.util.AWSClientUtil; +import org.apache.flink.connector.aws.util.AWSGeneralUtil; +import org.apache.flink.connector.base.sink.throwable.FatalExceptionClassifier; +import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter; +import org.apache.flink.connector.base.sink.writer.BufferedRequestState; +import org.apache.flink.connector.base.sink.writer.ElementConverter; +import org.apache.flink.connector.base.sink.writer.config.AsyncSinkWriterConfiguration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.http.async.SdkAsyncHttpClient; +import software.amazon.awssdk.services.sqs.SqsAsyncClient; +import software.amazon.awssdk.services.sqs.model.BatchResultErrorEntry; +import software.amazon.awssdk.services.sqs.model.SendMessageBatchRequest; +import software.amazon.awssdk.services.sqs.model.SendMessageBatchRequestEntry; +import software.amazon.awssdk.services.sqs.model.SendMessageBatchResponse; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; + +import static org.apache.flink.connector.aws.util.AWSCredentialFatalExceptionClassifiers.getInvalidCredentialsExceptionClassifier; +import static org.apache.flink.connector.aws.util.AWSCredentialFatalExceptionClassifiers.getSdkClientMisconfiguredExceptionClassifier; +import static org.apache.flink.connector.base.sink.writer.AsyncSinkFatalExceptionClassifiers.getInterruptedExceptionClassifier; + +/** + * Sink writer created by {@link SqsSink} to write to SQS. More details on the operation of this + * sink writer may be found in the doc for {@link SqsSink}. More details on the internals of this + * sink writer may be found in {@link AsyncSinkWriter}. + * + * <p>The {@link SqsAsyncClient} 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 +class SqsSinkWriter<InputT> extends AsyncSinkWriter<InputT, SendMessageBatchRequestEntry> { + + private static final Logger LOG = LoggerFactory.getLogger(SqsSinkWriter.class); + + private static SdkAsyncHttpClient createHttpClient(Properties sqsClientProperties) { + return AWSGeneralUtil.createAsyncHttpClient(sqsClientProperties); + } + + private static SqsAsyncClient createSqsClient( + Properties sqsClientProperties, SdkAsyncHttpClient httpClient) { + AWSGeneralUtil.validateAwsCredentials(sqsClientProperties); + return AWSClientUtil.createAwsAsyncClient( + sqsClientProperties, + httpClient, + SqsAsyncClient.builder(), + SqsConfigConstants.BASE_SQS_USER_AGENT_PREFIX_FORMAT, + SqsConfigConstants.SQS_CLIENT_USER_AGENT_PREFIX); + } + + private static final AWSExceptionHandler SQS_EXCEPTION_HANDLER = + AWSExceptionHandler.withClassifier( + FatalExceptionClassifier.createChain( + getInterruptedExceptionClassifier(), + getInvalidCredentialsExceptionClassifier(), + SqsExceptionClassifiers.getResourceNotFoundExceptionClassifier(), + SqsExceptionClassifiers.getAccessDeniedExceptionClassifier(), + SqsExceptionClassifiers.getNotAuthorizedExceptionClassifier(), + getSdkClientMisconfiguredExceptionClassifier())); + + private final Counter numRecordsOutErrorsCounter; + + /* Url of SQS */ + private final String sqsUrl; + + /* The sink writer metric group */ + private final SinkWriterMetricGroup metrics; + + /* The asynchronous http client */ + private SdkAsyncHttpClient httpClient; + + /* The asynchronous SQS client */ + private SqsAsyncClient sqsAsyncClient; + + /* Flag to whether fatally fail any time we encounter an exception when persisting records */ + private final boolean failOnError; + + SqsSinkWriter( + ElementConverter<InputT, SendMessageBatchRequestEntry> elementConverter, + Sink.InitContext context, + int maxBatchSize, + int maxInFlightRequests, + int maxBufferedRequests, + long maxBatchSizeInBytes, + long maxTimeInBufferMS, + long maxRecordSizeInBytes, + boolean failOnError, + String sqsUrl, + Properties sqsClientProperties, + Collection<BufferedRequestState<SendMessageBatchRequestEntry>> initialStates) { + super( + elementConverter, + context, + AsyncSinkWriterConfiguration.builder() + .setMaxBatchSize(maxBatchSize) + .setMaxBatchSizeInBytes(maxBatchSizeInBytes) + .setMaxInFlightRequests(maxInFlightRequests) + .setMaxBufferedRequests(maxBufferedRequests) + .setMaxTimeInBufferMS(maxTimeInBufferMS) + .setMaxRecordSizeInBytes(maxRecordSizeInBytes) + .build(), + initialStates); + this.failOnError = failOnError; + this.sqsUrl = sqsUrl; + this.metrics = context.metricGroup(); + this.numRecordsOutErrorsCounter = metrics.getNumRecordsOutErrorsCounter(); + this.httpClient = createHttpClient(sqsClientProperties); + this.sqsAsyncClient = createSqsClient(sqsClientProperties, httpClient); + } + + @Override + protected void submitRequestEntries( + List<SendMessageBatchRequestEntry> requestEntries, + Consumer<List<SendMessageBatchRequestEntry>> requestResult) { + + final SendMessageBatchRequest batchRequest = + SendMessageBatchRequest.builder().entries(requestEntries).queueUrl(sqsUrl).build(); + + CompletableFuture<SendMessageBatchResponse> future = + sqsAsyncClient.sendMessageBatch(batchRequest); + + future.whenComplete( + (response, err) -> { Review Comment: The SQS documentation for `sendMessageBatch` is a little bit worrying... https://docs.aws.amazon.com/AWSSimpleQueueService/latest/APIReference/API_SendMessageBatch.html > The result of sending each message is reported individually in the response. Because the batch request can result in a combination of successful and unsuccessful actions, you should check for batch errors even when the call returns an HTTP status code of 200. Have we verified that if some messages in batch fails, we will get an `err` here? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org