Skip to content

[FLINK-39540][Connectors/Kinesis][5.1] Addressed bugs for EFO subscriptions when they are completed#245

Open
pelaezryan wants to merge 2 commits into
apache:v5.1from
pelaezryan:v5.1-efo-fix
Open

[FLINK-39540][Connectors/Kinesis][5.1] Addressed bugs for EFO subscriptions when they are completed#245
pelaezryan wants to merge 2 commits into
apache:v5.1from
pelaezryan:v5.1-efo-fix

Conversation

@pelaezryan
Copy link
Copy Markdown
Contributor

@pelaezryan pelaezryan commented Apr 29, 2026

Purpose of the change

Address a bug in the Kinesis Flink Connector that would cause Enhanced Fan Out (EFO) customers to continously experience IllegalArgumentException when their stream is resharded. This occurs due to the subscription being restarted when it should be completed and cleaned up.

Example of error:

aws-java-sdk-NettyEventLoop-2-2] INFO org.apache.flink.connector.kinesis.source.reader.fanout.FanOutKinesisShardSubscription - Subscription complete - shardId-000000003081 (arn:aws:kinesis:us-east-1:890385144326:stream/test-source-stream/consumer/my-flink-efo-consumer:1774401121)
[aws-java-sdk-NettyEventLoop-2-2] INFO org.apache.flink.connector.kinesis.source.reader.fanout.FanOutKinesisShardSubscription - Activating subscription to shard shardId-000000003081 with starting position StartingPosition{shardIteratorType=AFTER_SEQUENCE_NUMBER, startingMarker=null} for consumer arn:aws:kinesis:us-east-1:890385144326:stream/test-source-stream/consumer/my-flink-efo-consumer:1774401121.
[aws-java-sdk-NettyEventLoop-2-2] ERROR software.amazon.awssdk.utils.async.FlatteningSubscriber - Unexpected exception encountered that violates the reactive streams specification. Attempting to terminate gracefully.
java.lang.IllegalArgumentException: Invalid StartingPosition. When ShardIteratorType is AT_SEQUENCE_NUMBER or AFTER_SEQUENCE_NUMBER, startingMarker must be a String.
        at org.apache.flink.util.Preconditions.checkArgument(Preconditions.java:138)
        at org.apache.flink.connector.kinesis.source.split.StartingPositionUtil.toSdkStartingPosition(StartingPositionUtil.java:65)
        at org.apache.flink.connector.kinesis.source.proxy.KinesisAsyncStreamProxy.subscribeToShard(KinesisAsyncStreamProxy.java:56)
        at org.apache.flink.connector.kinesis.source.reader.fanout.FanOutKinesisShardSubscription.activateSubscription(FanOutKinesisShardSubscription.java:137)
        at org.apache.flink.connector.kinesis.source.reader.fanout.FanOutKinesisShardSubscription$FanOutShardSubscriber.onComplete(FanOutKinesisShardSubscription.java:334)
        at software.amazon.awssdk.utils.async.DelegatingSubscriber.onComplete(DelegatingSubscriber.java:47)
        at software.amazon.awssdk.utils.async.EventListeningSubscriber.onComplete(EventListeningSubscriber.java:66)
        at software.amazon.awssdk.utils.async.DelegatingSubscriber.onComplete(DelegatingSubscriber.java:47)
        at software.amazon.awssdk.utils.async.EventListeningSubscriber.onComplete(EventListeningSubscriber.java:66)
        at software.amazon.awssdk.utils.async.DelegatingSubscriber.onComplete(DelegatingSubscriber.java:47)
        at software.amazon.awssdk.utils.async.EventListeningSubscriber.onComplete(EventListeningSubscriber.java:66)
        at software.amazon.awssdk.utils.async.FlatteningSubscriber.handleOnCompleteState(FlatteningSubscriber.java:229)
        at software.amazon.awssdk.utils.async.FlatteningSubscriber.handleStateUpdate(FlatteningSubscriber.java:169)
        at software.amazon.awssdk.utils.async.FlatteningSubscriber.onComplete(FlatteningSubscriber.java:129)
        at software.amazon.awssdk.utils.internal.MappingSubscriber.onComplete(MappingSubscriber.java:63)
        at software.amazon.awssdk.utils.async.FlatteningSubscriber.handleOnCompleteState(FlatteningSubscriber.java:229)
        at software.amazon.awssdk.utils.async.FlatteningSubscriber.handleStateUpdate(FlatteningSubscriber.java:169)
        at software.amazon.awssdk.utils.async.FlatteningSubscriber.onComplete(FlatteningSubscriber.java:129)
        at software.amazon.awssdk.utils.internal.MappingSubscriber.onComplete(MappingSubscriber.java:63)
        at software.amazon.awssdk.core.internal.metrics.BytesReadTrackingPublisher$BytesReadTracker.onComplete(BytesReadTrackingPublisher.java:74)
        at software.amazon.awssdk.http.nio.netty.internal.ResponseHandler$DataCountingPublisher$1.onComplete(ResponseHandler.java:519)
        at software.amazon.awssdk.http.nio.netty.internal.ResponseHandler.runAndLogError(ResponseHandler.java:254)
        at software.amazon.awssdk.http.nio.netty.internal.ResponseHandler.access$600(ResponseHandler.java:77)
        at software.amazon.awssdk.http.nio.netty.internal.ResponseHandler$PublisherAdapter$1.onComplete(ResponseHandler.java:375)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HandlerPublisher.complete(HandlerPublisher.java:447)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HandlerPublisher.handlerRemoved(HandlerPublisher.java:435)
        at io.netty.channel.AbstractChannelHandlerContext.callHandlerRemoved(AbstractChannelHandlerContext.java:1138)
        at io.netty.channel.DefaultChannelPipeline.callHandlerRemoved0(DefaultChannelPipeline.java:636)
        at io.netty.channel.DefaultChannelPipeline.remove(DefaultChannelPipeline.java:476)
        at io.netty.channel.DefaultChannelPipeline.remove(DefaultChannelPipeline.java:422)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HttpStreamsHandler.removeHandlerIfActive(HttpStreamsHandler.java:370)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HttpStreamsHandler.handleReadHttpContent(HttpStreamsHandler.java:232)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HttpStreamsHandler.channelRead(HttpStreamsHandler.java:203)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HttpStreamsClientHandler.channelRead(HttpStreamsClientHandler.java:173)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:442)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at software.amazon.awssdk.http.nio.netty.internal.http2.Http2ToHttpInboundAdapter.onDataRead(Http2ToHttpInboundAdapter.java:87)
        at software.amazon.awssdk.http.nio.netty.internal.http2.Http2ToHttpInboundAdapter.channelRead0(Http2ToHttpInboundAdapter.java:49)
        at software.amazon.awssdk.http.nio.netty.internal.http2.Http2ToHttpInboundAdapter.channelRead0(Http2ToHttpInboundAdapter.java:42)
        at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:289)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:442)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
        at io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1407)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:440)
        at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
        at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:918)
        at io.netty.handler.codec.http2.AbstractHttp2StreamChannel$Http2ChannelUnsafe.doRead0(AbstractHttp2StreamChannel.java:981)
        at io.netty.handler.codec.http2.AbstractHttp2StreamChannel$Http2ChannelUnsafe.doBeginRead(AbstractHttp2StreamChannel.java:882)
        at io.netty.handler.codec.http2.AbstractHttp2StreamChannel$Http2ChannelUnsafe.beginRead(AbstractHttp2StreamChannel.java:845)
        at io.netty.channel.DefaultChannelPipeline$HeadContext.read(DefaultChannelPipeline.java:1359)
        at io.netty.channel.AbstractChannelHandlerContext.invokeRead(AbstractChannelHandlerContext.java:845)
        at io.netty.channel.AbstractChannelHandlerContext.read(AbstractChannelHandlerContext.java:824)
        at software.amazon.awssdk.http.nio.netty.internal.http2.FlushOnReadHandler.read(FlushOnReadHandler.java:40)
        at io.netty.channel.AbstractChannelHandlerContext.invokeRead(AbstractChannelHandlerContext.java:849)
        at io.netty.channel.AbstractChannelHandlerContext.read(AbstractChannelHandlerContext.java:824)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HandlerPublisher.requestDemand(HandlerPublisher.java:130)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HttpStreamsHandler$1.requestDemand(HttpStreamsHandler.java:195)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HandlerPublisher.receivedDemand(HandlerPublisher.java:304)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HandlerPublisher.access$200(HandlerPublisher.java:61)
        at software.amazon.awssdk.http.nio.netty.internal.nrs.HandlerPublisher$ChannelSubscription$1.run(HandlerPublisher.java:495)
        at io.netty.util.concurrent.AbstractEventExecutor.runTask(AbstractEventExecutor.java:173)
        at io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:166)
        at io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:469)
        at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:566)
        at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:994)
        at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
        at java.base/java.lang.Thread.run(Thread.java:1583)

