Skip to content
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

Kinesis enhanced fanout #23540

Merged
merged 35 commits into from
Apr 13, 2023
Merged

Kinesis enhanced fanout #23540

merged 35 commits into from
Apr 13, 2023

Conversation

psolomin
Copy link
Contributor

@psolomin psolomin commented Oct 7, 2022

addresses #19967
fixes #19967

This should be reviewed after #26117

Design decisions

As per #26117:

  • Maintained serialisation compatibility of KinesisReaderCheckpoint with Beam 2.46.0
  • Broken serialisation compatibility with Beam 2.46.0 for KinesisIO.Read and KinesisSource - this is confirmed for Flink runner, but may affect other runners as well, if they serialise these objects as parts of their checkpoint / savepoint. Unfortunately, KinesisIO.Read and its auto-generated counterpart had different default serialVersionUID, and it's impossible to keep them when we add withConsumerArn() option.

How this change was tested

With Flink runner - Flink 1.15.2, Java 11:

  • shard-up (up to 48 shards) and shard-down (down to 1)
  • failing pipelines (test recovery from some errors)
  • pipelines with slow processing (testing back-pressure)
  • start with a savepoint from previous Beam release (simulate Beam version upgrade)
  • start previous Beam release with a savepoint from new code (simulate Beam version downgrade)

