Skip to content
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

[Bug]: Python Runtime Type Introspection Mishandles PEP 585 Generics #23366

Closed
hmc-cs-mdrissi opened this issue Sep 26, 2022 · 7 comments
Closed
Labels
bug core done & done Issue has been reviewed after it was closed for verification, followups, etc. good first issue P2 python

Comments

@hmc-cs-mdrissi
Copy link
Contributor

What happened?

from typing import Tuple, TypeVar

from apache_beam.typehints.typehints import TypeVariable, is_consistent_with

K = TypeVariable("K")
V = TypeVariable("V")

A = TypeVar("A")
B = TypeVar("B")

print(is_consistent_with(tuple[int, str], Tuple[int, str]))
print(is_consistent_with(tuple[K, V], Tuple[int, str]))
print(is_consistent_with(tuple[A, B], Tuple[int, str]))

All of these print False. This is because is_consistent_with/normalize type machinery in beam does not handle tuple vs typing.Tuple/dict vs typing.Dict/list vs typing.List/etc equivalents. This leads to beam's runtime type checking crashing on valid code like,

import apache_beam as beam

def add_parity(n: int) -> tuple[bool, int]:
    return (n % 2 == 0, n)

with beam.Pipeline() as p:
    numbers = p | "Create" >> beam.Create([1, 2, 3, 4, 5])
    numbers = numbers | beam.Map(add_parity)
    _ = numbers | beam.GroupByKey() | beam.Map(print)

Here add_parity returns tuple, but GroupByKey expects Tuple. python 3.9+ recommends using tuple and some tools will even automatically replace Tuple -> tuple. So at moment workarounds are either stick with typing.Tuple or disable beam runtime type checking.

An example of assumption of typing.X generics that fails for pep 585 generics is here. getattr(x, '__module__', None) == 'typing' is not true for builtin generic equivalents as they come from there original source which may be various modules (most of basic ones are 'builtins'). There may be other code that needs changes as tuple[int, str] != Tuple[int, str] (equivalent but cpython they are handled by different types technically).

Issue Priority

Priority: 2

Issue Component

Component: sdk-py-core

@TheNeuralBit
Copy link
Member

Hi @hmc-cs-mdrissi, can you share how exactly it fails? Do you get a helpful error?

I'm not sure we can work on this in the short-term, but if the error is confusing we might at least do a quick change to make the error message more helpful.

@hmc-cs-mdrissi
Copy link
Contributor Author

The full error message is,

