Skip to content

Conversation

@prodriguezdefino
Copy link
Contributor

@prodriguezdefino prodriguezdefino commented Nov 18, 2022

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:

  • Choose reviewer(s) and mention them in a comment (R: @username).
  • 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.

@prodriguezdefino prodriguezdefino changed the title initial impl Support Avro GenericRecord as a valid format for StorageWrite API on BigQueryIO Nov 19, 2022
@prodriguezdefino prodriguezdefino marked this pull request as ready for review November 20, 2022 03:29
@github-actions
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

@github-actions
Copy link
Contributor

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

R: @kileys for label java.
R: @Abacn for label io.

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).

@prodriguezdefino
Copy link
Contributor Author

Run Java PreCommit

1 similar comment
@prodriguezdefino
Copy link
Contributor Author

Run Java PreCommit

@prodriguezdefino
Copy link
Contributor Author

Fixes #24329

@github-actions
Copy link
Contributor

Reminder, please take a look at this pr: @kileys @Abacn

@Abacn
Copy link
Contributor

Abacn commented Nov 30, 2022

Thanks for the contribution! Taking a look.

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 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)
Copy link
Contributor

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)

Copy link
Contributor Author

@prodriguezdefino prodriguezdefino Dec 1, 2022

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).

Copy link
Contributor

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.

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, 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?

Copy link
Contributor

@Abacn Abacn Dec 2, 2022

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

Copy link
Contributor Author

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) {
Copy link
Contributor

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?

Copy link
Contributor Author

@prodriguezdefino prodriguezdefino Dec 1, 2022

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.

Copy link
Contributor

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

Copy link
Contributor Author

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

Copy link
Contributor Author

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

@reuvenlax
Copy link
Contributor

Please resolve against current codebase

@prodriguezdefino
Copy link
Contributor Author

Run Spotless PreCommit

@prodriguezdefino
Copy link
Contributor Author

Run Java_GCP_IO_Direct PreCommit

@prodriguezdefino
Copy link
Contributor Author

Run Java PreCommit

Schema elementType = null;
switch (schema.getType()) {
case RECORD:
if (schema == null) {
Copy link
Contributor

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?

Copy link
Contributor Author

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());
Copy link
Contributor

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?

Copy link
Contributor Author

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;
Copy link
Contributor

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)?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Changed it to Iterable.

storageApiDynamicDestinations =
new StorageApiDynamicDestinationsBeamRow<>(
dynamicDestinations, elementSchema, elementToRowFunction);
} else if (getAvroRowWriterFactory() != null) {
Copy link
Contributor

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

@github-actions
Copy link
Contributor

github-actions bot commented Jan 6, 2023

Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment assign to next reviewer:

R: @robertwb for label java.
R: @pabloem for label io.

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)

@github-actions
Copy link
Contributor

Reminder, please take a look at this pr: @robertwb @pabloem

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.

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!

@github-actions
Copy link
Contributor

Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment assign to next reviewer:

R: @kileys for label java.
R: @johnjcasey for label io.

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)

@github-actions
Copy link
Contributor

Reminder, please take a look at this pr: @kileys @johnjcasey

@github-actions
Copy link
Contributor

Assigning new set of reviewers because Pr has gone too long without review. If you would like to opt out of this review, comment assign to next reviewer:

R: @kennknowles for label java.
R: @pabloem for label io.

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)

@prodriguezdefino
Copy link
Contributor Author

Run Java_Kafka_IO_Direct PreCommit

@prodriguezdefino
Copy link
Contributor Author

Tested this change with 2 very similar pipelines which:

  • read from PS
  • transform format into AVRO
  • and then one of them directly go into BQ and the other one transform to Row using beam schema (current possible path with code in master, the other one would be to write TableRow),
  • both pipelines process ~250MB/s
    and as expected the difference in resource utilization is significant.

Using beam rows as BigQueryIO input format:
Screenshot 2023-01-31 at 6 22 08 PM

Using write GenericRecords as BigQueryIO input format:
Screenshot 2023-01-31 at 6 22 24 PM

The difference in vCPU utilization at similar runtime is 176 vCPU/hr vs 117 vCPU/hr, more than 40% improvement when using GenericRecords.

@prodriguezdefino
Copy link
Contributor Author

Run Java_Kafka_IO_Direct PreCommit

@github-actions
Copy link
Contributor

Reminder, please take a look at this pr: @kennknowles @pabloem

@Abacn
Copy link
Contributor

Abacn commented Feb 14, 2023

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!

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 adding a piece of announcement in https://github.com/apache/beam/blob/master/CHANGES.md?plain=1#L62

will merge when test passed

@prodriguezdefino
Copy link
Contributor Author

Run Java PreCommit

@prodriguezdefino
Copy link
Contributor Author

Run Java_Kafka_IO_Direct PreCommit

@Abacn
Copy link
Contributor

Abacn commented Feb 17, 2023

Test passed. Some GitHub action job hanging due to self-host runner issues. merging for now.

@Abacn Abacn merged commit 40838f7 into apache:master Feb 17, 2023
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.

3 participants