Skip to content

[java] Fix avro logical-types conversions for BQ storage#33422

Merged
Abacn merged 6 commits intoapache:masterfrom
RustedBones:bq-storage-time
Jan 27, 2025
Merged

[java] Fix avro logical-types conversions for BQ storage#33422
Abacn merged 6 commits intoapache:masterfrom
RustedBones:bq-storage-time

Conversation

@RustedBones
Copy link
Contributor

Most of the avro logical-type to BQ are broken.

Add support for both joda and java time to ensure compatibility with older avro versions


static Long convertTimestamp(Object value, boolean micros) {
if (value instanceof ReadableInstant) {
return ((ReadableInstant) value).getMillis() * (micros ? 1000 : 1);
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This was wrong. BQ always expects epoch microseconds. Conversion should be applied on the raw type depending if it represents millis or micros

Comment on lines +192 to +200
.setScale(type.getScale(), RoundingMode.DOWN)
.round(new MathContext(type.getPrecision(), RoundingMode.DOWN));
Copy link
Contributor Author

@RustedBones RustedBones Dec 19, 2024

Choose a reason for hiding this comment

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

Does this seems legit to round ? We might also fail if the BigDecimal precision and scale do not match with expected logical type

Copy link
Contributor

Choose a reason for hiding this comment

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

I understand previously it only accepts a ByteBuffer, now it's adding support of java.math.BigDecimal? If previously it would fail or it's not lose precision compared to the existing behavior I think it's fine.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

All other conversions accept the logical-type as well as the underlying-type.

This is mandatory to support both: even if the logical-type is present on the schema, it can be discarded when the GenericData used for serialization does not have the feature enabled.

Concerning the rounding, The doc states to use BigDecimalByteStringEncoder. BeamRowToStorageApiProto is a copy of that.

It's however not supporting parameterized NUMERIC and BIGNUMERIC types.

@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

@RustedBones
Copy link
Contributor Author

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: @kennknowles for label java.
R: @chamikaramj 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).

@github-actions
Copy link
Contributor

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

@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: @Abacn 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

github-actions bot commented Jan 8, 2025

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

Most of the avro logical-type to BQ are broken.

Add support for both joda and java time to ensure compatibility with
older avro versions
@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: @damondouglas for label java.
R: @damondouglas 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)

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 change! Had a few questions.

.put(LogicalTypes.timestampMicros().getName(), TableFieldSchema.Type.TIMESTAMP)
.put(LogicalTypes.timestampMillis().getName(), TableFieldSchema.Type.TIMESTAMP)
.put(LogicalTypes.uuid().getName(), TableFieldSchema.Type.STRING)
.put("date", TableFieldSchema.Type.DATE)
Copy link
Contributor

Choose a reason for hiding this comment

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

what's the reason to change them to hard coded names here? I understand they should be equivalent? Or keep the getName() while note the resolved name as comments?

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 equivalent. I changed because type name for decimal is not accessible and requires creation of a 'fake' logical-type.

LogicalTypes.decimal(1).getName()

I can revert to the old style if that's prefered.

Copy link
Contributor

Choose a reason for hiding this comment

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

both are fine, if choose to go with resolved names, good to add a comment these come from corresponding avro LogicalTypes implementations' getName()

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This code has been refactored to accomodate conversion from avro decimal type to BQ NUMERIC/BIGNUMERIC

Comment on lines +192 to +200
.setScale(type.getScale(), RoundingMode.DOWN)
.round(new MathContext(type.getPrecision(), RoundingMode.DOWN));
Copy link
Contributor

Choose a reason for hiding this comment

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

I understand previously it only accepts a ByteBuffer, now it's adding support of java.math.BigDecimal? If previously it would fail or it's not lose precision compared to the existing behavior I think it's fine.

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, just a nit. Will merge after tests passed

@Abacn Abacn merged commit 72102b5 into apache:master Jan 27, 2025
14 of 16 checks passed
tomstepp pushed a commit to tomstepp/apache-beam that referenced this pull request Feb 3, 2025
* [java] Fix avro logical-types conversions for BQ storage

Most of the avro logical-type to BQ are broken.

Add support for both joda and java time to ensure compatibility with
older avro versions

* expected int raw type for time-millis

* Remove unused qualifier

* Fix avro numeric convertion

* Add support for parametrized NUMERIC and BIGNUMERIC
VardhanThigle pushed a commit to VardhanThigle/beam that referenced this pull request Mar 21, 2025
* [java] Fix avro logical-types conversions for BQ storage

Most of the avro logical-type to BQ are broken.

Add support for both joda and java time to ensure compatibility with
older avro versions

* expected int raw type for time-millis

* Remove unused qualifier

* Fix avro numeric convertion

* Add support for parametrized NUMERIC and BIGNUMERIC
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.

2 participants