Traceback (most recent call last):
  File "/Users/mdrissi/Snapchat/Dev/training-platform/scratch/beam_type_issue.py", line 11, in <module>
    _ = numbers | beam.GroupByKey() | beam.Map(print)
  File "/Users/mdrissi/Snapchat/Dev/.virtual_envs/bento/lib/python3.9/site-packages/apache_beam/pvalue.py", line 137, in __or__
    return self.pipeline.apply(ptransform, self)
  File "/Users/mdrissi/Snapchat/Dev/.virtual_envs/bento/lib/python3.9/site-packages/apache_beam/pipeline.py", line 706, in apply
    transform.type_check_inputs(pvalueish)
  File "/Users/mdrissi/Snapchat/Dev/.virtual_envs/bento/lib/python3.9/site-packages/apache_beam/transforms/ptransform.py", line 457, in type_check_inputs
    self.type_check_inputs_or_outputs(pvalueish, 'input')
  File "/Users/mdrissi/Snapchat/Dev/.virtual_envs/bento/lib/python3.9/site-packages/apache_beam/transforms/ptransform.py", line 486, in type_check_inputs_or_outputs
    raise TypeCheckError(
apache_beam.typehints.decorators.TypeCheckError: Input type hint violation at GroupByKey: expected Tuple[TypeVariable[K], TypeVariable[V]], got tuple[bool, int]
Full type hint:
IOTypeHints[inputs=((Tuple[TypeVariable[K], TypeVariable[V]],), {}), outputs=((Tuple[TypeVariable[K], Iterable[TypeVariable[V]]],), {})]
File "<frozen importlib._bootstrap>", line 680, in _load_unlocked
File "<frozen importlib._bootstrap_external>", line 850, in exec_module
File "<frozen importlib._bootstrap>", line 228, in _call_with_frames_removed
File "/Users/mdrissi/Snapchat/Dev/.virtual_envs/bento/lib/python3.9/site-packages/apache_beam/transforms/core.py", line 2637, in <module>
    class GroupByKey(PTransform):
File "/Users/mdrissi/Snapchat/Dev/.virtual_envs/bento/lib/python3.9/site-packages/apache_beam/typehints/decorators.py", line 775, in annotate_input_types
    th = getattr(f, '_type_hints', IOTypeHints.empty()).with_input_types(

based on:
  IOTypeHints[inputs=None, outputs=((Tuple[TypeVariable[K], Iterable[TypeVariable[V]]],), {})]
  File "<frozen importlib._bootstrap>", line 680, in _load_unlocked
  File "<frozen importlib._bootstrap_external>", line 850, in exec_module
  File "<frozen importlib._bootstrap>", line 228, in _call_with_frames_removed
  File "/Users/mdrissi/Snapchat/Dev/.virtual_envs/bento/lib/python3.9/site-packages/apache_beam/transforms/core.py", line 2637, in <module>
      class GroupByKey(PTransform):
  File "/Users/mdrissi/Snapchat/Dev/.virtual_envs/bento/lib/python3.9/site-packages/apache_beam/typehints/decorators.py", line 863, in annotate_output_types
      f._type_hints = th.with_output_types(return_type_hint)  # pylint: disable=protected-access

With key line being Input type hint violation at GroupByKey: expected Tuple[TypeVariable[K], TypeVariable[V]], got tuple[bool, int] caused by is_consistent_with(Tuple[K, V], tuple[bool, int]) returning false.

@TheNeuralBit
Copy link
Member

Got it thanks. We could certainly do better by just raising an error in normalization stating that PEP 585 typehints are not currently supported. That could be a first step and should be pretty low effort, could be a good task for someone wanting to learn about our typehint infrastructure.

@jrmccluskey
Copy link
Contributor

@github-actions github-actions bot added this to the 2.46.0 Release milestone Jan 24, 2023
@damccorm damccorm added the done & done Issue has been reviewed after it was closed for verification, followups, etc. label Jan 24, 2023
@ssjhv
Copy link

ssjhv commented Mar 16, 2023

Is PEP-585 supposed to be fully handled now? I noticed that imports from collections.abc raised typehint errors.

Example:

from collections.abc import Iterator
import apache_beam as beam

def add_range(n: int) -> Iterator[int]:
  for i in range(n):
    yield i

with beam.Pipeline() as p:
    numbers = p | "Create" >> beam.Create([1, 2, 3, 4, 5])
    numbers = numbers | beam.FlatMap(add_range)
    _ = numbers | beam.combiners.Count.PerElement() | beam.Map(print
---------------------------------------------------------------------------
ValueError                                Traceback (most recent call last)
/usr/local/lib/python3.9/dist-packages/apache_beam/transforms/core.py in default_type_hints(self)
    959     try:
--> 960       type_hints = type_hints.strip_iterable()
    961     except ValueError as e:

9 frames
ValueError: collections.abc.Iterator[int] is not iterable

During handling of the above exception, another exception occurred:

TypeCheckError                            Traceback (most recent call last)
/usr/local/lib/python3.9/dist-packages/apache_beam/transforms/core.py in default_type_hints(self)
    960       type_hints = type_hints.strip_iterable()
    961     except ValueError as e:
--> 962       raise TypeCheckError(
    963           'Return value not iterable: %s: %s' %
    964           (self.display_data()['fn'].value, e))

TypeCheckError: Return value not iterable: add_range: collections.abc.Iterator[int] is not iterable

@jrmccluskey
Copy link
Contributor

@ssjhv this will be fixed in the next release, #25798 and #25866 fix this since the initial PR missed the collections types

@ssjhv
Copy link

ssjhv commented Mar 16, 2023

@jrmccluskey Thank you for the quick reply! Looking forward to the next release 👍

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug core done & done Issue has been reviewed after it was closed for verification, followups, etc. good first issue P2 python
Projects
None yet
Development

No branches or pull requests

5 participants