Skip to content

Disable logical type cast of fastavro #34603

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

Open
wants to merge 13 commits into
base: master
Choose a base branch
from
Open

Disable logical type cast of fastavro #34603

wants to merge 13 commits into from

Conversation

DKER2
Copy link
Contributor

@DKER2 DKER2 commented Apr 10, 2025

fix #31656

The bug is caused by the fact that when fastavro library read avro file with logicaltype, it will try to cast it to native type in python. For example, when avro field has long type and logical type is timestamp-millis, they will try cast orignal timestamp into datetime. Example cast from fastavro is attached in the image

image


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.

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 or the workflows README to see a list of phrases to trigger workflows.

Copy link
Contributor

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

R: @claudevdm for label python.

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

@DKER2 DKER2 marked this pull request as draft April 10, 2025 16:26
@DKER2 DKER2 marked this pull request as ready for review April 10, 2025 17:02
@DKER2
Copy link
Contributor Author

DKER2 commented Apr 11, 2025

When I am trying to add test coverage by adding logical type field in RECORDS. This unit test failed, but by reading the test, I do not understand what is 14.5 constant. So I have not add in tests

@claudevdm
Copy link
Collaborator

When I am trying to add test coverage by adding logical type field in RECORDS. This unit test failed, but by reading the test, I do not understand what is 14.5 constant. So I have not add in tests

Can you just extend or use a new list of records to test this fix (don't change the shared RECORDS)?

@@ -163,6 +163,8 @@ def __init__(
super().__init__()
self._source = _FastAvroSource(
file_pattern, min_bundle_size, validate=validate)
# Disable fastavro's automatic logical type conversion
fastavro.read.LOGICAL_READERS.clear()
Copy link
Collaborator

Choose a reason for hiding this comment

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

This will remove all avro LOGICAL_READERS in both the case where someone outputs as_row, or not meaning whatever the atomic value of the logical type is will be passed and all other info will be lost..

This might break for people who depend on the current LOGICAL_READERS behavior when as_row=False. If we decide to pass timsetamp-millis as integers, a more targeted fix would be to remove the LOGICAL_READER for 'long-timestamp-millis' only if as_row=True.

Also worth considering whether we want to treat the "timestamp-millis" as a Timestamp like mentioned in #31656 (comment).

@Abacn do you have an opinion whether the "timestamp-millis" type should be treated as an integer or a "Timestamp" type?

Copy link
Contributor

Choose a reason for hiding this comment

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

Yeah agree, a generic suggestion is to avoid breaking change to users.

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.

[Bug]: Python TypeError when converting Avro logicalType timestamp-millis to Beam Schema
3 participants