-
Notifications
You must be signed in to change notification settings - Fork 94
[FLINK-39540][Connectors/Kinesis][5.1] Addressed bugs for EFO subscriptions when they are completed #245
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
base: v5.1
Are you sure you want to change the base?
[FLINK-39540][Connectors/Kinesis][5.1] Addressed bugs for EFO subscriptions when they are completed #245
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -80,7 +80,6 @@ public class FanOutKinesisShardSubscription { | |
| // Queue is meant for eager retrieval of records from the Kinesis stream. We will always have 2 | ||
| // record batches available on next read. | ||
| private final BlockingQueue<SubscribeToShardEvent> eventQueue = new LinkedBlockingQueue<>(2); | ||
| private final AtomicBoolean subscriptionActive = new AtomicBoolean(false); | ||
| private final AtomicReference<Throwable> subscriptionException = new AtomicReference<>(); | ||
|
|
||
| // Store the current starting position for this subscription. Will be updated each time new | ||
|
|
@@ -108,8 +107,9 @@ public void activateSubscription() { | |
| shardId, | ||
| startingPosition, | ||
| consumerArn); | ||
| if (subscriptionActive.get()) { | ||
| LOG.warn("Skipping activation of subscription since it is already active."); | ||
| if (shardSubscriber != null | ||
| && shardSubscriber.getSubscriptionState() == SubscriptionState.SUBSCRIBED) { | ||
| LOG.warn("Skipping activation of subscription since it is active & subscribed."); | ||
| return; | ||
| } | ||
|
|
||
|
|
@@ -166,9 +166,9 @@ public void activateSubscription() { | |
| shardId, | ||
| startingPosition, | ||
| consumerArn); | ||
| subscriptionActive.set(true); | ||
| // Request first batch of records. | ||
| shardSubscriber.requestRecords(); | ||
|
|
||
| } else { | ||
| String errorMessage = | ||
| "Timeout when subscribing to shard " | ||
|
|
@@ -236,16 +236,43 @@ public SubscribeToShardEvent nextEvent() { | |
| throw new KinesisStreamsSourceException( | ||
| "Subscription encountered unrecoverable exception.", throwable); | ||
| } | ||
| final SubscriptionState state = | ||
| Optional.ofNullable(shardSubscriber) | ||
| .map(FanOutShardSubscriber::getSubscriptionState) | ||
| .orElse(SubscriptionState.NOT_STARTED); | ||
|
|
||
| if (!subscriptionActive.get()) { | ||
| LOG.debug( | ||
| "Subscription to shard {} for consumer {} is not yet active. Skipping.", | ||
| shardId, | ||
| consumerArn); | ||
| return null; | ||
| switch (state) { | ||
| case NOT_STARTED: | ||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug( | ||
| "Subscription to shard {} for consumer {} is not yet active. Skipping.", | ||
| shardId, | ||
| consumerArn); | ||
| } | ||
| return null; | ||
| case COMPLETED: | ||
| if (shardSubscriber.isShardEndReached()) { | ||
| if (LOG.isInfoEnabled()) { | ||
| LOG.info( | ||
| "Subscription reached SHARD_END for shard {} for consumer {}.", | ||
| shardId, | ||
| consumerArn); | ||
| } | ||
| return null; | ||
| } | ||
| if (LOG.isInfoEnabled()) { | ||
| LOG.info( | ||
| "Subscription expired to shard {} for consumer {}. Restarting.", | ||
| shardId, | ||
| consumerArn); | ||
| } | ||
| activateSubscription(); | ||
| return null; | ||
| case SUBSCRIBED: | ||
| return eventQueue.poll(); | ||
| default: | ||
| throw new IllegalStateException("Unknown subscription state: " + state); | ||
| } | ||
|
|
||
| return eventQueue.poll(); | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -254,26 +281,48 @@ public SubscribeToShardEvent nextEvent() { | |
| */ | ||
| private class FanOutShardSubscriber implements Subscriber<SubscribeToShardEventStream> { | ||
| private final CountDownLatch subscriptionLatch; | ||
|
|
||
| private Subscription subscription; | ||
|
|
||
| private final AtomicReference<SubscriptionState> subscriptionState = | ||
| new AtomicReference<>(SubscriptionState.NOT_STARTED); | ||
| private final AtomicBoolean isShardEnd = new AtomicBoolean(false); | ||
|
|
||
| private FanOutShardSubscriber(CountDownLatch subscriptionLatch) { | ||
| this.subscriptionLatch = subscriptionLatch; | ||
| } | ||
|
|
||
| /** | ||
| * Fetch the state that the subscriber is in. | ||
| * | ||
| * @return Subscription state for the subscriber. | ||
| */ | ||
| public SubscriptionState getSubscriptionState() { | ||
| return subscriptionState.get(); | ||
| } | ||
|
|
||
| /** | ||
| * Boolean whether this subscriber has reached the end of a shard. | ||
| * | ||
| * @return True if ShardEnd. false otherwise. | ||
| */ | ||
| public boolean isShardEndReached() { | ||
| return isShardEnd.get(); | ||
| } | ||
|
|
||
| public void requestRecords() { | ||
| subscription.request(1); | ||
| } | ||
|
|
||
| public void cancel() { | ||
| if (!subscriptionActive.get()) { | ||
| LOG.warn("Trying to cancel inactive subscription. Ignoring."); | ||
| if (this.subscriptionState.get() == SubscriptionState.COMPLETED) { | ||
|
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. change the message to be more accurate?
Contributor
Author
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. The log statement is technically correct in that its canceling a subscription that is no longer active. But ill update the wording to more closely align with the new states |
||
| LOG.warn("Subscription is already completed. Ignoring request to cancel."); | ||
| return; | ||
| } | ||
| subscriptionActive.set(false); | ||
|
|
||
| if (subscription != null) { | ||
| subscription.cancel(); | ||
| } | ||
| this.subscriptionState.set(SubscriptionState.COMPLETED); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -284,6 +333,7 @@ public void onSubscribe(Subscription subscription) { | |
| startingPosition, | ||
| consumerArn); | ||
| this.subscription = subscription; | ||
| this.subscriptionState.set(SubscriptionState.SUBSCRIBED); | ||
| subscriptionLatch.countDown(); | ||
| } | ||
|
|
||
|
|
@@ -300,6 +350,15 @@ public void visit(SubscribeToShardEvent event) { | |
| event); | ||
| eventQueue.put(event); | ||
|
|
||
| if (event.continuationSequenceNumber() == null) { | ||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug("continuationSequenceNumber is null. " + | ||
| "Reached ShardEnd for shard: {}", shardId); | ||
| } | ||
| isShardEnd.set(true); | ||
|
pelaezryan marked this conversation as resolved.
|
||
| return; | ||
| } | ||
|
|
||
| // Update the starting position in case we have to recreate the | ||
| // subscription | ||
| startingPosition = | ||
|
|
@@ -330,8 +389,14 @@ public void onError(Throwable throwable) { | |
| @Override | ||
| public void onComplete() { | ||
| LOG.info("Subscription complete - {} ({})", shardId, consumerArn); | ||
| cancel(); | ||
| activateSubscription(); | ||
| this.subscriptionState.set(SubscriptionState.COMPLETED); | ||
| } | ||
| } | ||
|
|
||
| /** States that the {@code FanOutShardSubscriber} may be in. */ | ||
| private enum SubscriptionState { | ||
| NOT_STARTED, | ||
| SUBSCRIBED, | ||
| COMPLETED | ||
|
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. can there be an error state?
Contributor
Author
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. Not that I am aware of. When any exceptions occur we terminate the subscription (if it's not done so already): |
||
| } | ||
| } | ||
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.
nit: should we change the message. It refers to the subscription being active , which seems to be tied to the subscriptionActive variable we have removed.
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.
Updated wording