vahmed-hamdy commented on code in PR #141: URL: https://github.com/apache/flink-connector-aws/pull/141#discussion_r1597498488
########## flink-connector-aws/pom.xml: ########## @@ -18,8 +18,8 @@ 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" Review Comment: change not needed ########## 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) Review Comment: This setter in the java doc doesn't exist. ########## flink-connector-aws/flink-connector-sqs/src/test/java/org.apache.flink/connector.sqs/sink/SqsExceptionClassifiersTest.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.sqs.sink; + +import org.apache.flink.connector.base.sink.throwable.FatalExceptionClassifier; + +import org.junit.jupiter.api.Test; +import software.amazon.awssdk.awscore.exception.AwsErrorDetails; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.services.sqs.model.ResourceNotFoundException; +import software.amazon.awssdk.services.sqs.model.SqsException; + +import static org.junit.jupiter.api.Assertions.assertFalse; + +/** Unit tests for {@link SqsExceptionClassifiers}. */ +public class SqsExceptionClassifiersTest { + + private final FatalExceptionClassifier classifier = + FatalExceptionClassifier.createChain( + SqsExceptionClassifiers.getAccessDeniedExceptionClassifier(), + SqsExceptionClassifiers.getNotAuthorizedExceptionClassifier(), + SqsExceptionClassifiers.getResourceNotFoundExceptionClassifier()); + + @Test + public void shouldClassifyNotAuthorizedAsFatal() { + AwsServiceException sqsException = + SqsException.builder() + .awsErrorDetails( + AwsErrorDetails.builder().errorCode("NotAuthorized").build()) + .build(); + + // isFatal returns `true` if an exception is non-fatal + assertFalse(classifier.isFatal(sqsException, ex -> {})); + } + + @Test + public void shouldClassifyAccessDeniedExceptionAsFatal() { + AwsServiceException sqsException = + SqsException.builder() + .awsErrorDetails( + AwsErrorDetails.builder() + .errorCode("AccessDeniedException") + .build()) + .build(); + + // isFatal returns `true` if an exception is non-fatal + assertFalse(classifier.isFatal(sqsException, ex -> {})); Review Comment: why `assertFalse` ? ########## flink-connector-aws/flink-connector-sqs/archunit-violations/stored.rules: ########## @@ -0,0 +1,4 @@ +# +#Tue Feb 22 12:19:26 CET 2022 +Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=54da9a7d-14d2-4632-a045-1dd8fc665c8f Review Comment: yes this is part of arch tests build ########## flink-connector-aws/flink-connector-sqs/src/main/java/org.apache.flink.connector.sqs/sink/SqsSinkWriter.java: ########## @@ -0,0 +1,272 @@ +/* + * 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.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.io.UnsupportedEncodingException; +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 final SdkAsyncHttpClient httpClient; + + /* The asynchronous SQS client */ + private final SqsAsyncClient sqsAsyncClient; + + /* Flag to whether fatally fail any time we encounter an exception when persisting records */ + private final boolean failOnError; + + SqsSinkWriter( Review Comment: nit: this constructor is never used ########## flink-connector-aws/flink-connector-sqs/src/test/java/org.apache.flink/connector.sqs/sink/SqsExceptionClassifiersTest.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.sqs.sink; + +import org.apache.flink.connector.base.sink.throwable.FatalExceptionClassifier; + +import org.junit.jupiter.api.Test; +import software.amazon.awssdk.awscore.exception.AwsErrorDetails; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.services.sqs.model.ResourceNotFoundException; +import software.amazon.awssdk.services.sqs.model.SqsException; + +import static org.junit.jupiter.api.Assertions.assertFalse; + +/** Unit tests for {@link SqsExceptionClassifiers}. */ +public class SqsExceptionClassifiersTest { + + private final FatalExceptionClassifier classifier = + FatalExceptionClassifier.createChain( + SqsExceptionClassifiers.getAccessDeniedExceptionClassifier(), + SqsExceptionClassifiers.getNotAuthorizedExceptionClassifier(), + SqsExceptionClassifiers.getResourceNotFoundExceptionClassifier()); + + @Test + public void shouldClassifyNotAuthorizedAsFatal() { + AwsServiceException sqsException = + SqsException.builder() + .awsErrorDetails( + AwsErrorDetails.builder().errorCode("NotAuthorized").build()) + .build(); + + // isFatal returns `true` if an exception is non-fatal + assertFalse(classifier.isFatal(sqsException, ex -> {})); + } + + @Test + public void shouldClassifyAccessDeniedExceptionAsFatal() { + AwsServiceException sqsException = + SqsException.builder() + .awsErrorDetails( + AwsErrorDetails.builder() + .errorCode("AccessDeniedException") + .build()) + .build(); + + // isFatal returns `true` if an exception is non-fatal + assertFalse(classifier.isFatal(sqsException, ex -> {})); + } + + @Test + public void shouldClassifyResourceNotFoundAsFatal() { + AwsServiceException sqsException = ResourceNotFoundException.builder().build(); + + // isFatal returns `true` if an exception is non-fatal + assertFalse(classifier.isFatal(sqsException, ex -> {})); Review Comment: why `assertFalse` ########## flink-connector-aws/flink-connector-sqs/src/test/java/org.apache.flink/connector.sqs/sink/SqsExceptionClassifiersTest.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.sqs.sink; + +import org.apache.flink.connector.base.sink.throwable.FatalExceptionClassifier; + +import org.junit.jupiter.api.Test; +import software.amazon.awssdk.awscore.exception.AwsErrorDetails; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.services.sqs.model.ResourceNotFoundException; +import software.amazon.awssdk.services.sqs.model.SqsException; + +import static org.junit.jupiter.api.Assertions.assertFalse; + +/** Unit tests for {@link SqsExceptionClassifiers}. */ +public class SqsExceptionClassifiersTest { + + private final FatalExceptionClassifier classifier = + FatalExceptionClassifier.createChain( + SqsExceptionClassifiers.getAccessDeniedExceptionClassifier(), + SqsExceptionClassifiers.getNotAuthorizedExceptionClassifier(), + SqsExceptionClassifiers.getResourceNotFoundExceptionClassifier()); + + @Test + public void shouldClassifyNotAuthorizedAsFatal() { + AwsServiceException sqsException = + SqsException.builder() + .awsErrorDetails( + AwsErrorDetails.builder().errorCode("NotAuthorized").build()) + .build(); + + // isFatal returns `true` if an exception is non-fatal + assertFalse(classifier.isFatal(sqsException, ex -> {})); Review Comment: why `assertFalse` ? ########## flink-connector-aws/flink-connector-sqs/src/main/java/org.apache.flink.connector.sqs/sink/SqsStateSerializer.java: ########## @@ -0,0 +1,54 @@ +/* + * 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.connector.base.sink.writer.AsyncSinkWriterStateSerializer; + +import software.amazon.awssdk.services.sqs.model.SendMessageBatchRequestEntry; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +/** SQS implementation {@link AsyncSinkWriterStateSerializer}. */ +@Internal +public class SqsStateSerializer extends AsyncSinkWriterStateSerializer<SendMessageBatchRequestEntry> { + @Override + protected void serializeRequestToStream(final SendMessageBatchRequestEntry request, final DataOutputStream out) + throws IOException + { + out.write(request.messageBody().getBytes(StandardCharsets.UTF_8)); + } + + @Override + protected SendMessageBatchRequestEntry deserializeRequestFromStream(final long requestSize, final DataInputStream in) + throws IOException + { + final byte[] requestData = new byte[(int) requestSize]; + in.read(requestData); + return SendMessageBatchRequestEntry.builder().messageBody(new String(requestData, StandardCharsets.UTF_8)).build(); Review Comment: It seems that request Id is used for requeuing failed entries. Do we need to generate a random Id here as well? If so could we add tests for it? ########## flink-connector-aws/flink-connector-sqs/src/main/java/org.apache.flink.connector.sqs/sink/SqsSinkWriter.java: ########## @@ -0,0 +1,272 @@ +/* + * 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.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.io.UnsupportedEncodingException; +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 final SdkAsyncHttpClient httpClient; + + /* The asynchronous SQS client */ + private final 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) { + this( + elementConverter, + context, + maxBatchSize, + maxInFlightRequests, + maxBufferedRequests, + maxBatchSizeInBytes, + maxTimeInBufferMS, + maxRecordSizeInBytes, + failOnError, + sqsUrl, + sqsClientProperties, + Collections.emptyList()); + } + + 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) { + try { + return requestEntry.messageBody().getBytes("UTF-8").length; Review Comment: nit: ```suggestion return requestEntry.messageBody().getBytes(StandardCharsets.UTF_8).length; ``` ########## flink-connector-aws/flink-connector-sqs/src/test/java/org.apache.flink/connector.sqs/sink/SqsExceptionClassifiersTest.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.sqs.sink; + +import org.apache.flink.connector.base.sink.throwable.FatalExceptionClassifier; + +import org.junit.jupiter.api.Test; +import software.amazon.awssdk.awscore.exception.AwsErrorDetails; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.services.sqs.model.ResourceNotFoundException; +import software.amazon.awssdk.services.sqs.model.SqsException; + +import static org.junit.jupiter.api.Assertions.assertFalse; + +/** Unit tests for {@link SqsExceptionClassifiers}. */ +public class SqsExceptionClassifiersTest { + + private final FatalExceptionClassifier classifier = + FatalExceptionClassifier.createChain( + SqsExceptionClassifiers.getAccessDeniedExceptionClassifier(), + SqsExceptionClassifiers.getNotAuthorizedExceptionClassifier(), + SqsExceptionClassifiers.getResourceNotFoundExceptionClassifier()); + + @Test + public void shouldClassifyNotAuthorizedAsFatal() { + AwsServiceException sqsException = + SqsException.builder() + .awsErrorDetails( + AwsErrorDetails.builder().errorCode("NotAuthorized").build()) + .build(); + + // isFatal returns `true` if an exception is non-fatal Review Comment: this comment is confusing ########## flink-connector-aws/flink-connector-sqs/src/main/java/org.apache.flink.connector.sqs/sink/SqsSinkWriter.java: ########## @@ -0,0 +1,272 @@ +/* + * 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.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.io.UnsupportedEncodingException; +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 final SdkAsyncHttpClient httpClient; + + /* The asynchronous SQS client */ + private final 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) { + this( + elementConverter, + context, + maxBatchSize, + maxInFlightRequests, + maxBufferedRequests, + maxBatchSizeInBytes, + maxTimeInBufferMS, + maxRecordSizeInBytes, + failOnError, + sqsUrl, + sqsClientProperties, + Collections.emptyList()); + } + + 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) { + try { + return requestEntry.messageBody().getBytes("UTF-8").length; + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + + @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.debug("handlePartiallyFailedRequest: SQS Sink failed to write and will retry {} entries to SQS", Review Comment: Could we change this to `warn`, we have experienced issues due to silent failures similar to [this](https://issues.apache.org/jira/browse/FLINK-35269) ########## flink-connector-aws/flink-connector-sqs/src/main/java/org.apache.flink.connector.sqs/sink/SqsSinkWriter.java: ########## @@ -0,0 +1,272 @@ +/* + * 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.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.io.UnsupportedEncodingException; +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 = Review Comment: could we add tests for this integration? ########## flink-connector-aws/flink-connector-sqs/src/test/java/org.apache.flink/connector.sqs/sink/SqsExceptionClassifiersTest.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.sqs.sink; + +import org.apache.flink.connector.base.sink.throwable.FatalExceptionClassifier; + +import org.junit.jupiter.api.Test; +import software.amazon.awssdk.awscore.exception.AwsErrorDetails; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.services.sqs.model.ResourceNotFoundException; +import software.amazon.awssdk.services.sqs.model.SqsException; + +import static org.junit.jupiter.api.Assertions.assertFalse; + +/** Unit tests for {@link SqsExceptionClassifiers}. */ +public class SqsExceptionClassifiersTest { Review Comment: could we add tests for retryable exceptions as well ########## flink-connector-aws/flink-connector-sqs/src/test/java/org.apache.flink/connector.sqs/sink/testutils/SqsTestUtils.java: ########## @@ -0,0 +1,76 @@ +/* + * 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.testutils; + +import org.apache.flink.connector.aws.testutils.AWSServicesTestUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; +import com.fasterxml.jackson.datatype.jdk8.Jdk8Module; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import software.amazon.awssdk.http.SdkHttpClient; +import software.amazon.awssdk.services.sqs.SqsClient; +import software.amazon.awssdk.utils.ImmutableMap; + +import java.util.ArrayList; +import java.util.List; + +/** + * A set of static methods that can be used to call common AWS services on the Localstack container. + */ +public class SqsTestUtils { + + private static final ObjectMapper MAPPER = createObjectMapper(); + + public static SqsClient createSqsClient(String endpoint, SdkHttpClient httpClient) { + return AWSServicesTestUtils.createAwsSyncClient(endpoint, httpClient, SqsClient.builder()); + } + + public static DataStream<String> getSampleDataGenerator( + StreamExecutionEnvironment env, int endValue) { + return env.fromSequence(1, endValue) + .map(Object::toString) + .returns(String.class) + .map(data -> MAPPER.writeValueAsString(ImmutableMap.of("data", data))); + } + + public static List<String> getSampleData(int endValue) throws JsonProcessingException { + List<String> expectedElements = new ArrayList<>(); + for (int i = 1; i <= endValue; i++) { + expectedElements.add( + MAPPER.writeValueAsString(ImmutableMap.of("data", String.valueOf(i)))); + } + return expectedElements; + } + + private static ObjectMapper createObjectMapper() { + ObjectMapper objectMapper = new ObjectMapper(); + registerModules(objectMapper); + return objectMapper; + } + + private static void registerModules(ObjectMapper mapper) { + mapper.registerModule(new JavaTimeModule()) Review Comment: why is this needed? ########## flink-connector-aws/flink-connector-sqs/src/test/java/org.apache.flink/connector.sqs/sink/SqsSinkITCase.java: ########## @@ -0,0 +1,121 @@ +/* + * 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.api.common.serialization.SimpleStringSchema; +import org.apache.flink.connector.aws.testutils.AWSServicesTestUtils; +import org.apache.flink.connector.aws.testutils.LocalstackContainer; +import org.apache.flink.connector.sqs.sink.testutils.SqsTestUtils; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.util.DockerImageVersions; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; +import org.testcontainers.utility.DockerImageName; +import software.amazon.awssdk.core.SdkSystemSetting; +import software.amazon.awssdk.http.SdkHttpClient; +import software.amazon.awssdk.services.sqs.SqsClient; +import software.amazon.awssdk.services.sqs.model.Message; +import software.amazon.awssdk.services.sqs.model.ReceiveMessageRequest; + +import java.util.ArrayList; +import java.util.Base64; +import java.util.List; + +import static org.apache.flink.connector.aws.testutils.AWSServicesTestUtils.createConfig; +import static org.apache.flink.connector.sqs.sink.testutils.SqsTestUtils.createSqsClient; +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * Integration test suite for the {@code SqsSink} using a localstack container. + */ +@Testcontainers +@ExtendWith(MiniClusterExtension.class) +class SqsSinkITCase { Review Comment: This is better suited in `flink-connector-aws-e2e-tests` could we move it? ########## flink-connector-aws/flink-connector-sqs/src/main/java/org.apache.flink.connector.sqs/sink/SqsSinkWriter.java: ########## @@ -0,0 +1,272 @@ +/* + * 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.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.io.UnsupportedEncodingException; +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 final SdkAsyncHttpClient httpClient; + + /* The asynchronous SQS client */ + private final 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) { + this( + elementConverter, + context, + maxBatchSize, + maxInFlightRequests, + maxBufferedRequests, + maxBatchSizeInBytes, + maxTimeInBufferMS, + maxRecordSizeInBytes, + failOnError, + sqsUrl, + sqsClientProperties, + Collections.emptyList()); + } + + 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) { + try { + return requestEntry.messageBody().getBytes("UTF-8").length; + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + + @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.debug("handlePartiallyFailedRequest: SQS Sink failed to write and will retry {} entries to SQS", + response.failed().size()); + numRecordsOutErrorsCounter.inc(response.failed().size()); + } + + if (failOnError) { Review Comment: This is not tested ########## flink-connector-aws/flink-connector-sqs/src/main/java/org.apache.flink.connector.sqs/sink/SqsSinkWriter.java: ########## @@ -0,0 +1,272 @@ +/* + * 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.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.io.UnsupportedEncodingException; +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 final SdkAsyncHttpClient httpClient; + + /* The asynchronous SQS client */ + private final 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) { + this( + elementConverter, + context, + maxBatchSize, + maxInFlightRequests, + maxBufferedRequests, + maxBatchSizeInBytes, + maxTimeInBufferMS, + maxRecordSizeInBytes, + failOnError, + sqsUrl, + sqsClientProperties, + Collections.emptyList()); + } + + 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) { + try { + return requestEntry.messageBody().getBytes("UTF-8").length; + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + + @Override + public void close() { + AWSGeneralUtil.closeResources(httpClient, sqsAsyncClient); Review Comment: could we add tests for this? ########## flink-connector-aws/flink-connector-sqs/src/test/java/org.apache.flink/connector.sqs/sink/testutils/SqsTestUtils.java: ########## @@ -0,0 +1,76 @@ +/* + * 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.testutils; + +import org.apache.flink.connector.aws.testutils.AWSServicesTestUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; +import com.fasterxml.jackson.datatype.jdk8.Jdk8Module; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import software.amazon.awssdk.http.SdkHttpClient; +import software.amazon.awssdk.services.sqs.SqsClient; +import software.amazon.awssdk.utils.ImmutableMap; + +import java.util.ArrayList; +import java.util.List; + +/** + * A set of static methods that can be used to call common AWS services on the Localstack container. + */ +public class SqsTestUtils { + + private static final ObjectMapper MAPPER = createObjectMapper(); + + public static SqsClient createSqsClient(String endpoint, SdkHttpClient httpClient) { + return AWSServicesTestUtils.createAwsSyncClient(endpoint, httpClient, SqsClient.builder()); Review Comment: nit: Is there a reason this is async as well? ########## flink-connector-aws-base/src/test/java/org/apache/flink/connector/aws/testutils/AWSServicesTestUtils.java: ########## @@ -114,6 +116,14 @@ public static void createBucket(S3Client s3Client, String bucketName) { } } + public static void createSqs(String sqsName, SqsClient sqsClient) { Review Comment: This shouldn't be in `aws-base` we could move it to `SqsTestUtils` ########## flink-connector-aws/flink-connector-sqs/src/test/java/org.apache.flink/connector.sqs/sink/testutils/SqsTestUtils.java: ########## @@ -0,0 +1,76 @@ +/* + * 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.testutils; + +import org.apache.flink.connector.aws.testutils.AWSServicesTestUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; +import com.fasterxml.jackson.datatype.jdk8.Jdk8Module; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import software.amazon.awssdk.http.SdkHttpClient; +import software.amazon.awssdk.services.sqs.SqsClient; +import software.amazon.awssdk.utils.ImmutableMap; + +import java.util.ArrayList; +import java.util.List; + +/** + * A set of static methods that can be used to call common AWS services on the Localstack container. + */ +public class SqsTestUtils { + + private static final ObjectMapper MAPPER = createObjectMapper(); + + public static SqsClient createSqsClient(String endpoint, SdkHttpClient httpClient) { + return AWSServicesTestUtils.createAwsSyncClient(endpoint, httpClient, SqsClient.builder()); + } + + public static DataStream<String> getSampleDataGenerator( + StreamExecutionEnvironment env, int endValue) { + return env.fromSequence(1, endValue) + .map(Object::toString) + .returns(String.class) + .map(data -> MAPPER.writeValueAsString(ImmutableMap.of("data", data))); + } + + public static List<String> getSampleData(int endValue) throws JsonProcessingException { Review Comment: this is unused ########## flink-connector-aws-base/pom.xml: ########## @@ -76,6 +76,12 @@ under the License. <scope>test</scope> </dependency> + <dependency> Review Comment: This should be moved from base ########## flink-connector-aws/flink-connector-sqs/src/test/java/org.apache.flink/connector.sqs/sink/SqsSinkWriterTest.java: ########## @@ -0,0 +1,105 @@ +/* + * 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.api.common.serialization.SimpleStringSchema; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.connector.aws.testutils.AWSServicesTestUtils; +import org.apache.flink.connector.base.sink.writer.ElementConverter; +import org.apache.flink.connector.base.sink.writer.TestSinkInitContext; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.services.sqs.model.SendMessageBatchRequestEntry; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Properties; +import java.util.concurrent.CompletionException; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + +/** Covers construction, defaults and sanity checking of {@link SqsSinkWriter}. */ +public class SqsSinkWriterTest { + + private SqsSinkWriter<String> sinkWriter; + + private static final ElementConverter<String, SendMessageBatchRequestEntry> ELEMENT_CONVERTER_PLACEHOLDER = + SqsSinkElementConverter.<String>builder() + .setSerializationSchema(new SimpleStringSchema()) + .build(); + + @BeforeEach + void setup() throws IOException { + TestSinkInitContext sinkInitContext = new TestSinkInitContext(); + Properties sinkProperties = AWSServicesTestUtils.createConfig("https://fake_aws_endpoint"); + SqsSink<String> sink = + new SqsSink<>( + ELEMENT_CONVERTER_PLACEHOLDER, + 50, + 16, + 10000, + 4 * 1024 * 1024L, + 5000L, + 1000 * 1024L, + true, + "sqsUrl", + sinkProperties); + sinkWriter = (SqsSinkWriter<String>) sink.createWriter(sinkInitContext); + } + + @Test + void getSizeInBytesReturnsSizeOfBlobBeforeBase64Encoding() { + String testString = "{many hands make light work;"; + SendMessageBatchRequestEntry record = SendMessageBatchRequestEntry.builder().messageBody(testString).build(); + assertThat(sinkWriter.getSizeInBytes(record)) + .isEqualTo(testString.getBytes(StandardCharsets.US_ASCII).length); Review Comment: is this correct? why not `StandardCharsets.UTF_8` ? ########## flink-connector-aws/flink-connector-sqs/src/main/java/org.apache.flink.connector.sqs/sink/SqsSinkWriter.java: ########## @@ -0,0 +1,272 @@ +/* + * 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.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.io.UnsupportedEncodingException; Review Comment: this import is unused could we please run checkstyle and spotless checks and address the issues? ########## flink-connector-aws/flink-connector-sqs/src/main/java/org.apache.flink.connector.sqs/sink/SqsSinkWriter.java: ########## @@ -0,0 +1,272 @@ +/* + * 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.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.io.UnsupportedEncodingException; +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 final SdkAsyncHttpClient httpClient; + + /* The asynchronous SQS client */ + private final 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) { + this( + elementConverter, + context, + maxBatchSize, + maxInFlightRequests, + maxBufferedRequests, + maxBatchSizeInBytes, + maxTimeInBufferMS, + maxRecordSizeInBytes, + failOnError, + sqsUrl, + sqsClientProperties, + Collections.emptyList()); + } + + 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) { + try { + return requestEntry.messageBody().getBytes("UTF-8").length; + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + } + + @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.debug("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) + { + return requestEntries.stream() Review Comment: Could we avoid java streams as per [coding standards](https://flink.apache.org/how-to-contribute/code-style-and-quality-java/#java-streams) ########## 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: Is there a reason for choosing such a low value? -- 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