[
https://issues.apache.org/jira/browse/BEAM-11907?focusedWorklogId=598402&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-598402
]
ASF GitHub Bot logged work on BEAM-11907:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 18/May/21 02:57
Start Date: 18/May/21 02:57
Worklog Time Spent: 10m
Work Description: dennisylyung commented on a change in pull request
#14244:
URL: https://github.com/apache/beam/pull/14244#discussion_r634003803
##########
File path:
sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsUnboundedReader.java
##########
@@ -17,51 +17,385 @@
*/
package org.apache.beam.sdk.io.aws.sqs;
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Collectors.toMap;
+import static
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
+
+import com.amazonaws.services.sqs.AmazonSQS;
+import com.amazonaws.services.sqs.model.BatchResultErrorEntry;
+import
com.amazonaws.services.sqs.model.ChangeMessageVisibilityBatchRequestEntry;
+import com.amazonaws.services.sqs.model.ChangeMessageVisibilityBatchResult;
+import com.amazonaws.services.sqs.model.DeleteMessageBatchRequestEntry;
+import com.amazonaws.services.sqs.model.DeleteMessageBatchResult;
+import com.amazonaws.services.sqs.model.GetQueueAttributesRequest;
import com.amazonaws.services.sqs.model.Message;
+import com.amazonaws.services.sqs.model.MessageAttributeValue;
import com.amazonaws.services.sqs.model.MessageSystemAttributeName;
+import com.amazonaws.services.sqs.model.QueueAttributeName;
import com.amazonaws.services.sqs.model.ReceiveMessageRequest;
import com.amazonaws.services.sqs.model.ReceiveMessageResult;
-import java.io.Serializable;
-import java.nio.charset.StandardCharsets;
+import java.io.IOException;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Arrays;
-import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
import java.util.List;
+import java.util.Map;
import java.util.NoSuchElementException;
+import java.util.Objects;
import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
import org.apache.beam.sdk.io.UnboundedSource;
import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Sum;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.util.BucketingFunction;
+import org.apache.beam.sdk.util.MovingFunction;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.EvictingQueue;
+import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
+import org.joda.time.Duration;
import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
-class SqsUnboundedReader extends UnboundedSource.UnboundedReader<Message>
implements Serializable {
+class SqsUnboundedReader extends UnboundedSource.UnboundedReader<Message> {
+ private static final Logger LOG =
LoggerFactory.getLogger(SqsUnboundedReader.class);
+ /** Maximum number of messages to pull from SQS per request. */
public static final int MAX_NUMBER_OF_MESSAGES = 10;
+
+ /** Maximum times to retry batch SQS operations upon partial success. */
+ private static final int BATCH_OPERATION_MAX_RETIRES = 5;
Review comment:
To be honest most are copied from the pubsub connector, except the
update and delete batch size of 10, which is the limit of the AWS API.
--
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.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 598402)
Time Spent: 7.5h (was: 7h 20m)
> SqsIO checkpoint causes throttling
> ----------------------------------
>
> Key: BEAM-11907
> URL: https://issues.apache.org/jira/browse/BEAM-11907
> Project: Beam
> Issue Type: Bug
> Components: io-java-aws
> Affects Versions: 2.27.0
> Reporter: Dennis Yung
> Assignee: Dennis Yung
> Priority: P2
> Time Spent: 7.5h
> Remaining Estimate: 0h
>
> SqsIO currently calls the delete message API for each message when finalizing
> checkpoints.
> This creates large amount of requests to the AWS API, which are frequently
> throttled and t throw the exception:
> {code:java}
> Exception in thread "main"
> com.amazonaws.services.sqs.model.AmazonSQSException: Request is throttled.
> (Service: AmazonSQS; Status Code: 403; Error Code: RequestThrottled; Request
> ID: XXXXXXXXXXXXX) at
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1742)
> at
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1371)
> at
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1347)
> at
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1127)
> at
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:784)
> at
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:752)
> at
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:726)
> at
> com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:686)
> at
> com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:668)
> at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:532) at
> com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:512) at
> com.amazonaws.services.sqs.AmazonSQSClient.doInvoke(AmazonSQSClient.java:2207)
> at
> com.amazonaws.services.sqs.AmazonSQSClient.invoke(AmazonSQSClient.java:2174)
> at
> com.amazonaws.services.sqs.AmazonSQSClient.invoke(AmazonSQSClient.java:2163)
> at
> com.amazonaws.services.sqs.AmazonSQSClient.executeDeleteMessage(AmazonSQSClient.java:893)
> at
> com.amazonaws.services.sqs.AmazonSQSClient.deleteMessage(AmazonSQSClient.java:865)
> at
> com.amazonaws.services.sqs.AmazonSQSClient.deleteMessage(AmazonSQSClient.java:905)
> at
> org.apache.beam.sdk.io.aws.sqs.SqsUnboundedReader.delete(SqsUnboundedReader.java:131)
> at
> org.apache.beam.sdk.io.aws.sqs.SqsCheckpointMark.lambda$finalizeCheckpoint$0(SqsCheckpointMark.java:43)
> at java.util.Optional.ifPresent(Optional.java:159) at
> org.apache.beam.sdk.io.aws.sqs.SqsCheckpointMark.finalizeCheckpoint(SqsCheckpointMark.java:43)
> at
> org.apache.beam.runners.direct.UnboundedReadEvaluatorFactory$UnboundedReadEvaluator.finishRead(UnboundedReadEvaluatorFactory.java:254)
> at
> org.apache.beam.runners.direct.UnboundedReadEvaluatorFactory$UnboundedReadEvaluator.processElement(UnboundedReadEvaluatorFactory.java:147)
> at
> org.apache.beam.runners.direct.DirectTransformExecutor.processElements(DirectTransformExecutor.java:160)
> at
> org.apache.beam.runners.direct.DirectTransformExecutor.run(DirectTransformExecutor.java:124)
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266) at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748){code}
--
This message was sent by Atlassian Jira
(v8.3.4#803005)