I'm having some problems validating the RCs proposed here (both 3 and 4). User code that depends on versions newer than Avro 1.8.2 are having problems running on Dataflow.
> Caused by: java.io.InvalidClassException: org.apache.avro.specific.SpecificRecordBase; local class incompatible: stream classdesc serialVersionUID = -1463700717714793795, local class serialVersionUID = 189988654766568477 and > Caused by: java.lang.NoSuchMethodError: 'boolean org.apache.avro.generic.GenericRecord.hasField(java.lang.String)' It seems that after https://github.com/apache/beam/pull/27851, we can not choose the version of Avro that is used, as beam-sdks-java-harness.jar now contains Avro 1.8.2. This feels like a blocker. It breaks things for DataflowTemplates <https://github.com/GoogleCloudPlatform/DataflowTemplates> and would require code changes/refactoring to not use new functionality, and most importantly, the error is only observed during runtime -- so it gives the impression that things are working, but the SDK can't process any bundle. I've filed https://github.com/apache/beam/issues/29413. I also went ahead and proposed https://github.com/apache/beam/pull/29412, which I think would be sufficient to solve this issue, but would like to hear opinions. Best, Bruno On Mon, Nov 13, 2023 at 5:14 AM Jan Lukavský <je...@seznam.cz> wrote: > +1 (binding) > > Validated Java SDK with Flink runner on own use cases. > Jan > > On 11/12/23 00:44, Danny McCormick via dev wrote: > > Hi everyone, > Please review and vote on the release candidate #3 for the version 2.52.0, > as follows: > [ ] +1, Approve the release > [ ] -1, Do not approve the release (please provide specific comments) > > > Reviewers are encouraged to test their own use cases with the release > candidate, and vote +1 if no issues are found. Only PMC member votes will > count towards the final vote, but votes from all community members is > encouraged and helpful for finding regressions; you can either test your > own use cases or use cases from the validation sheet [10]. > > The complete staging area is available for your review, which includes: > > - GitHub Release notes [1] > - the official Apache source release to be deployed to dist.apache.org [2], > which is signed with the key with fingerprint D20316F712213422 [3] > - all artifacts to be deployed to the Maven Central Repository [4] > - source code tag "v2.52.0-RC4" [5] > - website pull request listing the release [6], the blog post [6], and > publishing the API reference manual [7] > - Python artifacts are deployed along with the source release to the > dist.apache.org [2] and PyPI[8]. > - Go artifacts and documentation are available at pkg.go.dev [9] > - Validation sheet with a tab for 2.52.0 release to help with > validation [10] > - Docker images published to Docker Hub [11] > - PR to run tests against release branch [12] > > > The vote will be open for at least 72 hours. It is adopted by majority > approval, with at least 3 PMC affirmative votes. > > For guidelines on how to try the release in your projects, check out our > blog post at https://beam.apache.org/blog/validate-beam-release/. > > Thanks, > Danny > > [1] https://github.com/apache/beam/milestone/16 > [2] https://dist.apache.org/repos/dist/dev/beam/2.52.0/ > [3] https://dist.apache.org/repos/dist/release/beam/KEYS > [4] https://repository.apache.org/content/repositories/orgapachebeam-1362/ > [5] https://github.com/apache/beam/tree/v2.52.0-RC4 > [6] https://github.com/apache/beam/pull/29331 > [7] https://github.com/apache/beam-site/pull/654 > [8] https://pypi.org/project/apache-beam/2.52.0rc4/ > [9] > https://pkg.go.dev/github.com/apache/beam/sdks/v2@v2.52.0-RC4/go/pkg/beam > [10] > https://docs.google.com/spreadsheets/d/1qk-N5vjXvbcEk68GjbkSZTR8AGqyNUM-oLFo_ZXBpJw/edit#gid=1387982510 > [11] https://hub.docker.com/search?q=apache%2Fbeam&type=image > [12] https://github.com/apache/beam/pull/29404 > >