Playground repo: https://github.com/psolomin/beam-playground/tree/master/kinesis-io-with-enhanced-fan-out


Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:

  • Choose reviewer(s) and mention them in a comment (R: @username).
  • Mention the appropriate issue in your description (for example: addresses #123), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, comment fixes #<ISSUE NUMBER> instead.
  • Update CHANGES.md with noteworthy changes.
  • If this contribution is large, please file an Apache Individual Contributor License Agreement.

See the Contributor Guide for more tips on how to make review process smoother.

To check the build health, please visit https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md

GitHub Actions Tests Status (on master branch)

Build python source distribution and wheels
Python tests
Java tests
Go tests

See CI.md for more information about GitHub Actions CI.

@mosche
Copy link
Member

mosche commented Oct 10, 2022

Thanks @psolomin! Sorry, I didn't get to look into this today but I'll have a look tomorrow.

Copy link
Member

@mosche mosche left a comment

Choose a reason for hiding this comment

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

@psolomin Thanks so much for working on this. Enhanced fanout is a long missed feature and it was frequently asked for on the mailing list. 🎉 💪

I have a few comments below.
Overall I'm honestly a bit torn and I'm wondering about your experience working on the IO?

One the one hand I feel this is pretty close and mostly misses the resharding stuff (which you're still working on). That's so awesome 🎉.

On the other hand the complexity of the IO has grown even further and I found it surprisingly tough to understand how things play together (or not). That part concerns me a bit, also thinking about maintenance.

Is it worth it to step back from the current implementation and think through how a pure subscriber based implementation would look like without the existing luggage? I suspect there's some potential to simplify things (a lot?). E.g. we're actually notified about child shards on the SubscribeToShardEvent, so no need for separate requests to figure these out. But I could be wrong ...

What do you think and how do you feel about the complexity aspect having worked on this for a bit now?

Comment on lines 70 to 64
kinesisAsyncSupplier = spec.getAWSClientsProvider()::getKinesisAsyncClient;
cloudWatchSupplier = spec.getAWSClientsProvider()::getCloudWatchClient;
Copy link
Member

Choose a reason for hiding this comment

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

We can forbid using the deprecated configuration path (AwsClientsProvider) when enabling enhanced fanout. Similar checks are in KinesisIO.Read.expand(...)

private void subscribeLoop(ShardRecordsIterator shardRecordsIterator) {
while (poolOpened.get()) {
try {
shardRecordsIterator.subscribeToShard(this::putRecord);
Copy link
Member

Choose a reason for hiding this comment

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

Somehow we have to get / process childShards from the SubscribeToShardEvent to handle any re-sharding event.

Also, shouldn't the next subscribe event use the continuationSequenceNumber?

Use this as SequenceNumber in the next call to SubscribeToShard, with
StartingPosition set to AT_SEQUENCE_NUMBER or AFTER_SEQUENCE_NUMBER. Use
ContinuationSequenceNumber for checkpointing because it captures your shard progress even when no
data is written to the shard.

Honestly, I'm not sure if that has any implications for checkpointing... I think the current approach should still be fine.

@psolomin
Copy link
Contributor Author

@mosche many thanks for the detailed feedback 👍

This was my first attempt. I basically tried to pull some pieces from #9899 to get myself more familiar with this codebase, and then after basic tests with real Kinesis stream found out this approach doesn't seem to work when I consider resharding.

Overall I'm honestly a bit torn and I'm wondering about your experience working on the IO?

First time I am putting my hands on into the guts of a Beam IO, but not the first time dealing with Kinesis consumers overall.

pretty close and mostly misses the resharding stuff

My impression was that I needed to re-think this approach - namely, avoid overloading ShardReadersPool and use a new ShardSubscribersPool as you suggested.

Is it worth it to step back from the current implementation and think through how a pure subscriber based implementation would look like without the existing luggage?

I felt the same and that's actually the thing I'm busy now with. Once I have something tangible which covers re-sharding cleanly, may I ping you again?

@mosche
Copy link
Member

mosche commented Oct 12, 2022

Once I have something tangible which covers re-sharding cleanly, may I ping you again?

Sure, absolutely, feel free to reach out any time!

@mosche
Copy link
Member

mosche commented Oct 12, 2022

First time I am putting my hands on into the guts of a Beam IO, but not the first time dealing with Kinesis consumers overall.

Also sorry @psolomin , you got me wrong there ... my fault. I wasn't asking for your Beam / Kinesis experience at all, but wanted to understand how you experienced it getting started on the KinesisIO. I personally think there's a lot that can be improved and it's unnecessarily complex (already). Glad to hear you got to the same conclusions and very keen to see your next version 🙂

@mosche
Copy link
Member

mosche commented Nov 2, 2022

@psolomin Great to see so much activity here :) Let me know when this is ready for another review, happy to have a look!

@mosche
Copy link
Member

mosche commented Nov 15, 2022

How's things going @psolomin ? Please let me know if there's any way I can support you on this.

@psolomin
Copy link
Contributor Author

Hello @mosche

Thanks for reaching out.

I went through implementing 1 design attempt two weeks ago, but was not happy with it in the end, and did not push that into this draft. I plan to finish and push the commits of another attempt by next Mon.

@mosche
Copy link
Member

mosche commented Nov 15, 2022

Sounds great @psolomin, happy to hear :) And no rush, I just wanted to reach out and see if I can support you.

@psolomin psolomin force-pushed the kinesis-enhanced-fanout branch 2 times, most recently from 74a60ab to c6dd506 Compare January 11, 2023 11:59
@psolomin psolomin force-pushed the kinesis-enhanced-fanout branch 3 times, most recently from 530d92a to 7a6ab3a Compare February 15, 2023 14:25
@psolomin psolomin force-pushed the kinesis-enhanced-fanout branch 2 times, most recently from 63577d1 to 746eb72 Compare February 28, 2023 18:36
@psolomin psolomin force-pushed the kinesis-enhanced-fanout branch 2 times, most recently from f30544b to f3250e2 Compare March 7, 2023 19:41
ShardState shardState = Preconditions.checkStateNotNull(state.get(shardId));
if (current.hasNext()) {
KinesisClientRecord r = current.next();
KinesisRecord kinesisRecord = new KinesisRecord(r, read.getStreamName(), shardId);
Copy link
Member

Choose a reason for hiding this comment

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

I think, at this point, this is the only doubt I actually have. Other things are already addressed, right @mosche ?

@psolomin yes, this is the only open issue I'm aware of. Before returning, shardState must be adequately updated if this was the last record of current to make sure checkpoint marks are correctly reflecting progress.

          KinesisClientRecord r = current.next();
          // Make sure to update shard state accordingly if `current` does not contain any more
          // events. This is necessary to account for any re-sharding, so we could correctly resume
          // from a checkpoint if taken once we advanced to the record returned by getNextRecord().
          if (!current.hasNext()) {
            onEventDone(shardState, current);
            current = null;
          }
          KinesisRecord kinesisRecord = new KinesisRecord(r, read.getStreamName(), shardId);
          if (shardState.isAfterInitialCheckpoint(kinesisRecord)) {
            shardState.update(kinesisRecord);
            return kinesisRecord;
          }

Looks like we're missing a test case for this as follows:

  • stub subscribeToShard with a reshard event that also contains 1 record.
  • wait for exactly that 1 record
  • verify the checkpoint mark now contains the new shards, but not the old one anymore

Copy link
Contributor Author

@psolomin psolomin Apr 12, 2023

Choose a reason for hiding this comment

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

reshard event that also contains 1 record

I am not sure such a thing exists. I was convinced that re-shard events always come with continuationSequenceNumber = null, and they never carry records with non-null sequence numbers.

Copy link
Member

Choose a reason for hiding this comment

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

I was convinced that re-shard events always come with continuationSequenceNumber = null

continuationSequenceNumber is unrelated to this. If it is null it simply means the shard doesn't contain any next records. But that doesn't mean the current event did not contain records.

Have you seen any documentation that explicitly states that events containing child shards do never contain records? If not we have to assume that this could be the case. Even if that is something we've never observed.

Copy link
Contributor Author

@psolomin psolomin Apr 12, 2023

Choose a reason for hiding this comment

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

Have you seen any documentation that explicitly states that events containing child shards do never contain records?

Not really, but I think I implicitly assumed that after I checked Flink implementation of EFO.

Even if that is something we've never observed.

Ok, got it. I am fine to go defensive here and assume that SubscribeToShardEvent can carry records AND child shards together. Let me try to put this out in the code, there might be other places it will be a trouble - I implicitly assumed this never happens.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Looks like we're missing a test case

Added it.

…m/sdk/io/aws2/kinesis/EFOShardSubscriber.java

Co-authored-by: Moritz Mack <mmack@talend.com>
@github-actions github-actions bot added build and removed build labels Apr 12, 2023
@github-actions github-actions bot added build and removed build labels Apr 12, 2023
@mosche
Copy link
Member

mosche commented Apr 12, 2023

Run Java PreCommit

@mosche
Copy link
Member

mosche commented Apr 12, 2023

Run Java_GCP_IO_Direct PreCommit

@mosche mosche self-requested a review April 12, 2023 17:21
Copy link
Member

@mosche mosche left a comment

Choose a reason for hiding this comment

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

🥇 Awesome work @psolomin. I'm just waiting for the tests to pass before merging.

@mosche mosche merged commit cd0f44b into apache:master Apr 13, 2023
20 of 21 checks passed
@psolomin psolomin deleted the kinesis-enhanced-fanout branch April 13, 2023 08:41
jakubrauch pushed a commit to jakubrauch/beam that referenced this pull request Apr 27, 2023
rezarokni pushed a commit to rezarokni/beam that referenced this pull request May 22, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Support for enhanced fan-out in KinesisIO.Read
2 participants