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

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

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

 ##########
 File path: 
sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RateLimitPolicyFactory.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.beam.sdk.io.kinesis;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+import org.apache.beam.sdk.util.BackOff;
+import org.apache.beam.sdk.util.FluentBackoff;
+import org.joda.time.Duration;
+
+/**
+ * Implement this interface to create a {@code RateLimitPolicy}. Used to 
create a rate limiter for
+ * each shard.
+ */
+public interface RateLimitPolicyFactory extends Serializable {
+
+  RateLimitPolicy getRateLimitPolicy();
+
+  static RateLimitPolicyFactory withoutLimiter() {
+    return () -> new RateLimitPolicy() {};
+  }
+
+  static RateLimitPolicyFactory withBackoff(FluentBackoff fluentBackoff) {
+    return () -> new BackoffRateLimiter(fluentBackoff);
+  }
+
+  static RateLimitPolicyFactory withFixedDelay() {
+    return FixedDelayRateLimiter::new;
+  }
+
+  static RateLimitPolicyFactory withFixedDelay(Duration delay) {
+    return () -> new FixedDelayRateLimiter(delay);
+  }
+
+  class BackoffRateLimiter implements RateLimitPolicy {
+
+    private final BackOff backoff;
+
+    public BackoffRateLimiter(FluentBackoff fluentBackoff) {
+      this.backoff =
+          fluentBackoff
+              // never stop retrying
+              .withMaxRetries(Integer.MAX_VALUE)
+              .withMaxCumulativeBackoff(Duration.standardDays(1000))
+              .backoff();
+    }
+
+    @Override
+    public void onThrottle(KinesisClientThrottledException t) throws 
InterruptedException {
+      try {
+        long backOffMillis = backoff.nextBackOffMillis();
+        if (backOffMillis != BackOff.STOP) {
+          Thread.sleep(backOffMillis);
+        }
+      } catch (IOException e) {
+        // do nothing
+      }
+    }
+
+    @Override
+    public void onSuccess(List<KinesisRecord> records) {
 
 Review comment:
   This onSuccess is called from ShardReadersPool and pass in list of records, 
but I don't see any implementations of the interface use this param?
 
----------------------------------------------------------------
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: 338939)
    Time Spent: 7h  (was: 6h 50m)

> 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: 7h
>  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