Verifying this change

  • Manually verified by running the Kinesis connector on a local Flink cluster.

After making the change, i 1. re-ran my flink application, 2. resharded my stream. Afterwards verified that subscriptions were being shut down and restarted without issues:

reshard

[Source: Kinesis source -> (Sink: Print to Std. Out, Kinesis sink: Writer) (1/1)#0] INFO org.apache.flink.connector.base.source.reader.SourceReaderBase - Finished reading split(s) [shardId-000000003079]
[Source Data Fetcher for Source: Kinesis source -> (Sink: Print to Std. Out, Kinesis sink: Writer) (1/1)#0] INFO org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher - Finished reading from splits [shardId-000000003079]
[Source: Kinesis source -> (Sink: Print to Std. Out, Kinesis sink: Writer) (1/1)#0] INFO org.apache.flink.connector.base.source.reader.SourceReaderBase - Finished reading split(s) [shardId-000000003080]
[Source Data Fetcher for Source: Kinesis source -> (Sink: Print to Std. Out, Kinesis sink: Writer) (1/1)#0] INFO org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher - Finished reading from splits [shardId-000000003080]
[Source: Kinesis source -> (Sink: Print to Std. Out, Kinesis sink: Writer) (1/1)#0] INFO org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager - Closing splitFetcher 0 because it is idle.
[Source: Kinesis source -> (Sink: Print to Std. Out, Kinesis sink: Writer) (1/1)#0] INFO org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher - Shutting down split fetcher 0
[SourceCoordinator-Source: Kinesis source] INFO org.apache.flink.connector.kinesis.source.enumerator.KinesisStreamsSourceEnumerator - Assigning shard shardId-000000003081 from stream arn:aws:kinesis:us-east-1:890385144326:stream/test-source-stream to subtask 0.
[Source: Kinesis source -> (Sink: Print to Std. Out, Kinesis sink: Writer) (1/1)#0] INFO org.apache.flink.connector.base.source.reader.SourceReaderBase - Adding split(s) to reader: [KinesisShardSplit{streamArn='arn:aws:kinesis:us-east-1:890385144326:stream/test-source-stream', shardId='shardId-000000003081', startingPosition=StartingPosition{shardIteratorType=TRIM_HORIZON, startingMarker=null}, parentShardIds=[shardId-000000003079,shardId-000000003080], startingHashKey='0', endingHashKey='340282366920938463463374607431768211455', finished=false}]
[Source Data Fetcher for Source: Kinesis source -> (Sink: Print to Std. Out, Kinesis sink: Writer) (1/1)#0] INFO org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher - Split fetcher 0 exited.
[Source Data Fetcher for Source: Kinesis source -> (Sink: Print to Std. Out, Kinesis sink: Writer) (1/1)#0] INFO org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher - Starting split fetcher 1
[Source Data Fetcher for Source: Kinesis source -> (Sink: Print to Std. Out, Kinesis sink: Writer) (1/1)#0] INFO org.apache.flink.connector.kinesis.source.reader.fanout.FanOutKinesisShardSubscription - Activating subscription to shard shardId-000000003081 with starting position StartingPosition{shardIteratorType=TRIM_HORIZON, startingMarker=null} for consumer arn:aws:kinesis:us-east-1:890385144326:stream/test-source-stream/consumer/my-flink-efo-consumer:1774401121.
[aws-java-sdk-NettyEventLoop-2-10] INFO org.apache.flink.connector.kinesis.source.reader.fanout.FanOutKinesisShardSubscription - Successfully subscribed to shard shardId-000000003081 at StartingPosition{shardIteratorType=TRIM_HORIZON, startingMarker=null} using consumer arn:aws:kinesis:us-east-1:890385144326:stream/test-source-stream/consumer/my-flink-efo-consumer:1774401121.
[ForkJoinPool.commonPool-worker-4] INFO org.apache.flink.connector.kinesis.source.reader.fanout.FanOutKinesisShardSubscription - Successfully subscribed to shard shardId-000000003081 with starting position StartingPosition{shardIteratorType=TRIM_HORIZON, startingMarker=null} for consumer arn:aws:kinesis:us-east-1:890385144326:stream/test-source-stream/consumer/my-flink-efo-consumer:1774401121.

Significant changes

N/A - Not a significant change

consumerArn);
if (subscriptionActive.get()) {
if (shardSubscriber != null
&& shardSubscriber.getSubscriptionState() == SubscriptionState.SUBSCRIBED) {
Copy link
Copy Markdown

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.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated wording


public void cancel() {
if (!subscriptionActive.get()) {
if (this.subscriptionState.get() == SubscriptionState.COMPLETED) {
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

change the message to be more accurate?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The 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

private enum SubscriptionState {
NOT_STARTED,
SUBSCRIBED,
COMPLETED
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can there be an error state?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The 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):

        kinesis.subscribeToShard(consumerArn, shardId, startingPosition, responseHandler)
                .exceptionally(
                        throwable -> {
                            // If consumer exists and is still activating, we want to countdown.
                            if (ExceptionUtils.findThrowable(
                                            throwable, ResourceInUseException.class)
                                    .isPresent()) {
                                waitForSubscriptionLatch.countDown();
                                return null;
                            }
                            LOG.error(
                                    "Error subscribing to shard {} with starting position {} for consumer {}.",
                                    shardId,
                                    startingPosition,
                                    consumerArn,
                                    throwable);
                            terminateSubscription(throwable);
                            return null;
                        });

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants