[
https://issues.apache.org/jira/browse/BEAM-8382?focusedWorklogId=330177&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-330177
]
ASF GitHub Bot logged work on BEAM-8382:
----------------------------------------
Author: ASF GitHub Bot
Created on: 17/Oct/19 22:56
Start Date: 17/Oct/19 22:56
Worklog Time Spent: 10m
Work Description: jfarr commented on issue #9765: [BEAM-8382] Add polling
interval to KinesisIO.Read
URL: https://github.com/apache/beam/pull/9765#issuecomment-543396587
@aromanenko-dev To me FluentBackoff seems like the wrong way to deal with
rate limiting. I would rather account for the rate limit up front than try to
react to it as if it were an error. I think the end result is going to be once
every second you'll get a few successful getRecords calls off then spend the
rest of that second being throttled. Possibly longer because as the AWS
documentation states, "Too many of these exceptions can result in exponential
back-offs and thereby cause significant unexpected latencies in processing."
They also state that they recommend a 1 second polling interval because,
"processing larger batches of data tends to be more efficient at reducing
network and other downstream latencies in your application," so polling as fast
as possible then getting throttled will not be as efficient according to AWS.
You have a good point about changing the polling interval while the pipeline
is running. A UDF sounds like it could be a good solution for that. We do not
have that use case though. We know ahead of time how many consumers our stream
will have and what their polling intervals will be. That would not change
unless our architecture changes.
If it's reasonable to allow configuring a backoff strategy why would it not
be reasonable to allow configuring a polling interval? I thought your goal was
to not introduce any new knobs. If that's the case honestly I would rather have
a hard-coded 1 second polling interval.
----------------------------------------------------------------
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: 330177)
Time Spent: 1h 50m (was: 1h 40m)
> Add polling interval to KinesisIO.Read
> --------------------------------------
>
> Key: BEAM-8382
> URL: https://issues.apache.org/jira/browse/BEAM-8382
> Project: Beam
> Issue Type: Improvement
> Components: io-java-kinesis
> Affects Versions: 2.13.0, 2.14.0, 2.15.0
> Reporter: Jonothan Farr
> Assignee: Jonothan Farr
> Priority: Major
> Time Spent: 1h 50m
> Remaining Estimate: 0h
>
> With the current implementation we are observing Kinesis throttling due to
> ReadProvisionedThroughputExceeded on the order of hundreds of times per
> second, regardless of the actual Kinesis throughput. This is because the
> ShardReadersPool readLoop() method is polling getRecords() as fast as
> possible.
> From the KDS documentation:
> {quote}Each shard can support up to five read transactions per second.
> {quote}
> and
> {quote}For best results, sleep for at least 1 second (1,000 milliseconds)
> between calls to getRecords to avoid exceeding the limit on getRecords
> frequency.
> {quote}
> [https://docs.aws.amazon.com/streams/latest/dev/service-sizes-and-limits.html]
> [https://docs.aws.amazon.com/streams/latest/dev/developing-consumers-with-sdk.html]
--
This message was sent by Atlassian Jira
(v8.3.4#803005)