-
Notifications
You must be signed in to change notification settings - Fork 4.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[BEAM-8382] Add rate limit policy to KinesisIO.Read #9765
Merged
aromanenko-dev
merged 2 commits into
apache:master
from
jfarr:kinesis-io-polling-interval
Mar 3, 2020
Merged
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
28 changes: 28 additions & 0 deletions
28
...kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisClientThrottledException.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,28 @@ | ||
/* | ||
* 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 com.amazonaws.AmazonClientException; | ||
|
||
/** Thrown when the Kinesis client was throttled due to rate limits. */ | ||
class KinesisClientThrottledException extends TransientKinesisException { | ||
|
||
public KinesisClientThrottledException(String s, AmazonClientException e) { | ||
super(s, e); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -40,6 +40,7 @@ | |
import java.util.concurrent.ExecutionException; | ||
import java.util.concurrent.Future; | ||
import java.util.concurrent.LinkedBlockingDeque; | ||
import java.util.function.Supplier; | ||
import javax.annotation.Nullable; | ||
import org.apache.beam.sdk.annotations.Experimental; | ||
import org.apache.beam.sdk.io.Read.Unbounded; | ||
|
@@ -94,7 +95,6 @@ | |
* | ||
* <pre>{@code | ||
* public class MyCustomKinesisClientProvider implements AWSClientsProvider { | ||
* {@literal @}Override | ||
* public AmazonKinesis getKinesisClient() { | ||
* // set up your client here | ||
* } | ||
|
@@ -148,20 +148,17 @@ | |
* this.customWatermarkPolicy = new WatermarkPolicyFactory.CustomWatermarkPolicy(WatermarkParameters.create()); | ||
* } | ||
* | ||
* @Override | ||
* public Instant getWatermark() { | ||
* return customWatermarkPolicy.getWatermark(); | ||
* } | ||
* | ||
* @Override | ||
* public void update(KinesisRecord record) { | ||
* customWatermarkPolicy.update(record); | ||
* } | ||
* } | ||
* | ||
* // custom factory | ||
* class MyCustomPolicyFactory implements WatermarkPolicyFactory { | ||
* @Override | ||
* public WatermarkPolicy createWatermarkPolicy() { | ||
* return new MyCustomPolicy(); | ||
* } | ||
|
@@ -173,6 +170,69 @@ | |
* .withCustomWatermarkPolicy(new MyCustomPolicyFactory()) | ||
* }</pre> | ||
* | ||
* <p>By default Kinesis IO will poll the Kinesis getRecords() API as fast as possible which may | ||
* lead to excessive read throttling. To limit the rate of getRecords() calls you can set a rate | ||
* limit policy. For example, the default fixed delay policy will limit the rate to one API call per | ||
* second per shard: | ||
* | ||
* <pre>{@code | ||
* p.apply(KinesisIO.read() | ||
* .withStreamName("streamName") | ||
* .withInitialPositionInStream(InitialPositionInStream.LATEST) | ||
* .withFixedDelayRateLimitPolicy()) | ||
* }</pre> | ||
* | ||
* <p>You can also use a fixed delay policy with a specified delay interval, for example: | ||
* | ||
* <pre>{@code | ||
* p.apply(KinesisIO.read() | ||
* .withStreamName("streamName") | ||
* .withInitialPositionInStream(InitialPositionInStream.LATEST) | ||
* .withFixedDelayRateLimitPolicy(Duration.millis(500)) | ||
* }</pre> | ||
* | ||
* <p>If you need to change the polling interval of a Kinesis pipeline at runtime, for example to | ||
* compensate for adding and removing additional consumers to the stream, then you can supply the | ||
* delay interval as a function so that you can obtain the current delay interval from some external | ||
* source: | ||
* | ||
* <pre>{@code | ||
* p.apply(KinesisIO.read() | ||
* .withStreamName("streamName") | ||
* .withInitialPositionInStream(InitialPositionInStream.LATEST) | ||
* .withDynamicDelayRateLimitPolicy(() -> Duration.millis(<some delay interval>)) | ||
* }</pre> | ||
* | ||
* <p>Finally, you can create a custom rate limit policy that responds to successful read calls | ||
* and/or read throttling exceptions with your own rate-limiting logic: | ||
* | ||
* <pre>{@code | ||
* // custom policy | ||
* public class MyCustomPolicy implements RateLimitPolicy { | ||
* | ||
* public void onSuccess(List<KinesisRecord> records) throws InterruptedException { | ||
* // handle successful getRecords() call | ||
* } | ||
* | ||
* public void onThrottle(KinesisClientThrottledException e) throws InterruptedException { | ||
* // handle Kinesis read throttling exception | ||
* } | ||
* } | ||
* | ||
* // custom factory | ||
* class MyCustomPolicyFactory implements RateLimitPolicyFactory { | ||
* | ||
* public RateLimitPolicy getRateLimitPolicy() { | ||
* return new MyCustomPolicy(); | ||
* } | ||
* } | ||
* | ||
* p.apply(KinesisIO.read() | ||
* .withStreamName("streamName") | ||
* .withInitialPositionInStream(InitialPositionInStream.LATEST) | ||
* .withCustomRateLimitPolicy(new MyCustomPolicyFactory()) | ||
* }</pre> | ||
* | ||
* <h3>Writing to Kinesis</h3> | ||
* | ||
* <p>Example usage: | ||
|
@@ -239,6 +299,7 @@ public static Read read() { | |
.setMaxNumRecords(Long.MAX_VALUE) | ||
.setUpToDateThreshold(Duration.ZERO) | ||
.setWatermarkPolicyFactory(WatermarkPolicyFactory.withArrivalTimePolicy()) | ||
.setRateLimitPolicyFactory(RateLimitPolicyFactory.withoutLimiter()) | ||
.setMaxCapacityPerShard(ShardReadersPool.DEFAULT_CAPACITY_PER_SHARD) | ||
.build(); | ||
} | ||
|
@@ -273,6 +334,8 @@ public abstract static class Read extends PTransform<PBegin, PCollection<Kinesis | |
|
||
abstract WatermarkPolicyFactory getWatermarkPolicyFactory(); | ||
|
||
abstract RateLimitPolicyFactory getRateLimitPolicyFactory(); | ||
|
||
abstract Integer getMaxCapacityPerShard(); | ||
|
||
abstract Builder toBuilder(); | ||
|
@@ -296,6 +359,8 @@ abstract static class Builder { | |
|
||
abstract Builder setWatermarkPolicyFactory(WatermarkPolicyFactory watermarkPolicyFactory); | ||
|
||
abstract Builder setRateLimitPolicyFactory(RateLimitPolicyFactory rateLimitPolicyFactory); | ||
|
||
abstract Builder setMaxCapacityPerShard(Integer maxCapacity); | ||
|
||
abstract Read build(); | ||
|
@@ -425,6 +490,45 @@ public Read withCustomWatermarkPolicy(WatermarkPolicyFactory watermarkPolicyFact | |
return toBuilder().setWatermarkPolicyFactory(watermarkPolicyFactory).build(); | ||
} | ||
|
||
/** Specifies a fixed delay rate limit policy with the default delay of 1 second. */ | ||
public Read withFixedDelayRateLimitPolicy() { | ||
return toBuilder().setRateLimitPolicyFactory(RateLimitPolicyFactory.withFixedDelay()).build(); | ||
} | ||
|
||
/** | ||
* Specifies a fixed delay rate limit policy with the given delay. | ||
* | ||
* @param delay Denotes the fixed delay duration. | ||
*/ | ||
public Read withFixedDelayRateLimitPolicy(Duration delay) { | ||
checkArgument(delay != null, "delay cannot be null"); | ||
return toBuilder() | ||
.setRateLimitPolicyFactory(RateLimitPolicyFactory.withFixedDelay(delay)) | ||
.build(); | ||
} | ||
|
||
/** | ||
* Specifies a dynamic delay rate limit policy with the given function being called at each | ||
* polling interval to get the next delay value. This can be used to change the polling interval | ||
* of a running pipeline based on some external configuration source, for example. | ||
* | ||
* @param delay The function to invoke to get the next delay duration. | ||
*/ | ||
public Read withDynamicDelayRateLimitPolicy(Supplier<Duration> delay) { | ||
checkArgument(delay != null, "delay cannot be null"); | ||
return toBuilder().setRateLimitPolicyFactory(RateLimitPolicyFactory.withDelay(delay)).build(); | ||
} | ||
|
||
/** | ||
* Specifies the {@code RateLimitPolicyFactory} for a custom rate limiter. | ||
* | ||
* @param rateLimitPolicyFactory Custom rate limit policy factory. | ||
*/ | ||
public Read withCustomRateLimitPolicy(RateLimitPolicyFactory rateLimitPolicyFactory) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please, add an example into KinesisIO javadoc how to use it. |
||
checkArgument(rateLimitPolicyFactory != null, "rateLimitPolicyFactory cannot be null"); | ||
return toBuilder().setRateLimitPolicyFactory(rateLimitPolicyFactory).build(); | ||
} | ||
|
||
/** Specifies the maximum number of messages per one shard. */ | ||
public Read withMaxCapacityPerShard(Integer maxCapacity) { | ||
checkArgument(maxCapacity > 0, "maxCapacity must be positive, but was: %s", maxCapacity); | ||
|
@@ -441,6 +545,7 @@ public PCollection<KinesisRecord> expand(PBegin input) { | |
getInitialPosition(), | ||
getUpToDateThreshold(), | ||
getWatermarkPolicyFactory(), | ||
getRateLimitPolicyFactory(), | ||
getRequestRecordsLimit(), | ||
getMaxCapacityPerShard())); | ||
|
||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
37 changes: 37 additions & 0 deletions
37
sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RateLimitPolicy.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,37 @@ | ||
/* | ||
* 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.util.List; | ||
|
||
public interface RateLimitPolicy { | ||
|
||
/** | ||
* Called after Kinesis records are successfully retrieved. | ||
* | ||
* @param records The list of retrieved records. | ||
*/ | ||
default void onSuccess(List<KinesisRecord> records) throws InterruptedException {} | ||
|
||
/** | ||
* Called after the Kinesis client is throttled. | ||
* | ||
* @param e The {@code KinesisClientThrottledException} thrown by the client. | ||
*/ | ||
default void onThrottle(KinesisClientThrottledException e) throws InterruptedException {} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please, add an example into KinesisIO javadoc how to use it.