clairemcginty opened a new pull request, #36809: URL: https://github.com/apache/beam/pull/36809
See: https://github.com/apache/avro/pull/3376/files In Avro 1.11.4+, schemas that utilize the `java-class` annotation throw runtime decoding errors. For example, applying a specific AvroCoder to a record with the following schema: ``` { "type": "record", "name": "TestJavaClassProp", "namespace": "com.test", "fields": [ { "name": "bd", "type": { "type": "string", "java-class": "java.math.BigDecimal" } } ] } ``` throws: ``` Caused by: java.lang.SecurityException: Forbidden java.math.BigDecimal! This class is not trusted to be included in Avro schemas using java-class. Please set the system property org.apache.avro.SERIALIZABLE_CLASSES to the comma separated list of classes you trust. You can also set the system property org.apache.avro.SERIALIZABLE_PACKAGES to the comma separated list of the packages you trust. at org.apache.avro.specific.SpecificDatumReader.checkSecurity(SpecificDatumReader.java:188) at org.apache.avro.specific.SpecificDatumReader.getPropAsClass(SpecificDatumReader.java:159) at org.apache.avro.specific.SpecificDatumReader.findStringClass(SpecificDatumReader.java:143) at com.spotify.scio.avro.SpecificRecordDatumFactory$ScioSpecificDatumReader.findStringClass(AvroDatumFactory.scala:80) at org.apache.avro.generic.GenericDatumReader$ReaderCache.lambda$getStringClass$1(GenericDatumReader.java:567) at java.base/java.util.concurrent.ConcurrentHashMap.computeIfAbsent(ConcurrentHashMap.java:1708) at org.apache.avro.generic.GenericDatumReader$ReaderCache.getStringClass(GenericDatumReader.java:567) at org.apache.avro.generic.GenericDatumReader.readString(GenericDatumReader.java:455) at org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:192) at org.apache.avro.specific.SpecificDatumReader.readField(SpecificDatumReader.java:232) at org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:248) at org.apache.avro.specific.SpecificDatumReader.readRecord(SpecificDatumReader.java:219) at org.apache.avro.generic.GenericDatumReader.readWithoutConversion(GenericDatumReader.java:180) at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:161) at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:154) at org.apache.beam.sdk.extensions.avro.coders.AvroCoder.decode(AvroCoder.java:476) ``` The fix, per https://github.com/apache/avro/pull/3376/files, is to set the sys prop `org.apache.avro.SERIALIZABLE_CLASSES`. We can do that easily with DirectRunner, but there's no way to set it on Dataflow unless we create a custom worker image This PR adds a proposed worker harness option allowing the user to specify trusted classes. An alternative would be to simply set a reasonable default; for example in the Avro project itself they set the [following default value](https://github.com/apache/avro/blob/release-1.11.5/lang/java/avro/pom.xml#L75-L77): ```xml <systemProperties> <org.apache.avro.SERIALIZABLE_CLASSES>java.math.BigDecimal,java.math.BigInteger,java.net.URI,java.net.URL,java.io.File,java.lang.Integer,org.apache.avro.reflect.TestReflect$R10</org.apache.avro.SERIALIZABLE_CLASSES> </systemProperties> ``` this would also solve my use case :) Let me know what you all think. If this looks good I can polish the javadoc/add tests/etc. ------------------------ 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](https://www.apache.org/licenses/icla.pdf). See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://github.com/apache/beam/blob/master/CONTRIBUTING.md#make-the-reviewers-job-easier). To check the build health, please visit [https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md](https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md) GitHub Actions Tests Status (on master branch) ------------------------------------------------------------------------------------------------ [](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule) [](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule) [](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule) [](https://github.com/apache/beam/actions?query=workflow%3A%22Go+tests%22+branch%3Amaster+event%3Aschedule) See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI or the [workflows README](https://github.com/apache/beam/blob/master/.github/workflows/README.md) to see a list of phrases to trigger workflows. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
