-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Description
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