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

Support Zstd codec in SerializableAvroCodecFactory #32352

Merged
merged 6 commits into from
Sep 9, 2024

Conversation

clairemcginty
Copy link
Contributor

fixes #32349

Adds support for Zstandard compression in Avro writes


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 or the workflows README to see a list of phrases to trigger workflows.

@@ -50,6 +50,9 @@ class SerializableAvroCodecFactory implements Externalizable {
private static final Pattern deflatePattern = Pattern.compile(DEFLATE_CODEC + "-(?<level>-?\\d)");
private static final Pattern xzPattern = Pattern.compile(XZ_CODEC + "-(?<level>\\d)");

// Don't reference `DataFileConstants.ZSTANDARD_CODEC` directly for Avro 1.8 compat
private static final Pattern zstdPattern = Pattern.compile("zstandard\\[(?<level>\\d+)\\]");
Copy link
Contributor Author

@clairemcginty clairemcginty Aug 28, 2024

Choose a reason for hiding this comment

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

Note: DataFileConstants.ZSTANDARD_CODEC was added in Avro 1.9, so I didn't want to reference it here and break 1.8 users. When Avro 1.8 is dropped it can be used directly

Copy link
Contributor

Choose a reason for hiding this comment

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

Sorry for late question. zstandard compression level range from negative 7 to 22. This regex does not handle negative, is this a concern?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

oh great catch! added a -? to the regex and updated the tests.

@clairemcginty clairemcginty marked this pull request as ready for review August 28, 2024 19:55
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

Copy link
Contributor

Assigning reviewers. If you would like to opt out of this review, comment assign to next reviewer:

R: @kennknowles for label java.

Available commands:

  • stop reviewer notifications - opt out of the automated review tooling
  • remind me after tests pass - tag the comment author after tests pass
  • waiting on author - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)

The PR bot will only process comments in the main thread (not review comments).

@clairemcginty
Copy link
Contributor Author

cc @Abacn, since I saw you made some recent commits to Beam's avro module? :)

Copy link
Contributor

@Abacn Abacn left a comment

Choose a reason for hiding this comment

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

Thanks, would you mind add this change to https://github.com/apache/beam/blob/master/CHANGES.md ?

Under New Features / Improvements Section,

Support for Zstd codec SerializableAvroCodecFactory
 is added (Java/Python) ([#32349](https://github.com/apache/beam/issues/32349)).

@Abacn
Copy link
Contributor

Abacn commented Sep 8, 2024

merge conflicts after other change pushed CHANGES.md change, would you mind rebase onto the latest master? thanks!

@clairemcginty
Copy link
Contributor Author

merge conflicts after other change pushed CHANGES.md change, would you mind rebase onto the latest master? thanks!

cc @Abacn done! 🫡

@Abacn Abacn merged commit 1d0e09a into apache:master Sep 9, 2024
19 checks passed
@Abacn
Copy link
Contributor

Abacn commented Sep 9, 2024

thank you for prompt responses!

@clairemcginty clairemcginty deleted the avro-io-zstd-codec branch September 9, 2024 17:35
reeba212 pushed a commit to reeba212/beam that referenced this pull request Dec 4, 2024
* Support Zstd codec in SerializableAvroCodecFactory

* Test AvroIO.write

* Make tests compilable on Avro 1.8

* format

* Update CHANGES.md

* Support negative levels for zstd
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.

[Feature Request]: Support Zstd codec in SerializableAvroCodecFactory
2 participants