Remove TableSchema to JSON conversion.#28274
Conversation
|
Checks are failing. Will not request review until checks are succeeding. If you'd like to override that behavior, comment |
|
assign set of reviewers |
|
Assigning reviewers. If you would like to opt out of this review, comment R: @kennknowles for label java. Available commands:
The PR bot will only process comments in the main thread (not review comments). |
|
It is not obvious to me wether it was intended / this would introduce breaking change. Have you tried injecting an error in the try block and see when and where the exception will throw? |
|
@Abacn Thank you for reviewing. This was challenging to replicate. Due to the use of a static final BigQueryIO.JSON_FACTORY. However, when I executed the tests in debug mode, I observed this lambda was called first. We should prevent downstream processing of data if the tableschema cannot be parsed which would present bigger problems later. |
|
I mean we can just change the current PR to or throw the Exception by chance, e.g. and see how the tests behaves. In general for changes that add exception to throw we can always tests like this |
| String jsonTableSchema = BigQueryIO.JSON_FACTORY.toString(input); | ||
| return (AvroSource.DatumReaderFactory<T>) | ||
| (writer, reader) -> | ||
| new GenericDatumTransformer<>(parseFn, jsonTableSchema, writer); |
There was a problem hiding this comment.
Analyzing the code further, I saw that the original GenericDatumTransformer's constructor received a JSON string so that later it would re-parse that JSON into a TableSchema again but via a Supplier function.
There was a problem hiding this comment.
that was because TableSchema is not serializable
| } | ||
| TableSchema safeInput = checkStateNotNull(input); | ||
| return (AvroSource.DatumReaderFactory<T>) | ||
| (writer, reader) -> new GenericDatumTransformer<>(parseFn, safeInput, writer); |
There was a problem hiding this comment.
This uses the new GenericDatumTransformer constructor that just takes the TableSchema input directly instead of needing to convert to JSON and then back again to the TableSchema later.
| public T read(T reuse, Decoder in) throws IOException { | ||
| GenericRecord record = (GenericRecord) this.reader.read(reuse, in); | ||
| return parseFn.apply(new SchemaAndRecord(record, this.tableSchema.get())); | ||
| return parseFn.apply(new SchemaAndRecord(record, this.tableSchema)); |
There was a problem hiding this comment.
Removes the use of a Supplier function to store the TableSchema and simply uses a class property to hold the TableSchema.
There was a problem hiding this comment.
I digged into this was added back to 7fde976 (https://issues.apache.org/jira/browse/BEAM-2532) then refactored into BigQueryIO.java by #22718. Is BEAM-2532 no longer an issue?
There was a problem hiding this comment.
(answer myself) I think after the change of #22718, the supplier is not needed, as it no longer involves preserving a TableSchema in SerializableFunction. It is now a named static class GenericDatumTransformer not implementing Serializable.
|
R: @johnjcasey :-) Thank you |
|
Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control |
|
Run Java_GCP_IO_Direct PreCommit |
| this.tableSchema = | ||
| Suppliers.memoize( | ||
| Suppliers.compose(new TableSchemaFunction(), Suppliers.ofInstance(tableSchema))); | ||
| this.tableSchema = new TableSchemaFunction().apply(tableSchema); |
There was a problem hiding this comment.
We can remove TableSchemaFunction as well as this first constructor
There was a problem hiding this comment.
Thank you for this suggestion. It was needed to convert the JSON string constructor parameter into a TableSchema. I felt that since the method was public, we needed to still support it and thus needed this conversion using TableSchemaFunction. @Abacn in light of this context, do you agree?
There was a problem hiding this comment.
TableSchemaFunction is private and GenericDatumTransformer is package private. The public constructor just mean it can be called within the package
There was a problem hiding this comment.
Well, as of this PR itself the current change suffices and LGTM. This could be cleaned up / or leave it as is
There was a problem hiding this comment.
+1 to removing TableSchemaFunction and the constructor, it can't be accessed by users anyways. We can get the tableschema from json directly with BigQueryHelpers.fromJsonString(tableschema, TableSchema.class)
|
@johnjcasey / @Abacn |
This was a test recently added, as part of effort for test coverage of BigQuery IO. It should be irrelevant (storage api write) CC: @ahmedabu98 |
|
Run Java_GCP_IO_Direct PreCommit |
ahmedabu98
left a comment
There was a problem hiding this comment.
Just one comment but otherwise looks good, lets see if this test passes
| this.tableSchema = | ||
| Suppliers.memoize( | ||
| Suppliers.compose(new TableSchemaFunction(), Suppliers.ofInstance(tableSchema))); | ||
| this.tableSchema = new TableSchemaFunction().apply(tableSchema); |
There was a problem hiding this comment.
+1 to removing TableSchemaFunction and the constructor, it can't be accessed by users anyways. We can get the tableschema from json directly with BigQueryHelpers.fromJsonString(tableschema, TableSchema.class)
This PR addresses #28080 by refactoring the static BigQueryIO.read(SerializableFunction) to not need a try/catch when invoking the DatumReaderFactory.
The existing code first converted the TableSchema to a JSON string using BigQueryIO.JSON_FACTORY's toString method. This throws an IOException that the original code logged as a warning. A GenericDatumTransformer's constructor received that JSON string that was used to set a private Suppler property, called tableSchema. This Supplier property, when invoked, would convert the JSON string back to a TableSchema object.
This PR simplifies the code by creating a new GenericDatumTransformer constructor that receives the TableSchema directly and sets the internal TableSchema property, removing the need for a Supplier and the need to re-parse the JSON string back to a TableSchema.
Since the existing GenericDatumTransformer constructor that received a JSON string representation of the TableSchema was public, I did not remove it.
To validate these changes, I ran:
./gradlew :sdks:java:io:google-cloud-platform:checkand
./gradlew :sdks:java:io:google-cloud-platform:integrationTestThank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
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, commentfixes #<ISSUE NUMBER>instead.UpdateCHANGES.mdwith 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)
See CI.md for more information about GitHub Actions CI or the workflows README to see a list of phrases to trigger workflows.