Skip to content

Conversation

shunping
Copy link
Collaborator

fixes #36013

@Abacn
Copy link
Contributor

Abacn commented Aug 29, 2025

Can we confirm it is caused by #34695? This suggests apache_beam.typehints.schemas.LogicalType.register_logical_type is fundamentally broken in the case of "python sdk worker running in docker env". We should fix the root cause

@Abacn Abacn added this to the 2.68.0 Release milestone Sep 2, 2025
@shunping shunping marked this pull request as ready for review September 3, 2025 04:17
@shunping
Copy link
Collaborator Author

shunping commented Sep 3, 2025

cc'ed @claudevdm

Copy link
Contributor

github-actions bot commented Sep 3, 2025

Checks are failing. Will not request review until checks are succeeding. If you'd like to override that behavior, comment assign set of reviewers

Copy link
Contributor

github-actions bot commented Sep 3, 2025

Assigning reviewers:

R: @claudevdm for label python.

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

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

@@ -690,7 +690,16 @@ def add(self, urn, logical_type):
self.by_language_type[logical_type.language_type()] = logical_type

def get_logical_type_by_urn(self, urn):
return self.by_urn.get(urn, None)
logical_type = self.by_urn.get(urn, None)
Copy link
Collaborator

Choose a reason for hiding this comment

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

Why don't we just register the beam:logical_type:javasdk_date:v1 and beam:logical_type:javasdk_time:v1 logical types in shcemas.py? Or do they they conflict with something?

Copy link
Collaborator Author

@shunping shunping Sep 3, 2025

Choose a reason for hiding this comment

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

The import here is for importing classes and logical type registration. We need those classes to be imported first before registration.

Also, we cannot put the import at the beginning of schemas.py due to circular dependency.

Copy link
Collaborator

Choose a reason for hiding this comment

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

Oh I meant we also define them in schemas.py, like other logical types in schemas.py

class MillisInstant(NoArgumentLogicalType[Timestamp, np.int64]):

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

I discussed with @Abacn who initially implemented these JDBC logical types. We think we can move them in schemas.py.

Note that we will inevitably introduce this non-portable piece into schemas.py, either by the import method in the current PR or by moving those logical types directly into the file.

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 we can revert "import apache_beam.io.jdbc" here now?

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Nice catch. Fixed.

@github-actions github-actions bot added io and removed io labels Sep 3, 2025
@github-actions github-actions bot added io and removed io labels Sep 3, 2025
@github-actions github-actions bot added io and removed io labels Sep 3, 2025
@@ -96,10 +95,11 @@
from apache_beam.transforms.external import BeamJarExpansionService
from apache_beam.transforms.external import ExternalTransform
from apache_beam.transforms.external import NamedTupleBasedPayloadBuilder
from apache_beam.typehints.schemas import JdbcDateType # pylint: disable=unused-import
Copy link
Collaborator

Choose a reason for hiding this comment

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

Why does this need to be imported?

Copy link
Contributor

Choose a reason for hiding this comment

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

To keep the backward compatibility when user uses from apache_beam.io.jdbc import JdbcDateType in their own scripts

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

To maintain backward compatibility. We don't want to introduce breaking changes for users who were previously importing these logical types directly from apache_beam.io.jdbc.

Copy link
Collaborator

Choose a reason for hiding this comment

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

Got it.

Copy link
Contributor

Choose a reason for hiding this comment

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

we probably want to add this comments to justify "# pylint: disable=unused-import" as well

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

Ack. We will eventually revert/remove this temporary fix, so let's get it in and unblock the release first. Thanks both!

@shunping shunping merged commit 67eaded into apache:master Sep 3, 2025
90 of 91 checks passed
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]: JDBC logical type not found when running a Python pipeline in docker env
3 participants