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

Port x-lang support to kinesis v2 #25416

Closed

Conversation

xinbinhuang
Copy link
Contributor

@xinbinhuang xinbinhuang commented Feb 10, 2023

Please add a meaningful description for your change here

cc: @aromanenko-dev @mosche

fixes: #23570

I've implemented the Java part and have some questions wanna clear before adding the Python part:

Java Questions:

  • do I need to add the expand service gradle config: https://github.com/apache/beam/tree/master/sdks/java/io/kinesis/expansion-service?
  • What would be the proper input type to KinesisIO.Write? I'm not sure how Python type translate to x-lang transforms.. I hard-coded it to byte[] for now.
  • Why does KinesisIO implements its own KinesisIO.Write.Result instead of using the provided PDone? They looks the same in terms of implementation.
  • Testing: is adding tests to xlang_kinesisio_it_test.py after changes on the python part the correct approach?

Python questions:

  • Because some fields are removed (i.e. partition_key, verify_certificate), and mechanisms are changes (i.e. partitioning). What's the general policy for beam to introduce breaking changes? Do we just break it at the code-level and release it with major version? Or do we introduce a v2 version and deprecate the old one to allow gradual transition.

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

  • 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.

@github-actions
Copy link
Contributor

Checks are failing. Will not request review until checks are succeeding. If you'd like to override that behavior, comment assign set of reviewers

@aromanenko-dev
Copy link
Contributor

@xinbinhuang Many thanks for working on this!

Ping @chamikaramj to help with x-lang questions

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.

Thanks @xinbinhuang !

@chamikaramj can you provide more details? Pls correct me if I'm wrong below.

do I need to add the expand service gradle config: https://github.com/apache/beam/tree/master/sdks/java/io/kinesis/expansion-service?

Yes, the expansion service is needed to run the Java side.

What would be the proper input type to KinesisIO.Write? I'm not sure how Python type translate to x-lang transforms. I hard-coded it to byte[] for now.

byte[] seems ok for both reading and writing. Though, that makes it hard to use decent, content based partitioning (see comment on random partitioning). For writing a tuple type such as KV would make a lot more sense. That allows users to pass a partition key. Have a look at the external KafkaIO as example.

Similarly, the record metadata might be required for certain use cases when reading. As far as I know hat requires a schema for KinesisRecord, but I'm not sure of the contracts / how this is supposed to be done. In any case, just returning the binary data seems to be ok in the read case.

Why does KinesisIO implements its own KinesisIO.Write.Result instead of using the provided PDone? They looks the same in terms of implementation.

This is a Beam convention improve evolution of write APIs:

The top-level I/O class will provide a static method to start constructing an I/O.Write 
transform. This returns a PTransform with a single input PCollection, and a Write.Result 
output.

For instance this allows to add more advanced error handling, e.g. adding a mode that returns failed records rather than failing fast.

Because some fields are removed (i.e. partition_key, verify_certificate), and mechanisms are changes (i.e. partitioning). What's the general policy for beam to introduce breaking changes? Do we just break it at the code-level and release it with major version? Or do we introduce a v2 version and deprecate the old one to allow gradual transition.

This is a tough one... I suppose it would be possible to silently switch the implementation on the Java side carefully setting defaults and ignoring obsolete settings (such as the old partitioning key) with appropriate warnings added. On the other hand, if types are also changed it's impossible to not break anything. That would require to add a v2, deprecate the previous one and remove it after 3+ releases.

configuration.awsSecretKey,
configuration.region,
configuration.serviceEndpoint))
.withPartitioner(KinesisPartitioner.explicitRandomPartitioner(configuration.shards));
Copy link
Member

Choose a reason for hiding this comment

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

Not sure what strategy is best here. Using the explicitRandomPartitioner only works well if the number of shards is fixed, if shards get split the new ones wouldn't receive any data. On the other hand, a completely random strategy (generating new random partition keys each time) breaks record aggregation. In case re-sharding is (potentially) used, users should probably configure shards as the max number of expected shards.

String streamName;
@Nullable String awsAccessKey;
@Nullable String awsSecretKey;
Region region;
Copy link
Member

Choose a reason for hiding this comment

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

Region can be nullable as well, it will try to fallback to the AWS default region of the environment

this.region = Region.of(region);
}

public void setServiceEndpoint(@Nullable String serviceEndpoint) {
Copy link
Member

Choose a reason for hiding this comment

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

The param shouldn't be nullable, that would cause a NPE

@Override
public PTransform<PCollection<byte[]>, KinesisIO.Write.Result> buildExternal(
Configuration configuration) {
KinesisIO.Write<byte[]> writeTransform =
Copy link
Member

Choose a reason for hiding this comment

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

The writer requires a serializer, in this case a trivial one: byte[] -> byte[]

}

@Override
public PTransform<PBegin, PCollection<KinesisRecord>> buildExternal(
Copy link
Member

Choose a reason for hiding this comment

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

The output needs to be byte[]. You can just add a MapElements to extract data from the record.

@mosche
Copy link
Member

mosche commented May 5, 2023

Kind ping @xinbinhuang

@github-actions
Copy link
Contributor

github-actions bot commented Jul 4, 2023

This pull request has been marked as stale due to 60 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time and @mention a reviewer or discuss it on the [email protected] list. Thank you for your contributions.

@github-actions github-actions bot added the stale label Jul 4, 2023
@aromanenko-dev
Copy link
Contributor

Hi @xinbinhuang, kind ping on this

@github-actions github-actions bot removed the stale label Jul 5, 2023
@github-actions
Copy link
Contributor

github-actions bot commented Sep 4, 2023

This pull request has been marked as stale due to 60 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time and @mention a reviewer or discuss it on the [email protected] list. Thank you for your contributions.

@github-actions github-actions bot added the stale label Sep 4, 2023
@github-actions
Copy link
Contributor

This pull request has been closed due to lack of activity. If you think that is incorrect, or the pull request requires review, you can revive the PR at any time.

@github-actions github-actions bot closed this Sep 11, 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.

Kinesis x-lang support depends on deprecated Kinesis IO (Aws Sdk v1)
3 participants