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

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

                Author: ASF GitHub Bot
            Created on: 07/Nov/19 01:55
            Start Date: 07/Nov/19 01:55
    Worklog Time Spent: 10m 
      Work Description: jfarr commented on pull request #9765: [WIP][BEAM-8382] 
Add rate limit policy to KinesisIO.Read
URL: https://github.com/apache/beam/pull/9765#discussion_r343418039
 
 

 ##########
 File path: 
sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java
 ##########
 @@ -420,6 +426,47 @@ public Read 
withCustomWatermarkPolicy(WatermarkPolicyFactory watermarkPolicyFact
       return 
toBuilder().setWatermarkPolicyFactory(watermarkPolicyFactory).build();
     }
 
+    /**
+     * Specifies the rate limit policy as BackoffRateLimiter.
+     *
+     * @param fluentBackoff The {@code FluentBackoff} used to create the 
backoff policy.
+     */
+    public Read withBackoffRateLimitPolicy(FluentBackoff fluentBackoff) {
+      checkArgument(fluentBackoff != null, "fluentBackoff cannot be null");
+      return toBuilder()
+          
.setRateLimitPolicyFactory(RateLimitPolicyFactory.withBackoff(fluentBackoff))
+          .build();
+    }
+
+    /**
+     * Specifies the rate limit policy as FixedDelayRateLimiter with the 
default delay of 1 second.
+     */
+    public Read withFixedDelayRateLimitPolicy() {
+      return 
toBuilder().setRateLimitPolicyFactory(RateLimitPolicyFactory.withFixedDelay()).build();
+    }
+
+    /**
+     * Specifies the rate limit policy as FixedDelayRateLimiter with the given 
delay.
+     *
+     * @param delay Denotes the fixed delay duration.
+     */
+    public Read withFixedDelayRateLimitPolicy(Duration delay) {
 
 Review comment:
   > 1. Have we try to use Kinesis's RetryPolicy and BackoffStrategy? This look 
like we reengineer this feature, which already supported by Kinesis's API. Here 
is the example: 
https://www.programcreek.com/java-api-examples/?api=com.amazonaws.retry.RetryPolicy.
 If we already tried it, then what would be the reasons not to use the 
Kinesis's one ?
   
   I'm not sure if RetryPolicy and BackoffStrategy apply to 
LimitExceededException / ProvisionedThroughputExceededException but I can look 
into that. If so I think it makes sense to just configure this in the Kinesis 
client instead of having a BackoffRateLimitPolicy. What do you think 
@aromanenko-dev and @lukecwik ?
 
----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 339686)
    Time Spent: 8h 40m  (was: 8.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: 8h 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