[ 
https://issues.apache.org/jira/browse/BEAM-8382?focusedWorklogId=329249&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-329249
 ]

ASF GitHub Bot logged work on BEAM-8382:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 16/Oct/19 16:46
            Start Date: 16/Oct/19 16:46
    Worklog Time Spent: 10m 
      Work Description: aromanenko-dev commented on issue #9765: [BEAM-8382] 
Add polling interval to KinesisIO.Read
URL: https://github.com/apache/beam/pull/9765#issuecomment-542790436
 
 
   @jfarr I agree that there is no perfect solution for that. In the same time, 
I don't see too much difference between hardcoded value and configured one 
taking into account that Kinesis is unbounded source and user pipeline has to 
run for a long time without interruption. So, what's happened if we set initial 
polling interval too small or too big? Would we need to stop a pipeline, set 
new value and restart it again?
    
   Unfortunately, there is no guarantee that this value will be efficient and 
work fine all the way since, as you said above, other client can start consume 
from the same shard in parallel. So, in this case it would better to provide 
adaptive solution. 
   
   Even if we should expose some knobs to user, so what's about using 
`FluentBackoff` [1] instead? It can be configured with retry configuration 
object, like we do in `SolrIO` and some other IOs with `RetryConfiguration`. 
   
   Other option could be to allow to provide UDF object which will manage 
polling delay dynamically (for example, `SerializableFunction`).
   
   Wdyt?
   
   [1] 
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java
 
----------------------------------------------------------------
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: 329249)
    Time Spent: 1h 40m  (was: 1.5h)

> 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 40m
>  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)

Reply via email to