-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Support Avro GenericRecord as a valid format for StorageWrite API on BigQueryIO #24274
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 Avro GenericRecord as a valid format for StorageWrite API on BigQueryIO #24274
Conversation
|
Checks are failing. Will not request review until checks are succeeding. If you'd like to override that behavior, comment |
|
Assigning reviewers. If you would like to opt out of this review, comment R: @kileys for label java. Available commands:
The PR bot will only process comments in the main thread (not review comments). |
|
Run Java PreCommit |
1 similar comment
|
Run Java PreCommit |
|
Fixes #24329 |
|
Thanks for the contribution! Taking a look. |
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java
Outdated
Show resolved
Hide resolved
Abacn
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the work! Looks solid to me. Haven't get into two proto related new classes yet. Left a few comments found.
| .put(SqlTypes.DATETIME.getIdentifier(), StandardSQLTypeName.DATETIME) | ||
| .put("SqlTimeWithLocalTzType", StandardSQLTypeName.TIME) | ||
| .put("Enum", StandardSQLTypeName.STRING) | ||
| .put(MicrosInstant.IDENTIFIER, StandardSQLTypeName.TIMESTAMP) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is there a test verifying that this logical type is supported? MicrosInstant is used for java.time.Instant (saying, testing write java.time.Instant to BigQuery)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
MicrosInstant is also used for the translation of the Avro logical type named TimestampMicros (which is being forced in the test BigQueryWriteIOTest.testWriteAvro().
For the time related logical types (timestampmicros and millis, alongside with Date), Avro types store internally a numeric value that represents time since epoch (micros, millis or days depending on the type).
I'm not sure if I can force an Avro object to carry a java.time.Instant (maybe not) and this code path executes only when the underlying Avro GenericRecord has a time related logical type (which is being tested) and its schema gets translated to a Beam Row.Schema yielding the MicrosInstant (AvroUtils.toFieldType method).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
fyi Support of Micros_instant in AvroUtils is about to rolled back due to a breaking change (#24489). By now MicrosInstant logical type (and java.time.Instant) support is incomplete throughout the code base. Sorry for inconvenience.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
oh, so the only supported time related logical types will be millis and date for now, is that right?
I can remove the timestamp in micros for the time being if that works for this PR. sounds good?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yeah sounds good to me
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ok, in fact I rebased from master and removed the MicrosInstant mapping and the tests are still running fine. So I will keep the micros support since now those get mapped to Instant and work fine.
| storageApiDynamicDestinations = | ||
| new StorageApiDynamicDestinationsBeamRow<>( | ||
| dynamicDestinations, elementSchema, elementToRowFunction); | ||
| } else if (getAvroRowWriterFactory() != null) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please add a writeGenericRecords() method so users don't have to deal with the avro format function.
Also - AFAICT from inspection, the AvroWriterFactory is a bit broken if side inputs are used for schemas. Do we have a test for this?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please add a writeGenericRecords() method so users don't have to deal with the avro format function.
Done.
AFAICT from inspection, the AvroWriterFactory is a bit broken if side inputs are used for schemas. Do we have a test for this?
You mean when the TableSchema is provided as a PCollectionView? If so I can add a test for that.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah - because setSideInputAccessorFromProcessContext needs to be called on the dynamic destinations in every DoFn that might access this, and I'm not sure that's correct for AvroWriterFactory
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added a test to check on this for StorageWrites API
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Also and JIC, besides the added unit test, checked this PR's code with a simple DynamicDestination definition and seems to work fine for job id: 2023-01-31_12_10_08-12389116961981372457
|
Please resolve against current codebase |
...ain/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsGenericRecord.java
Outdated
Show resolved
Hide resolved
...ain/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiDynamicDestinationsGenericRecord.java
Outdated
Show resolved
Hide resolved
|
Run Spotless PreCommit |
|
Run Java_GCP_IO_Direct PreCommit |
|
Run Java PreCommit |
| Schema elementType = null; | ||
| switch (schema.getType()) { | ||
| case RECORD: | ||
| if (schema == null) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If this were null, the above switch would already have thrown. Did you mean to check something else?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good catch, moved to before the switch statement.
| if (schema == null) { | ||
| throw new RuntimeException("Unexpected null schema!"); | ||
| } | ||
| Preconditions.checkState(!schema.getFields().isEmpty()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Are empty records disallowed in avro?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
They are allowed, the check was also in the BeamRowToStorageApiProto similar methods (Beam Row.Schema to Descriptors) and wanted to put the same safeguards.
Not sure how useful would be to have an empty record in this context, but let me know if you want me to remove the restriction.
| case RECORD: | ||
| return messageFromGenericRecord(fieldDescriptor.getMessageType(), (GenericRecord) value); | ||
| case ARRAY: | ||
| List<Object> list = (List<Object>) value; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Are we sure it's always a List (and not just an Iterable)?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Changed it to Iterable.
...a/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
Outdated
Show resolved
Hide resolved
...a/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
Outdated
Show resolved
Hide resolved
...a/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
Outdated
Show resolved
Hide resolved
| storageApiDynamicDestinations = | ||
| new StorageApiDynamicDestinationsBeamRow<>( | ||
| dynamicDestinations, elementSchema, elementToRowFunction); | ||
| } else if (getAvroRowWriterFactory() != null) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah - because setSideInputAccessorFromProcessContext needs to be called on the dynamic destinations in every DoFn that might access this, and I'm not sure that's correct for AvroWriterFactory
|
Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment R: @robertwb for label java. Available commands:
|
Abacn
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry for the delay. Will take another look to see if previous comments are resolved. Or @reuvenlax do you still have a chance to review, thanks!
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java
Show resolved
Hide resolved
|
Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment R: @kileys for label java. Available commands:
|
|
Reminder, please take a look at this pr: @kileys @johnjcasey |
|
Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment R: @kennknowles for label java. Available commands:
|
|
Run Java_Kafka_IO_Direct PreCommit |
|
Run Java_Kafka_IO_Direct PreCommit |
|
Reminder, please take a look at this pr: @kennknowles @pabloem |
|
Hi @prodriguezdefino thanks for the patience. This looks good to me. Would you mind addressing #24274 (comment) so that when #24992 also gets in it won't break? Thanks! |
Abacn
left a comment
There was a problem hiding this 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 adding a piece of announcement in https://github.com/apache/beam/blob/master/CHANGES.md?plain=1#L62
will merge when test passed
|
Run Java PreCommit |
|
Run Java_Kafka_IO_Direct PreCommit |
|
Test passed. Some GitHub action job hanging due to self-host runner issues. merging for now. |


This change enables the use of AVRO GenericRecord as part of the BigQueryIO storage write implementation (consistent and at-least-once).
When using AVRO GenericRecord as the transport object in their pipelines, users need to configure the use of Beam schemas in the BigQueryIO to transform their data into Beam Rows first, or provide a format function to transform into TableRow, to then have their data formatted as DynamicMessage protos for BQ StorageWrite to work. This PR enables the users to provide a schema (BQ TableSchema, dyn destination or schema from view) and an AvroRowFactoryWriter to have their data transformed to GenericRecord and then StorageWrite compatible protos.
When using GenericRecord as the input type for the PCollection, providing the identity function should be sufficient and inexpensive for their data ingestion.
Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
R: @username).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.CHANGES.mdwith noteworthy changes.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.